blob: 1809cf574c55b12cecf96cb44b25a60c3e407972 [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.master, class: HMaster, interface: TableDescriptorGetter">
<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.master;</span>
<span class="source-line-no">019</span><span id="line-19"></span>
<span class="source-line-no">020</span><span id="line-20">import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK;</span>
<span class="source-line-no">021</span><span id="line-21">import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;</span>
<span class="source-line-no">022</span><span id="line-22">import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_COORDINATED_BY_ZK;</span>
<span class="source-line-no">023</span><span id="line-23">import static org.apache.hadoop.hbase.master.cleaner.HFileCleaner.CUSTOM_POOL_SIZE;</span>
<span class="source-line-no">024</span><span id="line-24">import static org.apache.hadoop.hbase.util.DNS.MASTER_HOSTNAME_KEY;</span>
<span class="source-line-no">025</span><span id="line-25"></span>
<span class="source-line-no">026</span><span id="line-26">import com.google.errorprone.annotations.RestrictedApi;</span>
<span class="source-line-no">027</span><span id="line-27">import io.opentelemetry.api.trace.Span;</span>
<span class="source-line-no">028</span><span id="line-28">import io.opentelemetry.api.trace.StatusCode;</span>
<span class="source-line-no">029</span><span id="line-29">import io.opentelemetry.context.Scope;</span>
<span class="source-line-no">030</span><span id="line-30">import java.io.IOException;</span>
<span class="source-line-no">031</span><span id="line-31">import java.io.InterruptedIOException;</span>
<span class="source-line-no">032</span><span id="line-32">import java.lang.reflect.Constructor;</span>
<span class="source-line-no">033</span><span id="line-33">import java.lang.reflect.InvocationTargetException;</span>
<span class="source-line-no">034</span><span id="line-34">import java.net.InetAddress;</span>
<span class="source-line-no">035</span><span id="line-35">import java.net.InetSocketAddress;</span>
<span class="source-line-no">036</span><span id="line-36">import java.net.UnknownHostException;</span>
<span class="source-line-no">037</span><span id="line-37">import java.time.Instant;</span>
<span class="source-line-no">038</span><span id="line-38">import java.time.ZoneId;</span>
<span class="source-line-no">039</span><span id="line-39">import java.time.format.DateTimeFormatter;</span>
<span class="source-line-no">040</span><span id="line-40">import java.util.ArrayList;</span>
<span class="source-line-no">041</span><span id="line-41">import java.util.Arrays;</span>
<span class="source-line-no">042</span><span id="line-42">import java.util.Collection;</span>
<span class="source-line-no">043</span><span id="line-43">import java.util.Collections;</span>
<span class="source-line-no">044</span><span id="line-44">import java.util.Comparator;</span>
<span class="source-line-no">045</span><span id="line-45">import java.util.EnumSet;</span>
<span class="source-line-no">046</span><span id="line-46">import java.util.HashMap;</span>
<span class="source-line-no">047</span><span id="line-47">import java.util.HashSet;</span>
<span class="source-line-no">048</span><span id="line-48">import java.util.Iterator;</span>
<span class="source-line-no">049</span><span id="line-49">import java.util.LinkedList;</span>
<span class="source-line-no">050</span><span id="line-50">import java.util.List;</span>
<span class="source-line-no">051</span><span id="line-51">import java.util.Map;</span>
<span class="source-line-no">052</span><span id="line-52">import java.util.Objects;</span>
<span class="source-line-no">053</span><span id="line-53">import java.util.Optional;</span>
<span class="source-line-no">054</span><span id="line-54">import java.util.Set;</span>
<span class="source-line-no">055</span><span id="line-55">import java.util.concurrent.ExecutionException;</span>
<span class="source-line-no">056</span><span id="line-56">import java.util.concurrent.Future;</span>
<span class="source-line-no">057</span><span id="line-57">import java.util.concurrent.Semaphore;</span>
<span class="source-line-no">058</span><span id="line-58">import java.util.concurrent.TimeUnit;</span>
<span class="source-line-no">059</span><span id="line-59">import java.util.concurrent.TimeoutException;</span>
<span class="source-line-no">060</span><span id="line-60">import java.util.concurrent.atomic.AtomicInteger;</span>
<span class="source-line-no">061</span><span id="line-61">import java.util.regex.Pattern;</span>
<span class="source-line-no">062</span><span id="line-62">import java.util.stream.Collectors;</span>
<span class="source-line-no">063</span><span id="line-63">import javax.servlet.http.HttpServlet;</span>
<span class="source-line-no">064</span><span id="line-64">import org.apache.commons.lang3.StringUtils;</span>
<span class="source-line-no">065</span><span id="line-65">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">066</span><span id="line-66">import org.apache.hadoop.fs.FSDataInputStream;</span>
<span class="source-line-no">067</span><span id="line-67">import org.apache.hadoop.fs.FSDataOutputStream;</span>
<span class="source-line-no">068</span><span id="line-68">import org.apache.hadoop.fs.Path;</span>
<span class="source-line-no">069</span><span id="line-69">import org.apache.hadoop.hbase.CatalogFamilyFormat;</span>
<span class="source-line-no">070</span><span id="line-70">import org.apache.hadoop.hbase.Cell;</span>
<span class="source-line-no">071</span><span id="line-71">import org.apache.hadoop.hbase.CellBuilderFactory;</span>
<span class="source-line-no">072</span><span id="line-72">import org.apache.hadoop.hbase.CellBuilderType;</span>
<span class="source-line-no">073</span><span id="line-73">import org.apache.hadoop.hbase.ClusterId;</span>
<span class="source-line-no">074</span><span id="line-74">import org.apache.hadoop.hbase.ClusterMetrics;</span>
<span class="source-line-no">075</span><span id="line-75">import org.apache.hadoop.hbase.ClusterMetrics.Option;</span>
<span class="source-line-no">076</span><span id="line-76">import org.apache.hadoop.hbase.ClusterMetricsBuilder;</span>
<span class="source-line-no">077</span><span id="line-77">import org.apache.hadoop.hbase.DoNotRetryIOException;</span>
<span class="source-line-no">078</span><span id="line-78">import org.apache.hadoop.hbase.HBaseIOException;</span>
<span class="source-line-no">079</span><span id="line-79">import org.apache.hadoop.hbase.HBaseInterfaceAudience;</span>
<span class="source-line-no">080</span><span id="line-80">import org.apache.hadoop.hbase.HBaseServerBase;</span>
<span class="source-line-no">081</span><span id="line-81">import org.apache.hadoop.hbase.HConstants;</span>
<span class="source-line-no">082</span><span id="line-82">import org.apache.hadoop.hbase.HRegionLocation;</span>
<span class="source-line-no">083</span><span id="line-83">import org.apache.hadoop.hbase.InvalidFamilyOperationException;</span>
<span class="source-line-no">084</span><span id="line-84">import org.apache.hadoop.hbase.MasterNotRunningException;</span>
<span class="source-line-no">085</span><span id="line-85">import org.apache.hadoop.hbase.MetaTableAccessor;</span>
<span class="source-line-no">086</span><span id="line-86">import org.apache.hadoop.hbase.NamespaceDescriptor;</span>
<span class="source-line-no">087</span><span id="line-87">import org.apache.hadoop.hbase.PleaseHoldException;</span>
<span class="source-line-no">088</span><span id="line-88">import org.apache.hadoop.hbase.PleaseRestartMasterException;</span>
<span class="source-line-no">089</span><span id="line-89">import org.apache.hadoop.hbase.RegionMetrics;</span>
<span class="source-line-no">090</span><span id="line-90">import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;</span>
<span class="source-line-no">091</span><span id="line-91">import org.apache.hadoop.hbase.ScheduledChore;</span>
<span class="source-line-no">092</span><span id="line-92">import org.apache.hadoop.hbase.ServerMetrics;</span>
<span class="source-line-no">093</span><span id="line-93">import org.apache.hadoop.hbase.ServerName;</span>
<span class="source-line-no">094</span><span id="line-94">import org.apache.hadoop.hbase.ServerTask;</span>
<span class="source-line-no">095</span><span id="line-95">import org.apache.hadoop.hbase.ServerTaskBuilder;</span>
<span class="source-line-no">096</span><span id="line-96">import org.apache.hadoop.hbase.TableName;</span>
<span class="source-line-no">097</span><span id="line-97">import org.apache.hadoop.hbase.TableNotDisabledException;</span>
<span class="source-line-no">098</span><span id="line-98">import org.apache.hadoop.hbase.TableNotFoundException;</span>
<span class="source-line-no">099</span><span id="line-99">import org.apache.hadoop.hbase.UnknownRegionException;</span>
<span class="source-line-no">100</span><span id="line-100">import org.apache.hadoop.hbase.client.BalanceRequest;</span>
<span class="source-line-no">101</span><span id="line-101">import org.apache.hadoop.hbase.client.BalanceResponse;</span>
<span class="source-line-no">102</span><span id="line-102">import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;</span>
<span class="source-line-no">103</span><span id="line-103">import org.apache.hadoop.hbase.client.CompactionState;</span>
<span class="source-line-no">104</span><span id="line-104">import org.apache.hadoop.hbase.client.MasterSwitchType;</span>
<span class="source-line-no">105</span><span id="line-105">import org.apache.hadoop.hbase.client.NormalizeTableFilterParams;</span>
<span class="source-line-no">106</span><span id="line-106">import org.apache.hadoop.hbase.client.Put;</span>
<span class="source-line-no">107</span><span id="line-107">import org.apache.hadoop.hbase.client.RegionInfo;</span>
<span class="source-line-no">108</span><span id="line-108">import org.apache.hadoop.hbase.client.RegionInfoBuilder;</span>
<span class="source-line-no">109</span><span id="line-109">import org.apache.hadoop.hbase.client.RegionStatesCount;</span>
<span class="source-line-no">110</span><span id="line-110">import org.apache.hadoop.hbase.client.ResultScanner;</span>
<span class="source-line-no">111</span><span id="line-111">import org.apache.hadoop.hbase.client.Scan;</span>
<span class="source-line-no">112</span><span id="line-112">import org.apache.hadoop.hbase.client.TableDescriptor;</span>
<span class="source-line-no">113</span><span id="line-113">import org.apache.hadoop.hbase.client.TableDescriptorBuilder;</span>
<span class="source-line-no">114</span><span id="line-114">import org.apache.hadoop.hbase.client.TableState;</span>
<span class="source-line-no">115</span><span id="line-115">import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;</span>
<span class="source-line-no">116</span><span id="line-116">import org.apache.hadoop.hbase.exceptions.DeserializationException;</span>
<span class="source-line-no">117</span><span id="line-117">import org.apache.hadoop.hbase.exceptions.MasterStoppedException;</span>
<span class="source-line-no">118</span><span id="line-118">import org.apache.hadoop.hbase.executor.ExecutorType;</span>
<span class="source-line-no">119</span><span id="line-119">import org.apache.hadoop.hbase.favored.FavoredNodesManager;</span>
<span class="source-line-no">120</span><span id="line-120">import org.apache.hadoop.hbase.http.HttpServer;</span>
<span class="source-line-no">121</span><span id="line-121">import org.apache.hadoop.hbase.http.InfoServer;</span>
<span class="source-line-no">122</span><span id="line-122">import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;</span>
<span class="source-line-no">123</span><span id="line-123">import org.apache.hadoop.hbase.ipc.RpcServer;</span>
<span class="source-line-no">124</span><span id="line-124">import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;</span>
<span class="source-line-no">125</span><span id="line-125">import org.apache.hadoop.hbase.log.HBaseMarkers;</span>
<span class="source-line-no">126</span><span id="line-126">import org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode;</span>
<span class="source-line-no">127</span><span id="line-127">import org.apache.hadoop.hbase.master.assignment.AssignmentManager;</span>
<span class="source-line-no">128</span><span id="line-128">import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;</span>
<span class="source-line-no">129</span><span id="line-129">import org.apache.hadoop.hbase.master.assignment.RegionStateNode;</span>
<span class="source-line-no">130</span><span id="line-130">import org.apache.hadoop.hbase.master.assignment.RegionStateStore;</span>
<span class="source-line-no">131</span><span id="line-131">import org.apache.hadoop.hbase.master.assignment.RegionStates;</span>
<span class="source-line-no">132</span><span id="line-132">import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;</span>
<span class="source-line-no">133</span><span id="line-133">import org.apache.hadoop.hbase.master.balancer.BalancerChore;</span>
<span class="source-line-no">134</span><span id="line-134">import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;</span>
<span class="source-line-no">135</span><span id="line-135">import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;</span>
<span class="source-line-no">136</span><span id="line-136">import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;</span>
<span class="source-line-no">137</span><span id="line-137">import org.apache.hadoop.hbase.master.balancer.LoadBalancerStateStore;</span>
<span class="source-line-no">138</span><span id="line-138">import org.apache.hadoop.hbase.master.balancer.MaintenanceLoadBalancer;</span>
<span class="source-line-no">139</span><span id="line-139">import org.apache.hadoop.hbase.master.cleaner.DirScanPool;</span>
<span class="source-line-no">140</span><span id="line-140">import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;</span>
<span class="source-line-no">141</span><span id="line-141">import org.apache.hadoop.hbase.master.cleaner.LogCleaner;</span>
<span class="source-line-no">142</span><span id="line-142">import org.apache.hadoop.hbase.master.cleaner.ReplicationBarrierCleaner;</span>
<span class="source-line-no">143</span><span id="line-143">import org.apache.hadoop.hbase.master.cleaner.SnapshotCleanerChore;</span>
<span class="source-line-no">144</span><span id="line-144">import org.apache.hadoop.hbase.master.hbck.HbckChore;</span>
<span class="source-line-no">145</span><span id="line-145">import org.apache.hadoop.hbase.master.http.MasterDumpServlet;</span>
<span class="source-line-no">146</span><span id="line-146">import org.apache.hadoop.hbase.master.http.MasterRedirectServlet;</span>
<span class="source-line-no">147</span><span id="line-147">import org.apache.hadoop.hbase.master.http.MasterStatusServlet;</span>
<span class="source-line-no">148</span><span id="line-148">import org.apache.hadoop.hbase.master.http.api_v1.ResourceConfigFactory;</span>
<span class="source-line-no">149</span><span id="line-149">import org.apache.hadoop.hbase.master.http.hbck.HbckConfigFactory;</span>
<span class="source-line-no">150</span><span id="line-150">import org.apache.hadoop.hbase.master.janitor.CatalogJanitor;</span>
<span class="source-line-no">151</span><span id="line-151">import org.apache.hadoop.hbase.master.locking.LockManager;</span>
<span class="source-line-no">152</span><span id="line-152">import org.apache.hadoop.hbase.master.migrate.RollingUpgradeChore;</span>
<span class="source-line-no">153</span><span id="line-153">import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;</span>
<span class="source-line-no">154</span><span id="line-154">import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerManager;</span>
<span class="source-line-no">155</span><span id="line-155">import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerStateStore;</span>
<span class="source-line-no">156</span><span id="line-156">import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;</span>
<span class="source-line-no">157</span><span id="line-157">import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;</span>
<span class="source-line-no">158</span><span id="line-158">import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;</span>
<span class="source-line-no">159</span><span id="line-159">import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;</span>
<span class="source-line-no">160</span><span id="line-160">import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;</span>
<span class="source-line-no">161</span><span id="line-161">import org.apache.hadoop.hbase.master.procedure.FlushTableProcedure;</span>
<span class="source-line-no">162</span><span id="line-162">import org.apache.hadoop.hbase.master.procedure.InitMetaProcedure;</span>
<span class="source-line-no">163</span><span id="line-163">import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;</span>
<span class="source-line-no">164</span><span id="line-164">import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;</span>
<span class="source-line-no">165</span><span id="line-165">import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;</span>
<span class="source-line-no">166</span><span id="line-166">import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;</span>
<span class="source-line-no">167</span><span id="line-167">import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil.NonceProcedureRunnable;</span>
<span class="source-line-no">168</span><span id="line-168">import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;</span>
<span class="source-line-no">169</span><span id="line-169">import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;</span>
<span class="source-line-no">170</span><span id="line-170">import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;</span>
<span class="source-line-no">171</span><span id="line-171">import org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure;</span>
<span class="source-line-no">172</span><span id="line-172">import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;</span>
<span class="source-line-no">173</span><span id="line-173">import org.apache.hadoop.hbase.master.procedure.TruncateRegionProcedure;</span>
<span class="source-line-no">174</span><span id="line-174">import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;</span>
<span class="source-line-no">175</span><span id="line-175">import org.apache.hadoop.hbase.master.region.MasterRegion;</span>
<span class="source-line-no">176</span><span id="line-176">import org.apache.hadoop.hbase.master.region.MasterRegionFactory;</span>
<span class="source-line-no">177</span><span id="line-177">import org.apache.hadoop.hbase.master.replication.AbstractPeerProcedure;</span>
<span class="source-line-no">178</span><span id="line-178">import org.apache.hadoop.hbase.master.replication.AddPeerProcedure;</span>
<span class="source-line-no">179</span><span id="line-179">import org.apache.hadoop.hbase.master.replication.DisablePeerProcedure;</span>
<span class="source-line-no">180</span><span id="line-180">import org.apache.hadoop.hbase.master.replication.EnablePeerProcedure;</span>
<span class="source-line-no">181</span><span id="line-181">import org.apache.hadoop.hbase.master.replication.MigrateReplicationQueueFromZkToTableProcedure;</span>
<span class="source-line-no">182</span><span id="line-182">import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure;</span>
<span class="source-line-no">183</span><span id="line-183">import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;</span>
<span class="source-line-no">184</span><span id="line-184">import org.apache.hadoop.hbase.master.replication.ReplicationPeerModificationStateStore;</span>
<span class="source-line-no">185</span><span id="line-185">import org.apache.hadoop.hbase.master.replication.SyncReplicationReplayWALManager;</span>
<span class="source-line-no">186</span><span id="line-186">import org.apache.hadoop.hbase.master.replication.TransitPeerSyncReplicationStateProcedure;</span>
<span class="source-line-no">187</span><span id="line-187">import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;</span>
<span class="source-line-no">188</span><span id="line-188">import org.apache.hadoop.hbase.master.slowlog.SlowLogMasterService;</span>
<span class="source-line-no">189</span><span id="line-189">import org.apache.hadoop.hbase.master.snapshot.SnapshotCleanupStateStore;</span>
<span class="source-line-no">190</span><span id="line-190">import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;</span>
<span class="source-line-no">191</span><span id="line-191">import org.apache.hadoop.hbase.master.waleventtracker.WALEventTrackerTableCreator;</span>
<span class="source-line-no">192</span><span id="line-192">import org.apache.hadoop.hbase.master.zksyncer.MasterAddressSyncer;</span>
<span class="source-line-no">193</span><span id="line-193">import org.apache.hadoop.hbase.master.zksyncer.MetaLocationSyncer;</span>
<span class="source-line-no">194</span><span id="line-194">import org.apache.hadoop.hbase.mob.MobFileCleanerChore;</span>
<span class="source-line-no">195</span><span id="line-195">import org.apache.hadoop.hbase.mob.MobFileCompactionChore;</span>
<span class="source-line-no">196</span><span id="line-196">import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;</span>
<span class="source-line-no">197</span><span id="line-197">import org.apache.hadoop.hbase.monitoring.MonitoredTask;</span>
<span class="source-line-no">198</span><span id="line-198">import org.apache.hadoop.hbase.monitoring.TaskGroup;</span>
<span class="source-line-no">199</span><span id="line-199">import org.apache.hadoop.hbase.monitoring.TaskMonitor;</span>
<span class="source-line-no">200</span><span id="line-200">import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;</span>
<span class="source-line-no">201</span><span id="line-201">import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;</span>
<span class="source-line-no">202</span><span id="line-202">import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;</span>
<span class="source-line-no">203</span><span id="line-203">import org.apache.hadoop.hbase.procedure2.LockedResource;</span>
<span class="source-line-no">204</span><span id="line-204">import org.apache.hadoop.hbase.procedure2.Procedure;</span>
<span class="source-line-no">205</span><span id="line-205">import org.apache.hadoop.hbase.procedure2.ProcedureEvent;</span>
<span class="source-line-no">206</span><span id="line-206">import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;</span>
<span class="source-line-no">207</span><span id="line-207">import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;</span>
<span class="source-line-no">208</span><span id="line-208">import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;</span>
<span class="source-line-no">209</span><span id="line-209">import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;</span>
<span class="source-line-no">210</span><span id="line-210">import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureStoreListener;</span>
<span class="source-line-no">211</span><span id="line-211">import org.apache.hadoop.hbase.procedure2.store.region.RegionProcedureStore;</span>
<span class="source-line-no">212</span><span id="line-212">import org.apache.hadoop.hbase.quotas.MasterQuotaManager;</span>
<span class="source-line-no">213</span><span id="line-213">import org.apache.hadoop.hbase.quotas.MasterQuotasObserver;</span>
<span class="source-line-no">214</span><span id="line-214">import org.apache.hadoop.hbase.quotas.QuotaObserverChore;</span>
<span class="source-line-no">215</span><span id="line-215">import org.apache.hadoop.hbase.quotas.QuotaTableUtil;</span>
<span class="source-line-no">216</span><span id="line-216">import org.apache.hadoop.hbase.quotas.QuotaUtil;</span>
<span class="source-line-no">217</span><span id="line-217">import org.apache.hadoop.hbase.quotas.SnapshotQuotaObserverChore;</span>
<span class="source-line-no">218</span><span id="line-218">import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;</span>
<span class="source-line-no">219</span><span id="line-219">import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;</span>
<span class="source-line-no">220</span><span id="line-220">import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifier;</span>
<span class="source-line-no">221</span><span id="line-221">import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifierFactory;</span>
<span class="source-line-no">222</span><span id="line-222">import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;</span>
<span class="source-line-no">223</span><span id="line-223">import org.apache.hadoop.hbase.regionserver.HRegionServer;</span>
<span class="source-line-no">224</span><span id="line-224">import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;</span>
<span class="source-line-no">225</span><span id="line-225">import org.apache.hadoop.hbase.regionserver.storefiletracker.ModifyColumnFamilyStoreFileTrackerProcedure;</span>
<span class="source-line-no">226</span><span id="line-226">import org.apache.hadoop.hbase.regionserver.storefiletracker.ModifyTableStoreFileTrackerProcedure;</span>
<span class="source-line-no">227</span><span id="line-227">import org.apache.hadoop.hbase.replication.ReplicationException;</span>
<span class="source-line-no">228</span><span id="line-228">import org.apache.hadoop.hbase.replication.ReplicationLoadSource;</span>
<span class="source-line-no">229</span><span id="line-229">import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;</span>
<span class="source-line-no">230</span><span id="line-230">import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;</span>
<span class="source-line-no">231</span><span id="line-231">import org.apache.hadoop.hbase.replication.ReplicationUtils;</span>
<span class="source-line-no">232</span><span id="line-232">import org.apache.hadoop.hbase.replication.SyncReplicationState;</span>
<span class="source-line-no">233</span><span id="line-233">import org.apache.hadoop.hbase.replication.ZKReplicationQueueStorageForMigration;</span>
<span class="source-line-no">234</span><span id="line-234">import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;</span>
<span class="source-line-no">235</span><span id="line-235">import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;</span>
<span class="source-line-no">236</span><span id="line-236">import org.apache.hadoop.hbase.replication.master.ReplicationLogCleanerBarrier;</span>
<span class="source-line-no">237</span><span id="line-237">import org.apache.hadoop.hbase.replication.master.ReplicationSinkTrackerTableCreator;</span>
<span class="source-line-no">238</span><span id="line-238">import org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp;</span>
<span class="source-line-no">239</span><span id="line-239">import org.apache.hadoop.hbase.replication.regionserver.ReplicationSyncUp.ReplicationSyncUpToolInfo;</span>
<span class="source-line-no">240</span><span id="line-240">import org.apache.hadoop.hbase.rsgroup.RSGroupAdminEndpoint;</span>
<span class="source-line-no">241</span><span id="line-241">import org.apache.hadoop.hbase.rsgroup.RSGroupBasedLoadBalancer;</span>
<span class="source-line-no">242</span><span id="line-242">import org.apache.hadoop.hbase.rsgroup.RSGroupInfoManager;</span>
<span class="source-line-no">243</span><span id="line-243">import org.apache.hadoop.hbase.rsgroup.RSGroupUtil;</span>
<span class="source-line-no">244</span><span id="line-244">import org.apache.hadoop.hbase.security.AccessDeniedException;</span>
<span class="source-line-no">245</span><span id="line-245">import org.apache.hadoop.hbase.security.SecurityConstants;</span>
<span class="source-line-no">246</span><span id="line-246">import org.apache.hadoop.hbase.security.Superusers;</span>
<span class="source-line-no">247</span><span id="line-247">import org.apache.hadoop.hbase.security.UserProvider;</span>
<span class="source-line-no">248</span><span id="line-248">import org.apache.hadoop.hbase.trace.TraceUtil;</span>
<span class="source-line-no">249</span><span id="line-249">import org.apache.hadoop.hbase.util.Addressing;</span>
<span class="source-line-no">250</span><span id="line-250">import org.apache.hadoop.hbase.util.Bytes;</span>
<span class="source-line-no">251</span><span id="line-251">import org.apache.hadoop.hbase.util.CommonFSUtils;</span>
<span class="source-line-no">252</span><span id="line-252">import org.apache.hadoop.hbase.util.CoprocessorConfigurationUtil;</span>
<span class="source-line-no">253</span><span id="line-253">import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;</span>
<span class="source-line-no">254</span><span id="line-254">import org.apache.hadoop.hbase.util.FSTableDescriptors;</span>
<span class="source-line-no">255</span><span id="line-255">import org.apache.hadoop.hbase.util.FutureUtils;</span>
<span class="source-line-no">256</span><span id="line-256">import org.apache.hadoop.hbase.util.HBaseFsck;</span>
<span class="source-line-no">257</span><span id="line-257">import org.apache.hadoop.hbase.util.HFileArchiveUtil;</span>
<span class="source-line-no">258</span><span id="line-258">import org.apache.hadoop.hbase.util.IdLock;</span>
<span class="source-line-no">259</span><span id="line-259">import org.apache.hadoop.hbase.util.JVMClusterUtil;</span>
<span class="source-line-no">260</span><span id="line-260">import org.apache.hadoop.hbase.util.JsonMapper;</span>
<span class="source-line-no">261</span><span id="line-261">import org.apache.hadoop.hbase.util.ModifyRegionUtils;</span>
<span class="source-line-no">262</span><span id="line-262">import org.apache.hadoop.hbase.util.Pair;</span>
<span class="source-line-no">263</span><span id="line-263">import org.apache.hadoop.hbase.util.RetryCounter;</span>
<span class="source-line-no">264</span><span id="line-264">import org.apache.hadoop.hbase.util.RetryCounterFactory;</span>
<span class="source-line-no">265</span><span id="line-265">import org.apache.hadoop.hbase.util.TableDescriptorChecker;</span>
<span class="source-line-no">266</span><span id="line-266">import org.apache.hadoop.hbase.util.Threads;</span>
<span class="source-line-no">267</span><span id="line-267">import org.apache.hadoop.hbase.util.VersionInfo;</span>
<span class="source-line-no">268</span><span id="line-268">import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;</span>
<span class="source-line-no">269</span><span id="line-269">import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;</span>
<span class="source-line-no">270</span><span id="line-270">import org.apache.hadoop.hbase.zookeeper.ZKClusterId;</span>
<span class="source-line-no">271</span><span id="line-271">import org.apache.hadoop.hbase.zookeeper.ZKUtil;</span>
<span class="source-line-no">272</span><span id="line-272">import org.apache.hadoop.hbase.zookeeper.ZKWatcher;</span>
<span class="source-line-no">273</span><span id="line-273">import org.apache.hadoop.hbase.zookeeper.ZNodePaths;</span>
<span class="source-line-no">274</span><span id="line-274">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">275</span><span id="line-275">import org.apache.zookeeper.KeeperException;</span>
<span class="source-line-no">276</span><span id="line-276">import org.slf4j.Logger;</span>
<span class="source-line-no">277</span><span id="line-277">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">278</span><span id="line-278"></span>
<span class="source-line-no">279</span><span id="line-279">import org.apache.hbase.thirdparty.com.google.common.collect.Lists;</span>
<span class="source-line-no">280</span><span id="line-280">import org.apache.hbase.thirdparty.com.google.common.collect.Maps;</span>
<span class="source-line-no">281</span><span id="line-281">import org.apache.hbase.thirdparty.com.google.common.collect.Sets;</span>
<span class="source-line-no">282</span><span id="line-282">import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams;</span>
<span class="source-line-no">283</span><span id="line-283">import org.apache.hbase.thirdparty.com.google.common.io.Closeables;</span>
<span class="source-line-no">284</span><span id="line-284">import org.apache.hbase.thirdparty.com.google.gson.JsonParseException;</span>
<span class="source-line-no">285</span><span id="line-285">import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;</span>
<span class="source-line-no">286</span><span id="line-286">import org.apache.hbase.thirdparty.com.google.protobuf.Service;</span>
<span class="source-line-no">287</span><span id="line-287">import org.apache.hbase.thirdparty.org.eclipse.jetty.server.Server;</span>
<span class="source-line-no">288</span><span id="line-288">import org.apache.hbase.thirdparty.org.eclipse.jetty.server.ServerConnector;</span>
<span class="source-line-no">289</span><span id="line-289">import org.apache.hbase.thirdparty.org.eclipse.jetty.servlet.ServletHolder;</span>
<span class="source-line-no">290</span><span id="line-290">import org.apache.hbase.thirdparty.org.eclipse.jetty.webapp.WebAppContext;</span>
<span class="source-line-no">291</span><span id="line-291">import org.apache.hbase.thirdparty.org.glassfish.jersey.server.ResourceConfig;</span>
<span class="source-line-no">292</span><span id="line-292">import org.apache.hbase.thirdparty.org.glassfish.jersey.servlet.ServletContainer;</span>
<span class="source-line-no">293</span><span id="line-293"></span>
<span class="source-line-no">294</span><span id="line-294">import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;</span>
<span class="source-line-no">295</span><span id="line-295">import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;</span>
<span class="source-line-no">296</span><span id="line-296">import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;</span>
<span class="source-line-no">297</span><span id="line-297"></span>
<span class="source-line-no">298</span><span id="line-298">/**</span>
<span class="source-line-no">299</span><span id="line-299"> * HMaster is the "master server" for HBase. An HBase cluster has one active master. If many masters</span>
<span class="source-line-no">300</span><span id="line-300"> * are started, all compete. Whichever wins goes on to run the cluster. All others park themselves</span>
<span class="source-line-no">301</span><span id="line-301"> * in their constructor until master or cluster shutdown or until the active master loses its lease</span>
<span class="source-line-no">302</span><span id="line-302"> * in zookeeper. Thereafter, all running master jostle to take over master role.</span>
<span class="source-line-no">303</span><span id="line-303"> * &lt;p/&gt;</span>
<span class="source-line-no">304</span><span id="line-304"> * The Master can be asked shutdown the cluster. See {@link #shutdown()}. In this case it will tell</span>
<span class="source-line-no">305</span><span id="line-305"> * all regionservers to go down and then wait on them all reporting in that they are down. This</span>
<span class="source-line-no">306</span><span id="line-306"> * master will then shut itself down.</span>
<span class="source-line-no">307</span><span id="line-307"> * &lt;p/&gt;</span>
<span class="source-line-no">308</span><span id="line-308"> * You can also shutdown just this master. Call {@link #stopMaster()}.</span>
<span class="source-line-no">309</span><span id="line-309"> * @see org.apache.zookeeper.Watcher</span>
<span class="source-line-no">310</span><span id="line-310"> */</span>
<span class="source-line-no">311</span><span id="line-311">@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)</span>
<span class="source-line-no">312</span><span id="line-312">public class HMaster extends HBaseServerBase&lt;MasterRpcServices&gt; implements MasterServices {</span>
<span class="source-line-no">313</span><span id="line-313"></span>
<span class="source-line-no">314</span><span id="line-314"> private static final Logger LOG = LoggerFactory.getLogger(HMaster.class);</span>
<span class="source-line-no">315</span><span id="line-315"></span>
<span class="source-line-no">316</span><span id="line-316"> // MASTER is name of the webapp and the attribute name used stuffing this</span>
<span class="source-line-no">317</span><span id="line-317"> // instance into a web context !! AND OTHER PLACES !!</span>
<span class="source-line-no">318</span><span id="line-318"> public static final String MASTER = "master";</span>
<span class="source-line-no">319</span><span id="line-319"></span>
<span class="source-line-no">320</span><span id="line-320"> // Manager and zk listener for master election</span>
<span class="source-line-no">321</span><span id="line-321"> private final ActiveMasterManager activeMasterManager;</span>
<span class="source-line-no">322</span><span id="line-322"> // Region server tracker</span>
<span class="source-line-no">323</span><span id="line-323"> private final RegionServerTracker regionServerTracker;</span>
<span class="source-line-no">324</span><span id="line-324"> // Draining region server tracker</span>
<span class="source-line-no">325</span><span id="line-325"> private DrainingServerTracker drainingServerTracker;</span>
<span class="source-line-no">326</span><span id="line-326"> // Tracker for load balancer state</span>
<span class="source-line-no">327</span><span id="line-327"> LoadBalancerStateStore loadBalancerStateStore;</span>
<span class="source-line-no">328</span><span id="line-328"> // Tracker for meta location, if any client ZK quorum specified</span>
<span class="source-line-no">329</span><span id="line-329"> private MetaLocationSyncer metaLocationSyncer;</span>
<span class="source-line-no">330</span><span id="line-330"> // Tracker for active master location, if any client ZK quorum specified</span>
<span class="source-line-no">331</span><span id="line-331"> @InterfaceAudience.Private</span>
<span class="source-line-no">332</span><span id="line-332"> MasterAddressSyncer masterAddressSyncer;</span>
<span class="source-line-no">333</span><span id="line-333"> // Tracker for auto snapshot cleanup state</span>
<span class="source-line-no">334</span><span id="line-334"> SnapshotCleanupStateStore snapshotCleanupStateStore;</span>
<span class="source-line-no">335</span><span id="line-335"></span>
<span class="source-line-no">336</span><span id="line-336"> // Tracker for split and merge state</span>
<span class="source-line-no">337</span><span id="line-337"> private SplitOrMergeStateStore splitOrMergeStateStore;</span>
<span class="source-line-no">338</span><span id="line-338"></span>
<span class="source-line-no">339</span><span id="line-339"> private ClusterSchemaService clusterSchemaService;</span>
<span class="source-line-no">340</span><span id="line-340"></span>
<span class="source-line-no">341</span><span id="line-341"> public static final String HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS =</span>
<span class="source-line-no">342</span><span id="line-342"> "hbase.master.wait.on.service.seconds";</span>
<span class="source-line-no">343</span><span id="line-343"> public static final int DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS = 5 * 60;</span>
<span class="source-line-no">344</span><span id="line-344"></span>
<span class="source-line-no">345</span><span id="line-345"> public static final String HBASE_MASTER_CLEANER_INTERVAL = "hbase.master.cleaner.interval";</span>
<span class="source-line-no">346</span><span id="line-346"></span>
<span class="source-line-no">347</span><span id="line-347"> public static final int DEFAULT_HBASE_MASTER_CLEANER_INTERVAL = 600 * 1000;</span>
<span class="source-line-no">348</span><span id="line-348"></span>
<span class="source-line-no">349</span><span id="line-349"> private String clusterId;</span>
<span class="source-line-no">350</span><span id="line-350"></span>
<span class="source-line-no">351</span><span id="line-351"> // Metrics for the HMaster</span>
<span class="source-line-no">352</span><span id="line-352"> final MetricsMaster metricsMaster;</span>
<span class="source-line-no">353</span><span id="line-353"> // file system manager for the master FS operations</span>
<span class="source-line-no">354</span><span id="line-354"> private MasterFileSystem fileSystemManager;</span>
<span class="source-line-no">355</span><span id="line-355"> private MasterWalManager walManager;</span>
<span class="source-line-no">356</span><span id="line-356"></span>
<span class="source-line-no">357</span><span id="line-357"> // manager to manage procedure-based WAL splitting, can be null if current</span>
<span class="source-line-no">358</span><span id="line-358"> // is zk-based WAL splitting. SplitWALManager will replace SplitLogManager</span>
<span class="source-line-no">359</span><span id="line-359"> // and MasterWalManager, which means zk-based WAL splitting code will be</span>
<span class="source-line-no">360</span><span id="line-360"> // useless after we switch to the procedure-based one. our eventual goal</span>
<span class="source-line-no">361</span><span id="line-361"> // is to remove all the zk-based WAL splitting code.</span>
<span class="source-line-no">362</span><span id="line-362"> private SplitWALManager splitWALManager;</span>
<span class="source-line-no">363</span><span id="line-363"></span>
<span class="source-line-no">364</span><span id="line-364"> // server manager to deal with region server info</span>
<span class="source-line-no">365</span><span id="line-365"> private volatile ServerManager serverManager;</span>
<span class="source-line-no">366</span><span id="line-366"></span>
<span class="source-line-no">367</span><span id="line-367"> // manager of assignment nodes in zookeeper</span>
<span class="source-line-no">368</span><span id="line-368"> private AssignmentManager assignmentManager;</span>
<span class="source-line-no">369</span><span id="line-369"></span>
<span class="source-line-no">370</span><span id="line-370"> private RSGroupInfoManager rsGroupInfoManager;</span>
<span class="source-line-no">371</span><span id="line-371"></span>
<span class="source-line-no">372</span><span id="line-372"> private final ReplicationLogCleanerBarrier replicationLogCleanerBarrier =</span>
<span class="source-line-no">373</span><span id="line-373"> new ReplicationLogCleanerBarrier();</span>
<span class="source-line-no">374</span><span id="line-374"></span>
<span class="source-line-no">375</span><span id="line-375"> // Only allow to add one sync replication peer concurrently</span>
<span class="source-line-no">376</span><span id="line-376"> private final Semaphore syncReplicationPeerLock = new Semaphore(1);</span>
<span class="source-line-no">377</span><span id="line-377"></span>
<span class="source-line-no">378</span><span id="line-378"> // manager of replication</span>
<span class="source-line-no">379</span><span id="line-379"> private ReplicationPeerManager replicationPeerManager;</span>
<span class="source-line-no">380</span><span id="line-380"></span>
<span class="source-line-no">381</span><span id="line-381"> private SyncReplicationReplayWALManager syncReplicationReplayWALManager;</span>
<span class="source-line-no">382</span><span id="line-382"></span>
<span class="source-line-no">383</span><span id="line-383"> // buffer for "fatal error" notices from region servers</span>
<span class="source-line-no">384</span><span id="line-384"> // in the cluster. This is only used for assisting</span>
<span class="source-line-no">385</span><span id="line-385"> // operations/debugging.</span>
<span class="source-line-no">386</span><span id="line-386"> MemoryBoundedLogMessageBuffer rsFatals;</span>
<span class="source-line-no">387</span><span id="line-387"></span>
<span class="source-line-no">388</span><span id="line-388"> // flag set after we become the active master (used for testing)</span>
<span class="source-line-no">389</span><span id="line-389"> private volatile boolean activeMaster = false;</span>
<span class="source-line-no">390</span><span id="line-390"></span>
<span class="source-line-no">391</span><span id="line-391"> // flag set after we complete initialization once active</span>
<span class="source-line-no">392</span><span id="line-392"> private final ProcedureEvent&lt;?&gt; initialized = new ProcedureEvent&lt;&gt;("master initialized");</span>
<span class="source-line-no">393</span><span id="line-393"></span>
<span class="source-line-no">394</span><span id="line-394"> // flag set after master services are started,</span>
<span class="source-line-no">395</span><span id="line-395"> // initialization may have not completed yet.</span>
<span class="source-line-no">396</span><span id="line-396"> volatile boolean serviceStarted = false;</span>
<span class="source-line-no">397</span><span id="line-397"></span>
<span class="source-line-no">398</span><span id="line-398"> // Maximum time we should run balancer for</span>
<span class="source-line-no">399</span><span id="line-399"> private final int maxBalancingTime;</span>
<span class="source-line-no">400</span><span id="line-400"> // Maximum percent of regions in transition when balancing</span>
<span class="source-line-no">401</span><span id="line-401"> private final double maxRitPercent;</span>
<span class="source-line-no">402</span><span id="line-402"></span>
<span class="source-line-no">403</span><span id="line-403"> private final LockManager lockManager = new LockManager(this);</span>
<span class="source-line-no">404</span><span id="line-404"></span>
<span class="source-line-no">405</span><span id="line-405"> private RSGroupBasedLoadBalancer balancer;</span>
<span class="source-line-no">406</span><span id="line-406"> private BalancerChore balancerChore;</span>
<span class="source-line-no">407</span><span id="line-407"> private static boolean disableBalancerChoreForTest = false;</span>
<span class="source-line-no">408</span><span id="line-408"> private RegionNormalizerManager regionNormalizerManager;</span>
<span class="source-line-no">409</span><span id="line-409"> private ClusterStatusChore clusterStatusChore;</span>
<span class="source-line-no">410</span><span id="line-410"> private ClusterStatusPublisher clusterStatusPublisherChore = null;</span>
<span class="source-line-no">411</span><span id="line-411"> private SnapshotCleanerChore snapshotCleanerChore = null;</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 HbckChore hbckChore;</span>
<span class="source-line-no">414</span><span id="line-414"> CatalogJanitor catalogJanitorChore;</span>
<span class="source-line-no">415</span><span id="line-415"> // Threadpool for scanning the Old logs directory, used by the LogCleaner</span>
<span class="source-line-no">416</span><span id="line-416"> private DirScanPool logCleanerPool;</span>
<span class="source-line-no">417</span><span id="line-417"> private LogCleaner logCleaner;</span>
<span class="source-line-no">418</span><span id="line-418"> // HFile cleaners for the custom hfile archive paths and the default archive path</span>
<span class="source-line-no">419</span><span id="line-419"> // The archive path cleaner is the first element</span>
<span class="source-line-no">420</span><span id="line-420"> private List&lt;HFileCleaner&gt; hfileCleaners = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">421</span><span id="line-421"> // The hfile cleaner paths, including custom paths and the default archive path</span>
<span class="source-line-no">422</span><span id="line-422"> private List&lt;Path&gt; hfileCleanerPaths = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">423</span><span id="line-423"> // The shared hfile cleaner pool for the custom archive paths</span>
<span class="source-line-no">424</span><span id="line-424"> private DirScanPool sharedHFileCleanerPool;</span>
<span class="source-line-no">425</span><span id="line-425"> // The exclusive hfile cleaner pool for scanning the archive directory</span>
<span class="source-line-no">426</span><span id="line-426"> private DirScanPool exclusiveHFileCleanerPool;</span>
<span class="source-line-no">427</span><span id="line-427"> private ReplicationBarrierCleaner replicationBarrierCleaner;</span>
<span class="source-line-no">428</span><span id="line-428"> private MobFileCleanerChore mobFileCleanerChore;</span>
<span class="source-line-no">429</span><span id="line-429"> private MobFileCompactionChore mobFileCompactionChore;</span>
<span class="source-line-no">430</span><span id="line-430"> private RollingUpgradeChore rollingUpgradeChore;</span>
<span class="source-line-no">431</span><span id="line-431"> // used to synchronize the mobCompactionStates</span>
<span class="source-line-no">432</span><span id="line-432"> private final IdLock mobCompactionLock = new IdLock();</span>
<span class="source-line-no">433</span><span id="line-433"> // save the information of mob compactions in tables.</span>
<span class="source-line-no">434</span><span id="line-434"> // the key is table name, the value is the number of compactions in that table.</span>
<span class="source-line-no">435</span><span id="line-435"> private Map&lt;TableName, AtomicInteger&gt; mobCompactionStates = Maps.newConcurrentMap();</span>
<span class="source-line-no">436</span><span id="line-436"></span>
<span class="source-line-no">437</span><span id="line-437"> volatile MasterCoprocessorHost cpHost;</span>
<span class="source-line-no">438</span><span id="line-438"></span>
<span class="source-line-no">439</span><span id="line-439"> private final boolean preLoadTableDescriptors;</span>
<span class="source-line-no">440</span><span id="line-440"></span>
<span class="source-line-no">441</span><span id="line-441"> // Time stamps for when a hmaster became active</span>
<span class="source-line-no">442</span><span id="line-442"> private long masterActiveTime;</span>
<span class="source-line-no">443</span><span id="line-443"></span>
<span class="source-line-no">444</span><span id="line-444"> // Time stamp for when HMaster finishes becoming Active Master</span>
<span class="source-line-no">445</span><span id="line-445"> private long masterFinishedInitializationTime;</span>
<span class="source-line-no">446</span><span id="line-446"></span>
<span class="source-line-no">447</span><span id="line-447"> Map&lt;String, Service&gt; coprocessorServiceHandlers = Maps.newHashMap();</span>
<span class="source-line-no">448</span><span id="line-448"></span>
<span class="source-line-no">449</span><span id="line-449"> // monitor for snapshot of hbase tables</span>
<span class="source-line-no">450</span><span id="line-450"> SnapshotManager snapshotManager;</span>
<span class="source-line-no">451</span><span id="line-451"> // monitor for distributed procedures</span>
<span class="source-line-no">452</span><span id="line-452"> private MasterProcedureManagerHost mpmHost;</span>
<span class="source-line-no">453</span><span id="line-453"></span>
<span class="source-line-no">454</span><span id="line-454"> private RegionsRecoveryChore regionsRecoveryChore = null;</span>
<span class="source-line-no">455</span><span id="line-455"></span>
<span class="source-line-no">456</span><span id="line-456"> private RegionsRecoveryConfigManager regionsRecoveryConfigManager = null;</span>
<span class="source-line-no">457</span><span id="line-457"> // it is assigned after 'initialized' guard set to true, so should be volatile</span>
<span class="source-line-no">458</span><span id="line-458"> private volatile MasterQuotaManager quotaManager;</span>
<span class="source-line-no">459</span><span id="line-459"> private SpaceQuotaSnapshotNotifier spaceQuotaSnapshotNotifier;</span>
<span class="source-line-no">460</span><span id="line-460"> private QuotaObserverChore quotaObserverChore;</span>
<span class="source-line-no">461</span><span id="line-461"> private SnapshotQuotaObserverChore snapshotQuotaChore;</span>
<span class="source-line-no">462</span><span id="line-462"> private OldWALsDirSizeChore oldWALsDirSizeChore;</span>
<span class="source-line-no">463</span><span id="line-463"></span>
<span class="source-line-no">464</span><span id="line-464"> private ProcedureExecutor&lt;MasterProcedureEnv&gt; procedureExecutor;</span>
<span class="source-line-no">465</span><span id="line-465"> private ProcedureStore procedureStore;</span>
<span class="source-line-no">466</span><span id="line-466"></span>
<span class="source-line-no">467</span><span id="line-467"> // the master local storage to store procedure data, meta region locations, etc.</span>
<span class="source-line-no">468</span><span id="line-468"> private MasterRegion masterRegion;</span>
<span class="source-line-no">469</span><span id="line-469"></span>
<span class="source-line-no">470</span><span id="line-470"> private RegionServerList rsListStorage;</span>
<span class="source-line-no">471</span><span id="line-471"></span>
<span class="source-line-no">472</span><span id="line-472"> // handle table states</span>
<span class="source-line-no">473</span><span id="line-473"> private TableStateManager tableStateManager;</span>
<span class="source-line-no">474</span><span id="line-474"></span>
<span class="source-line-no">475</span><span id="line-475"> /** jetty server for master to redirect requests to regionserver infoServer */</span>
<span class="source-line-no">476</span><span id="line-476"> private Server masterJettyServer;</span>
<span class="source-line-no">477</span><span id="line-477"></span>
<span class="source-line-no">478</span><span id="line-478"> // Determine if we should do normal startup or minimal "single-user" mode with no region</span>
<span class="source-line-no">479</span><span id="line-479"> // servers and no user tables. Useful for repair and recovery of hbase:meta</span>
<span class="source-line-no">480</span><span id="line-480"> private final boolean maintenanceMode;</span>
<span class="source-line-no">481</span><span id="line-481"> static final String MAINTENANCE_MODE = "hbase.master.maintenance_mode";</span>
<span class="source-line-no">482</span><span id="line-482"></span>
<span class="source-line-no">483</span><span id="line-483"> // the in process region server for carry system regions in maintenanceMode</span>
<span class="source-line-no">484</span><span id="line-484"> private JVMClusterUtil.RegionServerThread maintenanceRegionServer;</span>
<span class="source-line-no">485</span><span id="line-485"></span>
<span class="source-line-no">486</span><span id="line-486"> // Cached clusterId on stand by masters to serve clusterID requests from clients.</span>
<span class="source-line-no">487</span><span id="line-487"> private final CachedClusterId cachedClusterId;</span>
<span class="source-line-no">488</span><span id="line-488"></span>
<span class="source-line-no">489</span><span id="line-489"> public static final String WARMUP_BEFORE_MOVE = "hbase.master.warmup.before.move";</span>
<span class="source-line-no">490</span><span id="line-490"> private static final boolean DEFAULT_WARMUP_BEFORE_MOVE = true;</span>
<span class="source-line-no">491</span><span id="line-491"></span>
<span class="source-line-no">492</span><span id="line-492"> private TaskGroup startupTaskGroup;</span>
<span class="source-line-no">493</span><span id="line-493"></span>
<span class="source-line-no">494</span><span id="line-494"> /**</span>
<span class="source-line-no">495</span><span id="line-495"> * Store whether we allow replication peer modification operations.</span>
<span class="source-line-no">496</span><span id="line-496"> */</span>
<span class="source-line-no">497</span><span id="line-497"> private ReplicationPeerModificationStateStore replicationPeerModificationStateStore;</span>
<span class="source-line-no">498</span><span id="line-498"></span>
<span class="source-line-no">499</span><span id="line-499"> /**</span>
<span class="source-line-no">500</span><span id="line-500"> * Initializes the HMaster. The steps are as follows:</span>
<span class="source-line-no">501</span><span id="line-501"> * &lt;p&gt;</span>
<span class="source-line-no">502</span><span id="line-502"> * &lt;ol&gt;</span>
<span class="source-line-no">503</span><span id="line-503"> * &lt;li&gt;Initialize the local HRegionServer</span>
<span class="source-line-no">504</span><span id="line-504"> * &lt;li&gt;Start the ActiveMasterManager.</span>
<span class="source-line-no">505</span><span id="line-505"> * &lt;/ol&gt;</span>
<span class="source-line-no">506</span><span id="line-506"> * &lt;p&gt;</span>
<span class="source-line-no">507</span><span id="line-507"> * Remaining steps of initialization occur in {@link #finishActiveMasterInitialization()} after</span>
<span class="source-line-no">508</span><span id="line-508"> * the master becomes the active one.</span>
<span class="source-line-no">509</span><span id="line-509"> */</span>
<span class="source-line-no">510</span><span id="line-510"> public HMaster(final Configuration conf) throws IOException {</span>
<span class="source-line-no">511</span><span id="line-511"> super(conf, "Master");</span>
<span class="source-line-no">512</span><span id="line-512"> final Span span = TraceUtil.createSpan("HMaster.cxtor");</span>
<span class="source-line-no">513</span><span id="line-513"> try (Scope ignored = span.makeCurrent()) {</span>
<span class="source-line-no">514</span><span id="line-514"> if (conf.getBoolean(MAINTENANCE_MODE, false)) {</span>
<span class="source-line-no">515</span><span id="line-515"> LOG.info("Detected {}=true via configuration.", MAINTENANCE_MODE);</span>
<span class="source-line-no">516</span><span id="line-516"> maintenanceMode = true;</span>
<span class="source-line-no">517</span><span id="line-517"> } else if (Boolean.getBoolean(MAINTENANCE_MODE)) {</span>
<span class="source-line-no">518</span><span id="line-518"> LOG.info("Detected {}=true via environment variables.", MAINTENANCE_MODE);</span>
<span class="source-line-no">519</span><span id="line-519"> maintenanceMode = true;</span>
<span class="source-line-no">520</span><span id="line-520"> } else {</span>
<span class="source-line-no">521</span><span id="line-521"> maintenanceMode = false;</span>
<span class="source-line-no">522</span><span id="line-522"> }</span>
<span class="source-line-no">523</span><span id="line-523"> this.rsFatals = new MemoryBoundedLogMessageBuffer(</span>
<span class="source-line-no">524</span><span id="line-524"> conf.getLong("hbase.master.buffer.for.rs.fatals", 1 * 1024 * 1024));</span>
<span class="source-line-no">525</span><span id="line-525"> LOG.info("hbase.rootdir={}, hbase.cluster.distributed={}",</span>
<span class="source-line-no">526</span><span id="line-526"> CommonFSUtils.getRootDir(this.conf),</span>
<span class="source-line-no">527</span><span id="line-527"> this.conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, false));</span>
<span class="source-line-no">528</span><span id="line-528"></span>
<span class="source-line-no">529</span><span id="line-529"> // Disable usage of meta replicas in the master</span>
<span class="source-line-no">530</span><span id="line-530"> this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);</span>
<span class="source-line-no">531</span><span id="line-531"></span>
<span class="source-line-no">532</span><span id="line-532"> decorateMasterConfiguration(this.conf);</span>
<span class="source-line-no">533</span><span id="line-533"></span>
<span class="source-line-no">534</span><span id="line-534"> // Hack! Maps DFSClient =&gt; Master for logs. HDFS made this</span>
<span class="source-line-no">535</span><span id="line-535"> // config param for task trackers, but we can piggyback off of it.</span>
<span class="source-line-no">536</span><span id="line-536"> if (this.conf.get("mapreduce.task.attempt.id") == null) {</span>
<span class="source-line-no">537</span><span id="line-537"> this.conf.set("mapreduce.task.attempt.id", "hb_m_" + this.serverName.toString());</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"> this.metricsMaster = new MetricsMaster(new MetricsMasterWrapperImpl(this));</span>
<span class="source-line-no">541</span><span id="line-541"></span>
<span class="source-line-no">542</span><span id="line-542"> // preload table descriptor at startup</span>
<span class="source-line-no">543</span><span id="line-543"> this.preLoadTableDescriptors = conf.getBoolean("hbase.master.preload.tabledescriptors", true);</span>
<span class="source-line-no">544</span><span id="line-544"></span>
<span class="source-line-no">545</span><span id="line-545"> this.maxBalancingTime = getMaxBalancingTime();</span>
<span class="source-line-no">546</span><span id="line-546"> this.maxRitPercent = conf.getDouble(HConstants.HBASE_MASTER_BALANCER_MAX_RIT_PERCENT,</span>
<span class="source-line-no">547</span><span id="line-547"> HConstants.DEFAULT_HBASE_MASTER_BALANCER_MAX_RIT_PERCENT);</span>
<span class="source-line-no">548</span><span id="line-548"></span>
<span class="source-line-no">549</span><span id="line-549"> // Do we publish the status?</span>
<span class="source-line-no">550</span><span id="line-550"> boolean shouldPublish =</span>
<span class="source-line-no">551</span><span id="line-551"> conf.getBoolean(HConstants.STATUS_PUBLISHED, HConstants.STATUS_PUBLISHED_DEFAULT);</span>
<span class="source-line-no">552</span><span id="line-552"> Class&lt;? extends ClusterStatusPublisher.Publisher&gt; publisherClass =</span>
<span class="source-line-no">553</span><span id="line-553"> conf.getClass(ClusterStatusPublisher.STATUS_PUBLISHER_CLASS,</span>
<span class="source-line-no">554</span><span id="line-554"> ClusterStatusPublisher.DEFAULT_STATUS_PUBLISHER_CLASS,</span>
<span class="source-line-no">555</span><span id="line-555"> ClusterStatusPublisher.Publisher.class);</span>
<span class="source-line-no">556</span><span id="line-556"></span>
<span class="source-line-no">557</span><span id="line-557"> if (shouldPublish) {</span>
<span class="source-line-no">558</span><span id="line-558"> if (publisherClass == null) {</span>
<span class="source-line-no">559</span><span id="line-559"> LOG.warn(HConstants.STATUS_PUBLISHED + " is true, but "</span>
<span class="source-line-no">560</span><span id="line-560"> + ClusterStatusPublisher.DEFAULT_STATUS_PUBLISHER_CLASS</span>
<span class="source-line-no">561</span><span id="line-561"> + " is not set - not publishing status");</span>
<span class="source-line-no">562</span><span id="line-562"> } else {</span>
<span class="source-line-no">563</span><span id="line-563"> clusterStatusPublisherChore = new ClusterStatusPublisher(this, conf, publisherClass);</span>
<span class="source-line-no">564</span><span id="line-564"> LOG.debug("Created {}", this.clusterStatusPublisherChore);</span>
<span class="source-line-no">565</span><span id="line-565"> getChoreService().scheduleChore(clusterStatusPublisherChore);</span>
<span class="source-line-no">566</span><span id="line-566"> }</span>
<span class="source-line-no">567</span><span id="line-567"> }</span>
<span class="source-line-no">568</span><span id="line-568"> this.activeMasterManager = createActiveMasterManager(zooKeeper, serverName, this);</span>
<span class="source-line-no">569</span><span id="line-569"> cachedClusterId = new CachedClusterId(this, conf);</span>
<span class="source-line-no">570</span><span id="line-570"> this.regionServerTracker = new RegionServerTracker(zooKeeper, this);</span>
<span class="source-line-no">571</span><span id="line-571"> this.rpcServices.start(zooKeeper);</span>
<span class="source-line-no">572</span><span id="line-572"> span.setStatus(StatusCode.OK);</span>
<span class="source-line-no">573</span><span id="line-573"> } catch (Throwable t) {</span>
<span class="source-line-no">574</span><span id="line-574"> // Make sure we log the exception. HMaster is often started via reflection and the</span>
<span class="source-line-no">575</span><span id="line-575"> // cause of failed startup is lost.</span>
<span class="source-line-no">576</span><span id="line-576"> TraceUtil.setError(span, t);</span>
<span class="source-line-no">577</span><span id="line-577"> LOG.error("Failed construction of Master", t);</span>
<span class="source-line-no">578</span><span id="line-578"> throw t;</span>
<span class="source-line-no">579</span><span id="line-579"> } finally {</span>
<span class="source-line-no">580</span><span id="line-580"> span.end();</span>
<span class="source-line-no">581</span><span id="line-581"> }</span>
<span class="source-line-no">582</span><span id="line-582"> }</span>
<span class="source-line-no">583</span><span id="line-583"></span>
<span class="source-line-no">584</span><span id="line-584"> /**</span>
<span class="source-line-no">585</span><span id="line-585"> * Protected to have custom implementations in tests override the default ActiveMaster</span>
<span class="source-line-no">586</span><span id="line-586"> * implementation.</span>
<span class="source-line-no">587</span><span id="line-587"> */</span>
<span class="source-line-no">588</span><span id="line-588"> protected ActiveMasterManager createActiveMasterManager(ZKWatcher zk, ServerName sn,</span>
<span class="source-line-no">589</span><span id="line-589"> org.apache.hadoop.hbase.Server server) throws InterruptedIOException {</span>
<span class="source-line-no">590</span><span id="line-590"> return new ActiveMasterManager(zk, sn, server);</span>
<span class="source-line-no">591</span><span id="line-591"> }</span>
<span class="source-line-no">592</span><span id="line-592"></span>
<span class="source-line-no">593</span><span id="line-593"> @Override</span>
<span class="source-line-no">594</span><span id="line-594"> protected String getUseThisHostnameInstead(Configuration conf) {</span>
<span class="source-line-no">595</span><span id="line-595"> return conf.get(MASTER_HOSTNAME_KEY);</span>
<span class="source-line-no">596</span><span id="line-596"> }</span>
<span class="source-line-no">597</span><span id="line-597"></span>
<span class="source-line-no">598</span><span id="line-598"> private void registerConfigurationObservers() {</span>
<span class="source-line-no">599</span><span id="line-599"> configurationManager.registerObserver(this.rpcServices);</span>
<span class="source-line-no">600</span><span id="line-600"> configurationManager.registerObserver(this);</span>
<span class="source-line-no">601</span><span id="line-601"> }</span>
<span class="source-line-no">602</span><span id="line-602"></span>
<span class="source-line-no">603</span><span id="line-603"> // Main run loop. Calls through to the regionserver run loop AFTER becoming active Master; will</span>
<span class="source-line-no">604</span><span id="line-604"> // block in here until then.</span>
<span class="source-line-no">605</span><span id="line-605"> @Override</span>
<span class="source-line-no">606</span><span id="line-606"> public void run() {</span>
<span class="source-line-no">607</span><span id="line-607"> try {</span>
<span class="source-line-no">608</span><span id="line-608"> installShutdownHook();</span>
<span class="source-line-no">609</span><span id="line-609"> registerConfigurationObservers();</span>
<span class="source-line-no">610</span><span id="line-610"> Threads.setDaemonThreadRunning(new Thread(TraceUtil.tracedRunnable(() -&gt; {</span>
<span class="source-line-no">611</span><span id="line-611"> try {</span>
<span class="source-line-no">612</span><span id="line-612"> int infoPort = putUpJettyServer();</span>
<span class="source-line-no">613</span><span id="line-613"> startActiveMasterManager(infoPort);</span>
<span class="source-line-no">614</span><span id="line-614"> } catch (Throwable t) {</span>
<span class="source-line-no">615</span><span id="line-615"> // Make sure we log the exception.</span>
<span class="source-line-no">616</span><span id="line-616"> String error = "Failed to become Active Master";</span>
<span class="source-line-no">617</span><span id="line-617"> LOG.error(error, t);</span>
<span class="source-line-no">618</span><span id="line-618"> // Abort should have been called already.</span>
<span class="source-line-no">619</span><span id="line-619"> if (!isAborted()) {</span>
<span class="source-line-no">620</span><span id="line-620"> abort(error, t);</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"> }, "HMaster.becomeActiveMaster")), getName() + ":becomeActiveMaster");</span>
<span class="source-line-no">624</span><span id="line-624"> while (!isStopped() &amp;&amp; !isAborted()) {</span>
<span class="source-line-no">625</span><span id="line-625"> sleeper.sleep();</span>
<span class="source-line-no">626</span><span id="line-626"> }</span>
<span class="source-line-no">627</span><span id="line-627"> final Span span = TraceUtil.createSpan("HMaster exiting main loop");</span>
<span class="source-line-no">628</span><span id="line-628"> try (Scope ignored = span.makeCurrent()) {</span>
<span class="source-line-no">629</span><span id="line-629"> stopInfoServer();</span>
<span class="source-line-no">630</span><span id="line-630"> closeClusterConnection();</span>
<span class="source-line-no">631</span><span id="line-631"> stopServiceThreads();</span>
<span class="source-line-no">632</span><span id="line-632"> if (this.rpcServices != null) {</span>
<span class="source-line-no">633</span><span id="line-633"> this.rpcServices.stop();</span>
<span class="source-line-no">634</span><span id="line-634"> }</span>
<span class="source-line-no">635</span><span id="line-635"> closeZooKeeper();</span>
<span class="source-line-no">636</span><span id="line-636"> closeTableDescriptors();</span>
<span class="source-line-no">637</span><span id="line-637"> span.setStatus(StatusCode.OK);</span>
<span class="source-line-no">638</span><span id="line-638"> } finally {</span>
<span class="source-line-no">639</span><span id="line-639"> span.end();</span>
<span class="source-line-no">640</span><span id="line-640"> }</span>
<span class="source-line-no">641</span><span id="line-641"> } finally {</span>
<span class="source-line-no">642</span><span id="line-642"> if (this.clusterSchemaService != null) {</span>
<span class="source-line-no">643</span><span id="line-643"> // If on way out, then we are no longer active master.</span>
<span class="source-line-no">644</span><span id="line-644"> this.clusterSchemaService.stopAsync();</span>
<span class="source-line-no">645</span><span id="line-645"> try {</span>
<span class="source-line-no">646</span><span id="line-646"> this.clusterSchemaService</span>
<span class="source-line-no">647</span><span id="line-647"> .awaitTerminated(getConfiguration().getInt(HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS,</span>
<span class="source-line-no">648</span><span id="line-648"> DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS), TimeUnit.SECONDS);</span>
<span class="source-line-no">649</span><span id="line-649"> } catch (TimeoutException te) {</span>
<span class="source-line-no">650</span><span id="line-650"> LOG.warn("Failed shutdown of clusterSchemaService", te);</span>
<span class="source-line-no">651</span><span id="line-651"> }</span>
<span class="source-line-no">652</span><span id="line-652"> }</span>
<span class="source-line-no">653</span><span id="line-653"> this.activeMaster = false;</span>
<span class="source-line-no">654</span><span id="line-654"> }</span>
<span class="source-line-no">655</span><span id="line-655"> }</span>
<span class="source-line-no">656</span><span id="line-656"></span>
<span class="source-line-no">657</span><span id="line-657"> // return the actual infoPort, -1 means disable info server.</span>
<span class="source-line-no">658</span><span id="line-658"> private int putUpJettyServer() throws IOException {</span>
<span class="source-line-no">659</span><span id="line-659"> if (!conf.getBoolean("hbase.master.infoserver.redirect", true)) {</span>
<span class="source-line-no">660</span><span id="line-660"> return -1;</span>
<span class="source-line-no">661</span><span id="line-661"> }</span>
<span class="source-line-no">662</span><span id="line-662"> final int infoPort =</span>
<span class="source-line-no">663</span><span id="line-663"> conf.getInt("hbase.master.info.port.orig", HConstants.DEFAULT_MASTER_INFOPORT);</span>
<span class="source-line-no">664</span><span id="line-664"> // -1 is for disabling info server, so no redirecting</span>
<span class="source-line-no">665</span><span id="line-665"> if (infoPort &lt; 0 || infoServer == null) {</span>
<span class="source-line-no">666</span><span id="line-666"> return -1;</span>
<span class="source-line-no">667</span><span id="line-667"> }</span>
<span class="source-line-no">668</span><span id="line-668"> if (infoPort == infoServer.getPort()) {</span>
<span class="source-line-no">669</span><span id="line-669"> // server is already running</span>
<span class="source-line-no">670</span><span id="line-670"> return infoPort;</span>
<span class="source-line-no">671</span><span id="line-671"> }</span>
<span class="source-line-no">672</span><span id="line-672"> final String addr = conf.get("hbase.master.info.bindAddress", "0.0.0.0");</span>
<span class="source-line-no">673</span><span id="line-673"> if (!Addressing.isLocalAddress(InetAddress.getByName(addr))) {</span>
<span class="source-line-no">674</span><span id="line-674"> String msg = "Failed to start redirecting jetty server. Address " + addr</span>
<span class="source-line-no">675</span><span id="line-675"> + " does not belong to this host. Correct configuration parameter: "</span>
<span class="source-line-no">676</span><span id="line-676"> + "hbase.master.info.bindAddress";</span>
<span class="source-line-no">677</span><span id="line-677"> LOG.error(msg);</span>
<span class="source-line-no">678</span><span id="line-678"> throw new IOException(msg);</span>
<span class="source-line-no">679</span><span id="line-679"> }</span>
<span class="source-line-no">680</span><span id="line-680"></span>
<span class="source-line-no">681</span><span id="line-681"> // TODO I'm pretty sure we could just add another binding to the InfoServer run by</span>
<span class="source-line-no">682</span><span id="line-682"> // the RegionServer and have it run the RedirectServlet instead of standing up</span>
<span class="source-line-no">683</span><span id="line-683"> // a second entire stack here.</span>
<span class="source-line-no">684</span><span id="line-684"> masterJettyServer = new Server();</span>
<span class="source-line-no">685</span><span id="line-685"> final ServerConnector connector = new ServerConnector(masterJettyServer);</span>
<span class="source-line-no">686</span><span id="line-686"> connector.setHost(addr);</span>
<span class="source-line-no">687</span><span id="line-687"> connector.setPort(infoPort);</span>
<span class="source-line-no">688</span><span id="line-688"> masterJettyServer.addConnector(connector);</span>
<span class="source-line-no">689</span><span id="line-689"> masterJettyServer.setStopAtShutdown(true);</span>
<span class="source-line-no">690</span><span id="line-690"> masterJettyServer.setHandler(HttpServer.buildGzipHandler(masterJettyServer.getHandler()));</span>
<span class="source-line-no">691</span><span id="line-691"></span>
<span class="source-line-no">692</span><span id="line-692"> final String redirectHostname =</span>
<span class="source-line-no">693</span><span id="line-693"> StringUtils.isBlank(useThisHostnameInstead) ? null : useThisHostnameInstead;</span>
<span class="source-line-no">694</span><span id="line-694"></span>
<span class="source-line-no">695</span><span id="line-695"> final MasterRedirectServlet redirect = new MasterRedirectServlet(infoServer, redirectHostname);</span>
<span class="source-line-no">696</span><span id="line-696"> final WebAppContext context =</span>
<span class="source-line-no">697</span><span id="line-697"> new WebAppContext(null, "/", null, null, null, null, WebAppContext.NO_SESSIONS);</span>
<span class="source-line-no">698</span><span id="line-698"> context.addServlet(new ServletHolder(redirect), "/*");</span>
<span class="source-line-no">699</span><span id="line-699"> context.setServer(masterJettyServer);</span>
<span class="source-line-no">700</span><span id="line-700"></span>
<span class="source-line-no">701</span><span id="line-701"> try {</span>
<span class="source-line-no">702</span><span id="line-702"> masterJettyServer.start();</span>
<span class="source-line-no">703</span><span id="line-703"> } catch (Exception e) {</span>
<span class="source-line-no">704</span><span id="line-704"> throw new IOException("Failed to start redirecting jetty server", e);</span>
<span class="source-line-no">705</span><span id="line-705"> }</span>
<span class="source-line-no">706</span><span id="line-706"> return connector.getLocalPort();</span>
<span class="source-line-no">707</span><span id="line-707"> }</span>
<span class="source-line-no">708</span><span id="line-708"></span>
<span class="source-line-no">709</span><span id="line-709"> /**</span>
<span class="source-line-no">710</span><span id="line-710"> * For compatibility, if failed with regionserver credentials, try the master one</span>
<span class="source-line-no">711</span><span id="line-711"> */</span>
<span class="source-line-no">712</span><span id="line-712"> @Override</span>
<span class="source-line-no">713</span><span id="line-713"> protected void login(UserProvider user, String host) throws IOException {</span>
<span class="source-line-no">714</span><span id="line-714"> try {</span>
<span class="source-line-no">715</span><span id="line-715"> user.login(SecurityConstants.REGIONSERVER_KRB_KEYTAB_FILE,</span>
<span class="source-line-no">716</span><span id="line-716"> SecurityConstants.REGIONSERVER_KRB_PRINCIPAL, host);</span>
<span class="source-line-no">717</span><span id="line-717"> } catch (IOException ie) {</span>
<span class="source-line-no">718</span><span id="line-718"> user.login(SecurityConstants.MASTER_KRB_KEYTAB_FILE, SecurityConstants.MASTER_KRB_PRINCIPAL,</span>
<span class="source-line-no">719</span><span id="line-719"> host);</span>
<span class="source-line-no">720</span><span id="line-720"> }</span>
<span class="source-line-no">721</span><span id="line-721"> }</span>
<span class="source-line-no">722</span><span id="line-722"></span>
<span class="source-line-no">723</span><span id="line-723"> public MasterRpcServices getMasterRpcServices() {</span>
<span class="source-line-no">724</span><span id="line-724"> return rpcServices;</span>
<span class="source-line-no">725</span><span id="line-725"> }</span>
<span class="source-line-no">726</span><span id="line-726"></span>
<span class="source-line-no">727</span><span id="line-727"> @Override</span>
<span class="source-line-no">728</span><span id="line-728"> protected MasterCoprocessorHost getCoprocessorHost() {</span>
<span class="source-line-no">729</span><span id="line-729"> return getMasterCoprocessorHost();</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"> public boolean balanceSwitch(final boolean b) throws IOException {</span>
<span class="source-line-no">733</span><span id="line-733"> return getMasterRpcServices().switchBalancer(b, BalanceSwitchMode.ASYNC);</span>
<span class="source-line-no">734</span><span id="line-734"> }</span>
<span class="source-line-no">735</span><span id="line-735"></span>
<span class="source-line-no">736</span><span id="line-736"> @Override</span>
<span class="source-line-no">737</span><span id="line-737"> protected String getProcessName() {</span>
<span class="source-line-no">738</span><span id="line-738"> return MASTER;</span>
<span class="source-line-no">739</span><span id="line-739"> }</span>
<span class="source-line-no">740</span><span id="line-740"></span>
<span class="source-line-no">741</span><span id="line-741"> @Override</span>
<span class="source-line-no">742</span><span id="line-742"> protected boolean canCreateBaseZNode() {</span>
<span class="source-line-no">743</span><span id="line-743"> return true;</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"> @Override</span>
<span class="source-line-no">747</span><span id="line-747"> protected boolean canUpdateTableDescriptor() {</span>
<span class="source-line-no">748</span><span id="line-748"> return true;</span>
<span class="source-line-no">749</span><span id="line-749"> }</span>
<span class="source-line-no">750</span><span id="line-750"></span>
<span class="source-line-no">751</span><span id="line-751"> @Override</span>
<span class="source-line-no">752</span><span id="line-752"> protected boolean cacheTableDescriptor() {</span>
<span class="source-line-no">753</span><span id="line-753"> return true;</span>
<span class="source-line-no">754</span><span id="line-754"> }</span>
<span class="source-line-no">755</span><span id="line-755"></span>
<span class="source-line-no">756</span><span id="line-756"> protected MasterRpcServices createRpcServices() throws IOException {</span>
<span class="source-line-no">757</span><span id="line-757"> return new MasterRpcServices(this);</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"> @Override</span>
<span class="source-line-no">761</span><span id="line-761"> protected void configureInfoServer(InfoServer infoServer) {</span>
<span class="source-line-no">762</span><span id="line-762"> infoServer.addUnprivilegedServlet("master-status", "/master-status", MasterStatusServlet.class);</span>
<span class="source-line-no">763</span><span id="line-763"> infoServer.addUnprivilegedServlet("api_v1", "/api/v1/*", buildApiV1Servlet());</span>
<span class="source-line-no">764</span><span id="line-764"> infoServer.addUnprivilegedServlet("hbck", "/hbck/*", buildHbckServlet());</span>
<span class="source-line-no">765</span><span id="line-765"></span>
<span class="source-line-no">766</span><span id="line-766"> infoServer.setAttribute(MASTER, this);</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"> private ServletHolder buildApiV1Servlet() {</span>
<span class="source-line-no">770</span><span id="line-770"> final ResourceConfig config = ResourceConfigFactory.createResourceConfig(conf, this);</span>
<span class="source-line-no">771</span><span id="line-771"> return new ServletHolder(new ServletContainer(config));</span>
<span class="source-line-no">772</span><span id="line-772"> }</span>
<span class="source-line-no">773</span><span id="line-773"></span>
<span class="source-line-no">774</span><span id="line-774"> private ServletHolder buildHbckServlet() {</span>
<span class="source-line-no">775</span><span id="line-775"> final ResourceConfig config = HbckConfigFactory.createResourceConfig(conf, this);</span>
<span class="source-line-no">776</span><span id="line-776"> return new ServletHolder(new ServletContainer(config));</span>
<span class="source-line-no">777</span><span id="line-777"> }</span>
<span class="source-line-no">778</span><span id="line-778"></span>
<span class="source-line-no">779</span><span id="line-779"> @Override</span>
<span class="source-line-no">780</span><span id="line-780"> protected Class&lt;? extends HttpServlet&gt; getDumpServlet() {</span>
<span class="source-line-no">781</span><span id="line-781"> return MasterDumpServlet.class;</span>
<span class="source-line-no">782</span><span id="line-782"> }</span>
<span class="source-line-no">783</span><span id="line-783"></span>
<span class="source-line-no">784</span><span id="line-784"> @Override</span>
<span class="source-line-no">785</span><span id="line-785"> public MetricsMaster getMasterMetrics() {</span>
<span class="source-line-no">786</span><span id="line-786"> return metricsMaster;</span>
<span class="source-line-no">787</span><span id="line-787"> }</span>
<span class="source-line-no">788</span><span id="line-788"></span>
<span class="source-line-no">789</span><span id="line-789"> /**</span>
<span class="source-line-no">790</span><span id="line-790"> * Initialize all ZK based system trackers. But do not include {@link RegionServerTracker}, it</span>
<span class="source-line-no">791</span><span id="line-791"> * should have already been initialized along with {@link ServerManager}.</span>
<span class="source-line-no">792</span><span id="line-792"> */</span>
<span class="source-line-no">793</span><span id="line-793"> private void initializeZKBasedSystemTrackers()</span>
<span class="source-line-no">794</span><span id="line-794"> throws IOException, KeeperException, ReplicationException, DeserializationException {</span>
<span class="source-line-no">795</span><span id="line-795"> if (maintenanceMode) {</span>
<span class="source-line-no">796</span><span id="line-796"> // in maintenance mode, always use MaintenanceLoadBalancer.</span>
<span class="source-line-no">797</span><span id="line-797"> conf.unset(LoadBalancer.HBASE_RSGROUP_LOADBALANCER_CLASS);</span>
<span class="source-line-no">798</span><span id="line-798"> conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, MaintenanceLoadBalancer.class,</span>
<span class="source-line-no">799</span><span id="line-799"> LoadBalancer.class);</span>
<span class="source-line-no">800</span><span id="line-800"> }</span>
<span class="source-line-no">801</span><span id="line-801"> this.balancer = new RSGroupBasedLoadBalancer();</span>
<span class="source-line-no">802</span><span id="line-802"> this.loadBalancerStateStore = new LoadBalancerStateStore(masterRegion, zooKeeper);</span>
<span class="source-line-no">803</span><span id="line-803"></span>
<span class="source-line-no">804</span><span id="line-804"> this.regionNormalizerManager =</span>
<span class="source-line-no">805</span><span id="line-805"> RegionNormalizerFactory.createNormalizerManager(conf, masterRegion, zooKeeper, this);</span>
<span class="source-line-no">806</span><span id="line-806"> this.configurationManager.registerObserver(regionNormalizerManager);</span>
<span class="source-line-no">807</span><span id="line-807"> this.regionNormalizerManager.start();</span>
<span class="source-line-no">808</span><span id="line-808"></span>
<span class="source-line-no">809</span><span id="line-809"> this.splitOrMergeStateStore = new SplitOrMergeStateStore(masterRegion, zooKeeper, conf);</span>
<span class="source-line-no">810</span><span id="line-810"></span>
<span class="source-line-no">811</span><span id="line-811"> // This is for backwards compatible. We do not need the CP for rs group now but if user want to</span>
<span class="source-line-no">812</span><span id="line-812"> // load it, we need to enable rs group.</span>
<span class="source-line-no">813</span><span id="line-813"> String[] cpClasses = conf.getStrings(MasterCoprocessorHost.MASTER_COPROCESSOR_CONF_KEY);</span>
<span class="source-line-no">814</span><span id="line-814"> if (cpClasses != null) {</span>
<span class="source-line-no">815</span><span id="line-815"> for (String cpClass : cpClasses) {</span>
<span class="source-line-no">816</span><span id="line-816"> if (RSGroupAdminEndpoint.class.getName().equals(cpClass)) {</span>
<span class="source-line-no">817</span><span id="line-817"> RSGroupUtil.enableRSGroup(conf);</span>
<span class="source-line-no">818</span><span id="line-818"> break;</span>
<span class="source-line-no">819</span><span id="line-819"> }</span>
<span class="source-line-no">820</span><span id="line-820"> }</span>
<span class="source-line-no">821</span><span id="line-821"> }</span>
<span class="source-line-no">822</span><span id="line-822"> this.rsGroupInfoManager = RSGroupInfoManager.create(this);</span>
<span class="source-line-no">823</span><span id="line-823"></span>
<span class="source-line-no">824</span><span id="line-824"> this.replicationPeerManager = ReplicationPeerManager.create(this, clusterId);</span>
<span class="source-line-no">825</span><span id="line-825"> this.configurationManager.registerObserver(replicationPeerManager);</span>
<span class="source-line-no">826</span><span id="line-826"> this.replicationPeerModificationStateStore =</span>
<span class="source-line-no">827</span><span id="line-827"> new ReplicationPeerModificationStateStore(masterRegion);</span>
<span class="source-line-no">828</span><span id="line-828"></span>
<span class="source-line-no">829</span><span id="line-829"> this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this, this.serverManager);</span>
<span class="source-line-no">830</span><span id="line-830"> this.drainingServerTracker.start();</span>
<span class="source-line-no">831</span><span id="line-831"></span>
<span class="source-line-no">832</span><span id="line-832"> this.snapshotCleanupStateStore = new SnapshotCleanupStateStore(masterRegion, zooKeeper);</span>
<span class="source-line-no">833</span><span id="line-833"></span>
<span class="source-line-no">834</span><span id="line-834"> String clientQuorumServers = conf.get(HConstants.CLIENT_ZOOKEEPER_QUORUM);</span>
<span class="source-line-no">835</span><span id="line-835"> boolean clientZkObserverMode = conf.getBoolean(HConstants.CLIENT_ZOOKEEPER_OBSERVER_MODE,</span>
<span class="source-line-no">836</span><span id="line-836"> HConstants.DEFAULT_CLIENT_ZOOKEEPER_OBSERVER_MODE);</span>
<span class="source-line-no">837</span><span id="line-837"> if (clientQuorumServers != null &amp;&amp; !clientZkObserverMode) {</span>
<span class="source-line-no">838</span><span id="line-838"> // we need to take care of the ZK information synchronization</span>
<span class="source-line-no">839</span><span id="line-839"> // if given client ZK are not observer nodes</span>
<span class="source-line-no">840</span><span id="line-840"> ZKWatcher clientZkWatcher = new ZKWatcher(conf,</span>
<span class="source-line-no">841</span><span id="line-841"> getProcessName() + ":" + rpcServices.getSocketAddress().getPort() + "-clientZK", this,</span>
<span class="source-line-no">842</span><span id="line-842"> false, true);</span>
<span class="source-line-no">843</span><span id="line-843"> this.metaLocationSyncer = new MetaLocationSyncer(zooKeeper, clientZkWatcher, this);</span>
<span class="source-line-no">844</span><span id="line-844"> this.metaLocationSyncer.start();</span>
<span class="source-line-no">845</span><span id="line-845"> this.masterAddressSyncer = new MasterAddressSyncer(zooKeeper, clientZkWatcher, this);</span>
<span class="source-line-no">846</span><span id="line-846"> this.masterAddressSyncer.start();</span>
<span class="source-line-no">847</span><span id="line-847"> // set cluster id is a one-go effort</span>
<span class="source-line-no">848</span><span id="line-848"> ZKClusterId.setClusterId(clientZkWatcher, fileSystemManager.getClusterId());</span>
<span class="source-line-no">849</span><span id="line-849"> }</span>
<span class="source-line-no">850</span><span id="line-850"></span>
<span class="source-line-no">851</span><span id="line-851"> // Set the cluster as up. If new RSs, they'll be waiting on this before</span>
<span class="source-line-no">852</span><span id="line-852"> // going ahead with their startup.</span>
<span class="source-line-no">853</span><span id="line-853"> boolean wasUp = this.clusterStatusTracker.isClusterUp();</span>
<span class="source-line-no">854</span><span id="line-854"> if (!wasUp) this.clusterStatusTracker.setClusterUp();</span>
<span class="source-line-no">855</span><span id="line-855"></span>
<span class="source-line-no">856</span><span id="line-856"> LOG.info("Active/primary master=" + this.serverName + ", sessionid=0x"</span>
<span class="source-line-no">857</span><span id="line-857"> + Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId())</span>
<span class="source-line-no">858</span><span id="line-858"> + ", setting cluster-up flag (Was=" + wasUp + ")");</span>
<span class="source-line-no">859</span><span id="line-859"></span>
<span class="source-line-no">860</span><span id="line-860"> // create/initialize the snapshot manager and other procedure managers</span>
<span class="source-line-no">861</span><span id="line-861"> this.snapshotManager = new SnapshotManager();</span>
<span class="source-line-no">862</span><span id="line-862"> this.mpmHost = new MasterProcedureManagerHost();</span>
<span class="source-line-no">863</span><span id="line-863"> this.mpmHost.register(this.snapshotManager);</span>
<span class="source-line-no">864</span><span id="line-864"> this.mpmHost.register(new MasterFlushTableProcedureManager());</span>
<span class="source-line-no">865</span><span id="line-865"> this.mpmHost.loadProcedures(conf);</span>
<span class="source-line-no">866</span><span id="line-866"> this.mpmHost.initialize(this, this.metricsMaster);</span>
<span class="source-line-no">867</span><span id="line-867"> }</span>
<span class="source-line-no">868</span><span id="line-868"></span>
<span class="source-line-no">869</span><span id="line-869"> // Will be overriden in test to inject customized AssignmentManager</span>
<span class="source-line-no">870</span><span id="line-870"> @InterfaceAudience.Private</span>
<span class="source-line-no">871</span><span id="line-871"> protected AssignmentManager createAssignmentManager(MasterServices master,</span>
<span class="source-line-no">872</span><span id="line-872"> MasterRegion masterRegion) {</span>
<span class="source-line-no">873</span><span id="line-873"> return new AssignmentManager(master, masterRegion);</span>
<span class="source-line-no">874</span><span id="line-874"> }</span>
<span class="source-line-no">875</span><span id="line-875"></span>
<span class="source-line-no">876</span><span id="line-876"> private void tryMigrateMetaLocationsFromZooKeeper() throws IOException, KeeperException {</span>
<span class="source-line-no">877</span><span id="line-877"> // try migrate data from zookeeper</span>
<span class="source-line-no">878</span><span id="line-878"> try (ResultScanner scanner =</span>
<span class="source-line-no">879</span><span id="line-879"> masterRegion.getScanner(new Scan().addFamily(HConstants.CATALOG_FAMILY))) {</span>
<span class="source-line-no">880</span><span id="line-880"> if (scanner.next() != null) {</span>
<span class="source-line-no">881</span><span id="line-881"> // notice that all replicas for a region are in the same row, so the migration can be</span>
<span class="source-line-no">882</span><span id="line-882"> // done with in a one row put, which means if we have data in catalog family then we can</span>
<span class="source-line-no">883</span><span id="line-883"> // make sure that the migration is done.</span>
<span class="source-line-no">884</span><span id="line-884"> LOG.info("The {} family in master local region already has data in it, skip migrating...",</span>
<span class="source-line-no">885</span><span id="line-885"> HConstants.CATALOG_FAMILY_STR);</span>
<span class="source-line-no">886</span><span id="line-886"> return;</span>
<span class="source-line-no">887</span><span id="line-887"> }</span>
<span class="source-line-no">888</span><span id="line-888"> }</span>
<span class="source-line-no">889</span><span id="line-889"> // start migrating</span>
<span class="source-line-no">890</span><span id="line-890"> byte[] row = CatalogFamilyFormat.getMetaKeyForRegion(RegionInfoBuilder.FIRST_META_REGIONINFO);</span>
<span class="source-line-no">891</span><span id="line-891"> Put put = new Put(row);</span>
<span class="source-line-no">892</span><span id="line-892"> List&lt;String&gt; metaReplicaNodes = zooKeeper.getMetaReplicaNodes();</span>
<span class="source-line-no">893</span><span id="line-893"> StringBuilder info = new StringBuilder("Migrating meta locations:");</span>
<span class="source-line-no">894</span><span id="line-894"> for (String metaReplicaNode : metaReplicaNodes) {</span>
<span class="source-line-no">895</span><span id="line-895"> int replicaId = zooKeeper.getZNodePaths().getMetaReplicaIdFromZNode(metaReplicaNode);</span>
<span class="source-line-no">896</span><span id="line-896"> RegionState state = MetaTableLocator.getMetaRegionState(zooKeeper, replicaId);</span>
<span class="source-line-no">897</span><span id="line-897"> info.append(" ").append(state);</span>
<span class="source-line-no">898</span><span id="line-898"> put.setTimestamp(state.getStamp());</span>
<span class="source-line-no">899</span><span id="line-899"> MetaTableAccessor.addRegionInfo(put, state.getRegion());</span>
<span class="source-line-no">900</span><span id="line-900"> if (state.getServerName() != null) {</span>
<span class="source-line-no">901</span><span id="line-901"> MetaTableAccessor.addLocation(put, state.getServerName(), HConstants.NO_SEQNUM, replicaId);</span>
<span class="source-line-no">902</span><span id="line-902"> }</span>
<span class="source-line-no">903</span><span id="line-903"> put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())</span>
<span class="source-line-no">904</span><span id="line-904"> .setFamily(HConstants.CATALOG_FAMILY)</span>
<span class="source-line-no">905</span><span id="line-905"> .setQualifier(RegionStateStore.getStateColumn(replicaId)).setTimestamp(put.getTimestamp())</span>
<span class="source-line-no">906</span><span id="line-906"> .setType(Cell.Type.Put).setValue(Bytes.toBytes(state.getState().name())).build());</span>
<span class="source-line-no">907</span><span id="line-907"> }</span>
<span class="source-line-no">908</span><span id="line-908"> if (!put.isEmpty()) {</span>
<span class="source-line-no">909</span><span id="line-909"> LOG.info(info.toString());</span>
<span class="source-line-no">910</span><span id="line-910"> masterRegion.update(r -&gt; r.put(put));</span>
<span class="source-line-no">911</span><span id="line-911"> } else {</span>
<span class="source-line-no">912</span><span id="line-912"> LOG.info("No meta location available on zookeeper, skip migrating...");</span>
<span class="source-line-no">913</span><span id="line-913"> }</span>
<span class="source-line-no">914</span><span id="line-914"> }</span>
<span class="source-line-no">915</span><span id="line-915"></span>
<span class="source-line-no">916</span><span id="line-916"> /**</span>
<span class="source-line-no">917</span><span id="line-917"> * Finish initialization of HMaster after becoming the primary master.</span>
<span class="source-line-no">918</span><span id="line-918"> * &lt;p/&gt;</span>
<span class="source-line-no">919</span><span id="line-919"> * The startup order is a bit complicated but very important, do not change it unless you know</span>
<span class="source-line-no">920</span><span id="line-920"> * what you are doing.</span>
<span class="source-line-no">921</span><span id="line-921"> * &lt;ol&gt;</span>
<span class="source-line-no">922</span><span id="line-922"> * &lt;li&gt;Initialize file system based components - file system manager, wal manager, table</span>
<span class="source-line-no">923</span><span id="line-923"> * descriptors, etc&lt;/li&gt;</span>
<span class="source-line-no">924</span><span id="line-924"> * &lt;li&gt;Publish cluster id&lt;/li&gt;</span>
<span class="source-line-no">925</span><span id="line-925"> * &lt;li&gt;Here comes the most complicated part - initialize server manager, assignment manager and</span>
<span class="source-line-no">926</span><span id="line-926"> * region server tracker</span>
<span class="source-line-no">927</span><span id="line-927"> * &lt;ol type='i'&gt;</span>
<span class="source-line-no">928</span><span id="line-928"> * &lt;li&gt;Create server manager&lt;/li&gt;</span>
<span class="source-line-no">929</span><span id="line-929"> * &lt;li&gt;Create master local region&lt;/li&gt;</span>
<span class="source-line-no">930</span><span id="line-930"> * &lt;li&gt;Create procedure executor, load the procedures, but do not start workers. We will start it</span>
<span class="source-line-no">931</span><span id="line-931"> * later after we finish scheduling SCPs to avoid scheduling duplicated SCPs for the same</span>
<span class="source-line-no">932</span><span id="line-932"> * server&lt;/li&gt;</span>
<span class="source-line-no">933</span><span id="line-933"> * &lt;li&gt;Create assignment manager and start it, load the meta region state, but do not load data</span>
<span class="source-line-no">934</span><span id="line-934"> * from meta region&lt;/li&gt;</span>
<span class="source-line-no">935</span><span id="line-935"> * &lt;li&gt;Start region server tracker, construct the online servers set and find out dead servers and</span>
<span class="source-line-no">936</span><span id="line-936"> * schedule SCP for them. The online servers will be constructed by scanning zk, and we will also</span>
<span class="source-line-no">937</span><span id="line-937"> * scan the wal directory and load from master local region to find out possible live region</span>
<span class="source-line-no">938</span><span id="line-938"> * servers, and the differences between these two sets are the dead servers&lt;/li&gt;</span>
<span class="source-line-no">939</span><span id="line-939"> * &lt;/ol&gt;</span>
<span class="source-line-no">940</span><span id="line-940"> * &lt;/li&gt;</span>
<span class="source-line-no">941</span><span id="line-941"> * &lt;li&gt;If this is a new deploy, schedule a InitMetaProcedure to initialize meta&lt;/li&gt;</span>
<span class="source-line-no">942</span><span id="line-942"> * &lt;li&gt;Start necessary service threads - balancer, catalog janitor, executor services, and also</span>
<span class="source-line-no">943</span><span id="line-943"> * the procedure executor, etc. Notice that the balancer must be created first as assignment</span>
<span class="source-line-no">944</span><span id="line-944"> * manager may use it when assigning regions.&lt;/li&gt;</span>
<span class="source-line-no">945</span><span id="line-945"> * &lt;li&gt;Wait for meta to be initialized if necessary, start table state manager.&lt;/li&gt;</span>
<span class="source-line-no">946</span><span id="line-946"> * &lt;li&gt;Wait for enough region servers to check-in&lt;/li&gt;</span>
<span class="source-line-no">947</span><span id="line-947"> * &lt;li&gt;Let assignment manager load data from meta and construct region states&lt;/li&gt;</span>
<span class="source-line-no">948</span><span id="line-948"> * &lt;li&gt;Start all other things such as chore services, etc&lt;/li&gt;</span>
<span class="source-line-no">949</span><span id="line-949"> * &lt;/ol&gt;</span>
<span class="source-line-no">950</span><span id="line-950"> * &lt;p/&gt;</span>
<span class="source-line-no">951</span><span id="line-951"> * Notice that now we will not schedule a special procedure to make meta online(unless the first</span>
<span class="source-line-no">952</span><span id="line-952"> * time where meta has not been created yet), we will rely on SCP to bring meta online.</span>
<span class="source-line-no">953</span><span id="line-953"> */</span>
<span class="source-line-no">954</span><span id="line-954"> private void finishActiveMasterInitialization() throws IOException, InterruptedException,</span>
<span class="source-line-no">955</span><span id="line-955"> KeeperException, ReplicationException, DeserializationException {</span>
<span class="source-line-no">956</span><span id="line-956"> /*</span>
<span class="source-line-no">957</span><span id="line-957"> * We are active master now... go initialize components we need to run.</span>
<span class="source-line-no">958</span><span id="line-958"> */</span>
<span class="source-line-no">959</span><span id="line-959"> startupTaskGroup.addTask("Initializing Master file system");</span>
<span class="source-line-no">960</span><span id="line-960"></span>
<span class="source-line-no">961</span><span id="line-961"> this.masterActiveTime = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">962</span><span id="line-962"> // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.</span>
<span class="source-line-no">963</span><span id="line-963"></span>
<span class="source-line-no">964</span><span id="line-964"> // always initialize the MemStoreLAB as we use a region to store data in master now, see</span>
<span class="source-line-no">965</span><span id="line-965"> // localStore.</span>
<span class="source-line-no">966</span><span id="line-966"> initializeMemStoreChunkCreator(null);</span>
<span class="source-line-no">967</span><span id="line-967"> this.fileSystemManager = new MasterFileSystem(conf);</span>
<span class="source-line-no">968</span><span id="line-968"> this.walManager = new MasterWalManager(this);</span>
<span class="source-line-no">969</span><span id="line-969"></span>
<span class="source-line-no">970</span><span id="line-970"> // warm-up HTDs cache on master initialization</span>
<span class="source-line-no">971</span><span id="line-971"> if (preLoadTableDescriptors) {</span>
<span class="source-line-no">972</span><span id="line-972"> startupTaskGroup.addTask("Pre-loading table descriptors");</span>
<span class="source-line-no">973</span><span id="line-973"> this.tableDescriptors.getAll();</span>
<span class="source-line-no">974</span><span id="line-974"> }</span>
<span class="source-line-no">975</span><span id="line-975"></span>
<span class="source-line-no">976</span><span id="line-976"> // Publish cluster ID; set it in Master too. The superclass RegionServer does this later but</span>
<span class="source-line-no">977</span><span id="line-977"> // only after it has checked in with the Master. At least a few tests ask Master for clusterId</span>
<span class="source-line-no">978</span><span id="line-978"> // before it has called its run method and before RegionServer has done the reportForDuty.</span>
<span class="source-line-no">979</span><span id="line-979"> ClusterId clusterId = fileSystemManager.getClusterId();</span>
<span class="source-line-no">980</span><span id="line-980"> startupTaskGroup.addTask("Publishing Cluster ID " + clusterId + " in ZooKeeper");</span>
<span class="source-line-no">981</span><span id="line-981"> ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());</span>
<span class="source-line-no">982</span><span id="line-982"> this.clusterId = clusterId.toString();</span>
<span class="source-line-no">983</span><span id="line-983"></span>
<span class="source-line-no">984</span><span id="line-984"> // Precaution. Put in place the old hbck1 lock file to fence out old hbase1s running their</span>
<span class="source-line-no">985</span><span id="line-985"> // hbck1s against an hbase2 cluster; it could do damage. To skip this behavior, set</span>
<span class="source-line-no">986</span><span id="line-986"> // hbase.write.hbck1.lock.file to false.</span>
<span class="source-line-no">987</span><span id="line-987"> if (this.conf.getBoolean("hbase.write.hbck1.lock.file", true)) {</span>
<span class="source-line-no">988</span><span id="line-988"> Pair&lt;Path, FSDataOutputStream&gt; result = null;</span>
<span class="source-line-no">989</span><span id="line-989"> try {</span>
<span class="source-line-no">990</span><span id="line-990"> result = HBaseFsck.checkAndMarkRunningHbck(this.conf,</span>
<span class="source-line-no">991</span><span id="line-991"> HBaseFsck.createLockRetryCounterFactory(this.conf).create());</span>
<span class="source-line-no">992</span><span id="line-992"> } finally {</span>
<span class="source-line-no">993</span><span id="line-993"> if (result != null) {</span>
<span class="source-line-no">994</span><span id="line-994"> Closeables.close(result.getSecond(), true);</span>
<span class="source-line-no">995</span><span id="line-995"> }</span>
<span class="source-line-no">996</span><span id="line-996"> }</span>
<span class="source-line-no">997</span><span id="line-997"> }</span>
<span class="source-line-no">998</span><span id="line-998"></span>
<span class="source-line-no">999</span><span id="line-999"> startupTaskGroup.addTask("Initialize ServerManager and schedule SCP for crash servers");</span>
<span class="source-line-no">1000</span><span id="line-1000"> // The below two managers must be created before loading procedures, as they will be used during</span>
<span class="source-line-no">1001</span><span id="line-1001"> // loading.</span>
<span class="source-line-no">1002</span><span id="line-1002"> // initialize master local region</span>
<span class="source-line-no">1003</span><span id="line-1003"> masterRegion = MasterRegionFactory.create(this);</span>
<span class="source-line-no">1004</span><span id="line-1004"> rsListStorage = new MasterRegionServerList(masterRegion, this);</span>
<span class="source-line-no">1005</span><span id="line-1005"></span>
<span class="source-line-no">1006</span><span id="line-1006"> // Initialize the ServerManager and register it as a configuration observer</span>
<span class="source-line-no">1007</span><span id="line-1007"> this.serverManager = createServerManager(this, rsListStorage);</span>
<span class="source-line-no">1008</span><span id="line-1008"> this.configurationManager.registerObserver(this.serverManager);</span>
<span class="source-line-no">1009</span><span id="line-1009"></span>
<span class="source-line-no">1010</span><span id="line-1010"> this.syncReplicationReplayWALManager = new SyncReplicationReplayWALManager(this);</span>
<span class="source-line-no">1011</span><span id="line-1011"> if (</span>
<span class="source-line-no">1012</span><span id="line-1012"> !conf.getBoolean(HBASE_SPLIT_WAL_COORDINATED_BY_ZK, DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK)</span>
<span class="source-line-no">1013</span><span id="line-1013"> ) {</span>
<span class="source-line-no">1014</span><span id="line-1014"> this.splitWALManager = new SplitWALManager(this);</span>
<span class="source-line-no">1015</span><span id="line-1015"> }</span>
<span class="source-line-no">1016</span><span id="line-1016"></span>
<span class="source-line-no">1017</span><span id="line-1017"> tryMigrateMetaLocationsFromZooKeeper();</span>
<span class="source-line-no">1018</span><span id="line-1018"></span>
<span class="source-line-no">1019</span><span id="line-1019"> createProcedureExecutor();</span>
<span class="source-line-no">1020</span><span id="line-1020"> Map&lt;Class&lt;?&gt;, List&lt;Procedure&lt;MasterProcedureEnv&gt;&gt;&gt; procsByType = procedureExecutor</span>
<span class="source-line-no">1021</span><span id="line-1021"> .getActiveProceduresNoCopy().stream().collect(Collectors.groupingBy(p -&gt; p.getClass()));</span>
<span class="source-line-no">1022</span><span id="line-1022"></span>
<span class="source-line-no">1023</span><span id="line-1023"> // Create Assignment Manager</span>
<span class="source-line-no">1024</span><span id="line-1024"> this.assignmentManager = createAssignmentManager(this, masterRegion);</span>
<span class="source-line-no">1025</span><span id="line-1025"> this.assignmentManager.start();</span>
<span class="source-line-no">1026</span><span id="line-1026"> // TODO: TRSP can perform as the sub procedure for other procedures, so even if it is marked as</span>
<span class="source-line-no">1027</span><span id="line-1027"> // completed, it could still be in the procedure list. This is a bit strange but is another</span>
<span class="source-line-no">1028</span><span id="line-1028"> // story, need to verify the implementation for ProcedureExecutor and ProcedureStore.</span>
<span class="source-line-no">1029</span><span id="line-1029"> List&lt;TransitRegionStateProcedure&gt; ritList =</span>
<span class="source-line-no">1030</span><span id="line-1030"> procsByType.getOrDefault(TransitRegionStateProcedure.class, Collections.emptyList()).stream()</span>
<span class="source-line-no">1031</span><span id="line-1031"> .filter(p -&gt; !p.isFinished()).map(p -&gt; (TransitRegionStateProcedure) p)</span>
<span class="source-line-no">1032</span><span id="line-1032"> .collect(Collectors.toList());</span>
<span class="source-line-no">1033</span><span id="line-1033"> this.assignmentManager.setupRIT(ritList);</span>
<span class="source-line-no">1034</span><span id="line-1034"></span>
<span class="source-line-no">1035</span><span id="line-1035"> // Start RegionServerTracker with listing of servers found with exiting SCPs -- these should</span>
<span class="source-line-no">1036</span><span id="line-1036"> // be registered in the deadServers set -- and the servernames loaded from the WAL directory</span>
<span class="source-line-no">1037</span><span id="line-1037"> // and master local region that COULD BE 'alive'(we'll schedule SCPs for each and let SCP figure</span>
<span class="source-line-no">1038</span><span id="line-1038"> // it out).</span>
<span class="source-line-no">1039</span><span id="line-1039"> // We also pass dirs that are already 'splitting'... so we can do some checks down in tracker.</span>
<span class="source-line-no">1040</span><span id="line-1040"> // TODO: Generate the splitting and live Set in one pass instead of two as we currently do.</span>
<span class="source-line-no">1041</span><span id="line-1041"> this.regionServerTracker.upgrade(</span>
<span class="source-line-no">1042</span><span id="line-1042"> procsByType.getOrDefault(ServerCrashProcedure.class, Collections.emptyList()).stream()</span>
<span class="source-line-no">1043</span><span id="line-1043"> .map(p -&gt; (ServerCrashProcedure) p).map(p -&gt; p.getServerName()).collect(Collectors.toSet()),</span>
<span class="source-line-no">1044</span><span id="line-1044"> Sets.union(rsListStorage.getAll(), walManager.getLiveServersFromWALDir()),</span>
<span class="source-line-no">1045</span><span id="line-1045"> walManager.getSplittingServersFromWALDir());</span>
<span class="source-line-no">1046</span><span id="line-1046"> // This manager must be accessed AFTER hbase:meta is confirmed on line..</span>
<span class="source-line-no">1047</span><span id="line-1047"> this.tableStateManager = new TableStateManager(this);</span>
<span class="source-line-no">1048</span><span id="line-1048"></span>
<span class="source-line-no">1049</span><span id="line-1049"> startupTaskGroup.addTask("Initializing ZK system trackers");</span>
<span class="source-line-no">1050</span><span id="line-1050"> initializeZKBasedSystemTrackers();</span>
<span class="source-line-no">1051</span><span id="line-1051"> startupTaskGroup.addTask("Loading last flushed sequence id of regions");</span>
<span class="source-line-no">1052</span><span id="line-1052"> try {</span>
<span class="source-line-no">1053</span><span id="line-1053"> this.serverManager.loadLastFlushedSequenceIds();</span>
<span class="source-line-no">1054</span><span id="line-1054"> } catch (IOException e) {</span>
<span class="source-line-no">1055</span><span id="line-1055"> LOG.info("Failed to load last flushed sequence id of regions" + " from file system", e);</span>
<span class="source-line-no">1056</span><span id="line-1056"> }</span>
<span class="source-line-no">1057</span><span id="line-1057"> // Set ourselves as active Master now our claim has succeeded up in zk.</span>
<span class="source-line-no">1058</span><span id="line-1058"> this.activeMaster = true;</span>
<span class="source-line-no">1059</span><span id="line-1059"></span>
<span class="source-line-no">1060</span><span id="line-1060"> // Start the Zombie master detector after setting master as active, see HBASE-21535</span>
<span class="source-line-no">1061</span><span id="line-1061"> Thread zombieDetector = new Thread(new MasterInitializationMonitor(this),</span>
<span class="source-line-no">1062</span><span id="line-1062"> "ActiveMasterInitializationMonitor-" + EnvironmentEdgeManager.currentTime());</span>
<span class="source-line-no">1063</span><span id="line-1063"> zombieDetector.setDaemon(true);</span>
<span class="source-line-no">1064</span><span id="line-1064"> zombieDetector.start();</span>
<span class="source-line-no">1065</span><span id="line-1065"></span>
<span class="source-line-no">1066</span><span id="line-1066"> if (!maintenanceMode) {</span>
<span class="source-line-no">1067</span><span id="line-1067"> startupTaskGroup.addTask("Initializing master coprocessors");</span>
<span class="source-line-no">1068</span><span id="line-1068"> setQuotasObserver(conf);</span>
<span class="source-line-no">1069</span><span id="line-1069"> initializeCoprocessorHost(conf);</span>
<span class="source-line-no">1070</span><span id="line-1070"> } else {</span>
<span class="source-line-no">1071</span><span id="line-1071"> // start an in process region server for carrying system regions</span>
<span class="source-line-no">1072</span><span id="line-1072"> maintenanceRegionServer =</span>
<span class="source-line-no">1073</span><span id="line-1073"> JVMClusterUtil.createRegionServerThread(getConfiguration(), HRegionServer.class, 0);</span>
<span class="source-line-no">1074</span><span id="line-1074"> maintenanceRegionServer.start();</span>
<span class="source-line-no">1075</span><span id="line-1075"> }</span>
<span class="source-line-no">1076</span><span id="line-1076"></span>
<span class="source-line-no">1077</span><span id="line-1077"> // Checking if meta needs initializing.</span>
<span class="source-line-no">1078</span><span id="line-1078"> startupTaskGroup.addTask("Initializing meta table if this is a new deploy");</span>
<span class="source-line-no">1079</span><span id="line-1079"> InitMetaProcedure initMetaProc = null;</span>
<span class="source-line-no">1080</span><span id="line-1080"> // Print out state of hbase:meta on startup; helps debugging.</span>
<span class="source-line-no">1081</span><span id="line-1081"> if (!this.assignmentManager.getRegionStates().hasTableRegionStates(TableName.META_TABLE_NAME)) {</span>
<span class="source-line-no">1082</span><span id="line-1082"> Optional&lt;InitMetaProcedure&gt; optProc = procedureExecutor.getProcedures().stream()</span>
<span class="source-line-no">1083</span><span id="line-1083"> .filter(p -&gt; p instanceof InitMetaProcedure).map(o -&gt; (InitMetaProcedure) o).findAny();</span>
<span class="source-line-no">1084</span><span id="line-1084"> initMetaProc = optProc.orElseGet(() -&gt; {</span>
<span class="source-line-no">1085</span><span id="line-1085"> // schedule an init meta procedure if meta has not been deployed yet</span>
<span class="source-line-no">1086</span><span id="line-1086"> InitMetaProcedure temp = new InitMetaProcedure();</span>
<span class="source-line-no">1087</span><span id="line-1087"> procedureExecutor.submitProcedure(temp);</span>
<span class="source-line-no">1088</span><span id="line-1088"> return temp;</span>
<span class="source-line-no">1089</span><span id="line-1089"> });</span>
<span class="source-line-no">1090</span><span id="line-1090"> }</span>
<span class="source-line-no">1091</span><span id="line-1091"></span>
<span class="source-line-no">1092</span><span id="line-1092"> // initialize load balancer</span>
<span class="source-line-no">1093</span><span id="line-1093"> this.balancer.setMasterServices(this);</span>
<span class="source-line-no">1094</span><span id="line-1094"> this.balancer.initialize();</span>
<span class="source-line-no">1095</span><span id="line-1095"> this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());</span>
<span class="source-line-no">1096</span><span id="line-1096"></span>
<span class="source-line-no">1097</span><span id="line-1097"> // try migrate replication data</span>
<span class="source-line-no">1098</span><span id="line-1098"> ZKReplicationQueueStorageForMigration oldReplicationQueueStorage =</span>
<span class="source-line-no">1099</span><span id="line-1099"> new ZKReplicationQueueStorageForMigration(zooKeeper, conf);</span>
<span class="source-line-no">1100</span><span id="line-1100"> // check whether there are something to migrate and we haven't scheduled a migration procedure</span>
<span class="source-line-no">1101</span><span id="line-1101"> // yet</span>
<span class="source-line-no">1102</span><span id="line-1102"> if (</span>
<span class="source-line-no">1103</span><span id="line-1103"> oldReplicationQueueStorage.hasData() &amp;&amp; procedureExecutor.getProcedures().stream()</span>
<span class="source-line-no">1104</span><span id="line-1104"> .allMatch(p -&gt; !(p instanceof MigrateReplicationQueueFromZkToTableProcedure))</span>
<span class="source-line-no">1105</span><span id="line-1105"> ) {</span>
<span class="source-line-no">1106</span><span id="line-1106"> procedureExecutor.submitProcedure(new MigrateReplicationQueueFromZkToTableProcedure());</span>
<span class="source-line-no">1107</span><span id="line-1107"> }</span>
<span class="source-line-no">1108</span><span id="line-1108"> // start up all service threads.</span>
<span class="source-line-no">1109</span><span id="line-1109"> startupTaskGroup.addTask("Initializing master service threads");</span>
<span class="source-line-no">1110</span><span id="line-1110"> startServiceThreads();</span>
<span class="source-line-no">1111</span><span id="line-1111"> // wait meta to be initialized after we start procedure executor</span>
<span class="source-line-no">1112</span><span id="line-1112"> if (initMetaProc != null) {</span>
<span class="source-line-no">1113</span><span id="line-1113"> initMetaProc.await();</span>
<span class="source-line-no">1114</span><span id="line-1114"> }</span>
<span class="source-line-no">1115</span><span id="line-1115"> // Wake up this server to check in</span>
<span class="source-line-no">1116</span><span id="line-1116"> sleeper.skipSleepCycle();</span>
<span class="source-line-no">1117</span><span id="line-1117"></span>
<span class="source-line-no">1118</span><span id="line-1118"> // Wait for region servers to report in.</span>
<span class="source-line-no">1119</span><span id="line-1119"> // With this as part of master initialization, it precludes our being able to start a single</span>
<span class="source-line-no">1120</span><span id="line-1120"> // server that is both Master and RegionServer. Needs more thought. TODO.</span>
<span class="source-line-no">1121</span><span id="line-1121"> String statusStr = "Wait for region servers to report in";</span>
<span class="source-line-no">1122</span><span id="line-1122"> MonitoredTask waitRegionServer = startupTaskGroup.addTask(statusStr);</span>
<span class="source-line-no">1123</span><span id="line-1123"> LOG.info(Objects.toString(waitRegionServer));</span>
<span class="source-line-no">1124</span><span id="line-1124"> waitForRegionServers(waitRegionServer);</span>
<span class="source-line-no">1125</span><span id="line-1125"></span>
<span class="source-line-no">1126</span><span id="line-1126"> // Check if master is shutting down because issue initializing regionservers or balancer.</span>
<span class="source-line-no">1127</span><span id="line-1127"> if (isStopped()) {</span>
<span class="source-line-no">1128</span><span id="line-1128"> return;</span>
<span class="source-line-no">1129</span><span id="line-1129"> }</span>
<span class="source-line-no">1130</span><span id="line-1130"></span>
<span class="source-line-no">1131</span><span id="line-1131"> startupTaskGroup.addTask("Starting assignment manager");</span>
<span class="source-line-no">1132</span><span id="line-1132"> // FIRST HBASE:META READ!!!!</span>
<span class="source-line-no">1133</span><span id="line-1133"> // The below cannot make progress w/o hbase:meta being online.</span>
<span class="source-line-no">1134</span><span id="line-1134"> // This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background</span>
<span class="source-line-no">1135</span><span id="line-1135"> // as procedures run -- in particular SCPs for crashed servers... One should put up hbase:meta</span>
<span class="source-line-no">1136</span><span id="line-1136"> // if it is down. It may take a while to come online. So, wait here until meta if for sure</span>
<span class="source-line-no">1137</span><span id="line-1137"> // available. That's what waitForMetaOnline does.</span>
<span class="source-line-no">1138</span><span id="line-1138"> if (!waitForMetaOnline()) {</span>
<span class="source-line-no">1139</span><span id="line-1139"> return;</span>
<span class="source-line-no">1140</span><span id="line-1140"> }</span>
<span class="source-line-no">1141</span><span id="line-1141"></span>
<span class="source-line-no">1142</span><span id="line-1142"> TableDescriptor metaDescriptor = tableDescriptors.get(TableName.META_TABLE_NAME);</span>
<span class="source-line-no">1143</span><span id="line-1143"> final ColumnFamilyDescriptor tableFamilyDesc =</span>
<span class="source-line-no">1144</span><span id="line-1144"> metaDescriptor.getColumnFamily(HConstants.TABLE_FAMILY);</span>
<span class="source-line-no">1145</span><span id="line-1145"> final ColumnFamilyDescriptor replBarrierFamilyDesc =</span>
<span class="source-line-no">1146</span><span id="line-1146"> metaDescriptor.getColumnFamily(HConstants.REPLICATION_BARRIER_FAMILY);</span>
<span class="source-line-no">1147</span><span id="line-1147"></span>
<span class="source-line-no">1148</span><span id="line-1148"> this.assignmentManager.joinCluster();</span>
<span class="source-line-no">1149</span><span id="line-1149"> // The below depends on hbase:meta being online.</span>
<span class="source-line-no">1150</span><span id="line-1150"> this.assignmentManager.processOfflineRegions();</span>
<span class="source-line-no">1151</span><span id="line-1151"> // this must be called after the above processOfflineRegions to prevent race</span>
<span class="source-line-no">1152</span><span id="line-1152"> this.assignmentManager.wakeMetaLoadedEvent();</span>
<span class="source-line-no">1153</span><span id="line-1153"></span>
<span class="source-line-no">1154</span><span id="line-1154"> // for migrating from a version without HBASE-25099, and also for honoring the configuration</span>
<span class="source-line-no">1155</span><span id="line-1155"> // first.</span>
<span class="source-line-no">1156</span><span id="line-1156"> if (conf.get(HConstants.META_REPLICAS_NUM) != null) {</span>
<span class="source-line-no">1157</span><span id="line-1157"> int replicasNumInConf =</span>
<span class="source-line-no">1158</span><span id="line-1158"> conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM);</span>
<span class="source-line-no">1159</span><span id="line-1159"> TableDescriptor metaDesc = tableDescriptors.get(TableName.META_TABLE_NAME);</span>
<span class="source-line-no">1160</span><span id="line-1160"> if (metaDesc.getRegionReplication() != replicasNumInConf) {</span>
<span class="source-line-no">1161</span><span id="line-1161"> // it is possible that we already have some replicas before upgrading, so we must set the</span>
<span class="source-line-no">1162</span><span id="line-1162"> // region replication number in meta TableDescriptor directly first, without creating a</span>
<span class="source-line-no">1163</span><span id="line-1163"> // ModifyTableProcedure, otherwise it may cause a double assign for the meta replicas.</span>
<span class="source-line-no">1164</span><span id="line-1164"> int existingReplicasCount =</span>
<span class="source-line-no">1165</span><span id="line-1165"> assignmentManager.getRegionStates().getRegionsOfTable(TableName.META_TABLE_NAME).size();</span>
<span class="source-line-no">1166</span><span id="line-1166"> if (existingReplicasCount &gt; metaDesc.getRegionReplication()) {</span>
<span class="source-line-no">1167</span><span id="line-1167"> LOG.info("Update replica count of hbase:meta from {}(in TableDescriptor)"</span>
<span class="source-line-no">1168</span><span id="line-1168"> + " to {}(existing ZNodes)", metaDesc.getRegionReplication(), existingReplicasCount);</span>
<span class="source-line-no">1169</span><span id="line-1169"> metaDesc = TableDescriptorBuilder.newBuilder(metaDesc)</span>
<span class="source-line-no">1170</span><span id="line-1170"> .setRegionReplication(existingReplicasCount).build();</span>
<span class="source-line-no">1171</span><span id="line-1171"> tableDescriptors.update(metaDesc);</span>
<span class="source-line-no">1172</span><span id="line-1172"> }</span>
<span class="source-line-no">1173</span><span id="line-1173"> // check again, and issue a ModifyTableProcedure if needed</span>
<span class="source-line-no">1174</span><span id="line-1174"> if (metaDesc.getRegionReplication() != replicasNumInConf) {</span>
<span class="source-line-no">1175</span><span id="line-1175"> LOG.info(</span>
<span class="source-line-no">1176</span><span id="line-1176"> "The {} config is {} while the replica count in TableDescriptor is {}"</span>
<span class="source-line-no">1177</span><span id="line-1177"> + " for hbase:meta, altering...",</span>
<span class="source-line-no">1178</span><span id="line-1178"> HConstants.META_REPLICAS_NUM, replicasNumInConf, metaDesc.getRegionReplication());</span>
<span class="source-line-no">1179</span><span id="line-1179"> procedureExecutor.submitProcedure(new ModifyTableProcedure(</span>
<span class="source-line-no">1180</span><span id="line-1180"> procedureExecutor.getEnvironment(), TableDescriptorBuilder.newBuilder(metaDesc)</span>
<span class="source-line-no">1181</span><span id="line-1181"> .setRegionReplication(replicasNumInConf).build(),</span>
<span class="source-line-no">1182</span><span id="line-1182"> null, metaDesc, false, true));</span>
<span class="source-line-no">1183</span><span id="line-1183"> }</span>
<span class="source-line-no">1184</span><span id="line-1184"> }</span>
<span class="source-line-no">1185</span><span id="line-1185"> }</span>
<span class="source-line-no">1186</span><span id="line-1186"> // Initialize after meta is up as below scans meta</span>
<span class="source-line-no">1187</span><span id="line-1187"> FavoredNodesManager fnm = getFavoredNodesManager();</span>
<span class="source-line-no">1188</span><span id="line-1188"> if (fnm != null) {</span>
<span class="source-line-no">1189</span><span id="line-1189"> fnm.initializeFromMeta();</span>
<span class="source-line-no">1190</span><span id="line-1190"> }</span>
<span class="source-line-no">1191</span><span id="line-1191"></span>
<span class="source-line-no">1192</span><span id="line-1192"> // set cluster status again after user regions are assigned</span>
<span class="source-line-no">1193</span><span id="line-1193"> this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());</span>
<span class="source-line-no">1194</span><span id="line-1194"></span>
<span class="source-line-no">1195</span><span id="line-1195"> // Start balancer and meta catalog janitor after meta and regions have been assigned.</span>
<span class="source-line-no">1196</span><span id="line-1196"> startupTaskGroup.addTask("Starting balancer and catalog janitor");</span>
<span class="source-line-no">1197</span><span id="line-1197"> this.clusterStatusChore = new ClusterStatusChore(this, balancer);</span>
<span class="source-line-no">1198</span><span id="line-1198"> getChoreService().scheduleChore(clusterStatusChore);</span>
<span class="source-line-no">1199</span><span id="line-1199"> this.balancerChore = new BalancerChore(this);</span>
<span class="source-line-no">1200</span><span id="line-1200"> if (!disableBalancerChoreForTest) {</span>
<span class="source-line-no">1201</span><span id="line-1201"> getChoreService().scheduleChore(balancerChore);</span>
<span class="source-line-no">1202</span><span id="line-1202"> }</span>
<span class="source-line-no">1203</span><span id="line-1203"> if (regionNormalizerManager != null) {</span>
<span class="source-line-no">1204</span><span id="line-1204"> getChoreService().scheduleChore(regionNormalizerManager.getRegionNormalizerChore());</span>
<span class="source-line-no">1205</span><span id="line-1205"> }</span>
<span class="source-line-no">1206</span><span id="line-1206"> this.catalogJanitorChore = new CatalogJanitor(this);</span>
<span class="source-line-no">1207</span><span id="line-1207"> getChoreService().scheduleChore(catalogJanitorChore);</span>
<span class="source-line-no">1208</span><span id="line-1208"> this.hbckChore = new HbckChore(this);</span>
<span class="source-line-no">1209</span><span id="line-1209"> getChoreService().scheduleChore(hbckChore);</span>
<span class="source-line-no">1210</span><span id="line-1210"> this.serverManager.startChore();</span>
<span class="source-line-no">1211</span><span id="line-1211"></span>
<span class="source-line-no">1212</span><span id="line-1212"> // Only for rolling upgrade, where we need to migrate the data in namespace table to meta table.</span>
<span class="source-line-no">1213</span><span id="line-1213"> if (!waitForNamespaceOnline()) {</span>
<span class="source-line-no">1214</span><span id="line-1214"> return;</span>
<span class="source-line-no">1215</span><span id="line-1215"> }</span>
<span class="source-line-no">1216</span><span id="line-1216"> startupTaskGroup.addTask("Starting cluster schema service");</span>
<span class="source-line-no">1217</span><span id="line-1217"> try {</span>
<span class="source-line-no">1218</span><span id="line-1218"> initClusterSchemaService();</span>
<span class="source-line-no">1219</span><span id="line-1219"> } catch (IllegalStateException e) {</span>
<span class="source-line-no">1220</span><span id="line-1220"> if (</span>
<span class="source-line-no">1221</span><span id="line-1221"> e.getCause() != null &amp;&amp; e.getCause() instanceof NoSuchColumnFamilyException</span>
<span class="source-line-no">1222</span><span id="line-1222"> &amp;&amp; tableFamilyDesc == null &amp;&amp; replBarrierFamilyDesc == null</span>
<span class="source-line-no">1223</span><span id="line-1223"> ) {</span>
<span class="source-line-no">1224</span><span id="line-1224"> LOG.info("ClusterSchema service could not be initialized. This is "</span>
<span class="source-line-no">1225</span><span id="line-1225"> + "expected during HBase 1 to 2 upgrade", e);</span>
<span class="source-line-no">1226</span><span id="line-1226"> } else {</span>
<span class="source-line-no">1227</span><span id="line-1227"> throw e;</span>
<span class="source-line-no">1228</span><span id="line-1228"> }</span>
<span class="source-line-no">1229</span><span id="line-1229"> }</span>
<span class="source-line-no">1230</span><span id="line-1230"></span>
<span class="source-line-no">1231</span><span id="line-1231"> if (this.cpHost != null) {</span>
<span class="source-line-no">1232</span><span id="line-1232"> try {</span>
<span class="source-line-no">1233</span><span id="line-1233"> this.cpHost.preMasterInitialization();</span>
<span class="source-line-no">1234</span><span id="line-1234"> } catch (IOException e) {</span>
<span class="source-line-no">1235</span><span id="line-1235"> LOG.error("Coprocessor preMasterInitialization() hook failed", e);</span>
<span class="source-line-no">1236</span><span id="line-1236"> }</span>
<span class="source-line-no">1237</span><span id="line-1237"> }</span>
<span class="source-line-no">1238</span><span id="line-1238"></span>
<span class="source-line-no">1239</span><span id="line-1239"> LOG.info(String.format("Master has completed initialization %.3fsec",</span>
<span class="source-line-no">1240</span><span id="line-1240"> (EnvironmentEdgeManager.currentTime() - masterActiveTime) / 1000.0f));</span>
<span class="source-line-no">1241</span><span id="line-1241"> this.masterFinishedInitializationTime = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">1242</span><span id="line-1242"> configurationManager.registerObserver(this.balancer);</span>
<span class="source-line-no">1243</span><span id="line-1243"> configurationManager.registerObserver(this.logCleanerPool);</span>
<span class="source-line-no">1244</span><span id="line-1244"> configurationManager.registerObserver(this.logCleaner);</span>
<span class="source-line-no">1245</span><span id="line-1245"> configurationManager.registerObserver(this.regionsRecoveryConfigManager);</span>
<span class="source-line-no">1246</span><span id="line-1246"> configurationManager.registerObserver(this.exclusiveHFileCleanerPool);</span>
<span class="source-line-no">1247</span><span id="line-1247"> if (this.sharedHFileCleanerPool != null) {</span>
<span class="source-line-no">1248</span><span id="line-1248"> configurationManager.registerObserver(this.sharedHFileCleanerPool);</span>
<span class="source-line-no">1249</span><span id="line-1249"> }</span>
<span class="source-line-no">1250</span><span id="line-1250"> if (this.hfileCleaners != null) {</span>
<span class="source-line-no">1251</span><span id="line-1251"> for (HFileCleaner cleaner : hfileCleaners) {</span>
<span class="source-line-no">1252</span><span id="line-1252"> configurationManager.registerObserver(cleaner);</span>
<span class="source-line-no">1253</span><span id="line-1253"> }</span>
<span class="source-line-no">1254</span><span id="line-1254"> }</span>
<span class="source-line-no">1255</span><span id="line-1255"> // Set master as 'initialized'.</span>
<span class="source-line-no">1256</span><span id="line-1256"> setInitialized(true);</span>
<span class="source-line-no">1257</span><span id="line-1257"> startupTaskGroup.markComplete("Initialization successful");</span>
<span class="source-line-no">1258</span><span id="line-1258"> MonitoredTask status =</span>
<span class="source-line-no">1259</span><span id="line-1259"> TaskMonitor.get().createStatus("Progress after master initialized", false, true);</span>
<span class="source-line-no">1260</span><span id="line-1260"></span>
<span class="source-line-no">1261</span><span id="line-1261"> if (tableFamilyDesc == null &amp;&amp; replBarrierFamilyDesc == null) {</span>
<span class="source-line-no">1262</span><span id="line-1262"> // create missing CFs in meta table after master is set to 'initialized'.</span>
<span class="source-line-no">1263</span><span id="line-1263"> createMissingCFsInMetaDuringUpgrade(metaDescriptor);</span>
<span class="source-line-no">1264</span><span id="line-1264"></span>
<span class="source-line-no">1265</span><span id="line-1265"> // Throwing this Exception to abort active master is painful but this</span>
<span class="source-line-no">1266</span><span id="line-1266"> // seems the only way to add missing CFs in meta while upgrading from</span>
<span class="source-line-no">1267</span><span id="line-1267"> // HBase 1 to 2 (where HBase 2 has HBASE-23055 &amp; HBASE-23782 checked-in).</span>
<span class="source-line-no">1268</span><span id="line-1268"> // So, why do we abort active master after adding missing CFs in meta?</span>
<span class="source-line-no">1269</span><span id="line-1269"> // When we reach here, we would have already bypassed NoSuchColumnFamilyException</span>
<span class="source-line-no">1270</span><span id="line-1270"> // in initClusterSchemaService(), meaning ClusterSchemaService is not</span>
<span class="source-line-no">1271</span><span id="line-1271"> // correctly initialized but we bypassed it. Similarly, we bypassed</span>
<span class="source-line-no">1272</span><span id="line-1272"> // tableStateManager.start() as well. Hence, we should better abort</span>
<span class="source-line-no">1273</span><span id="line-1273"> // current active master because our main task - adding missing CFs</span>
<span class="source-line-no">1274</span><span id="line-1274"> // in meta table is done (possible only after master state is set as</span>
<span class="source-line-no">1275</span><span id="line-1275"> // initialized) at the expense of bypassing few important tasks as part</span>
<span class="source-line-no">1276</span><span id="line-1276"> // of active master init routine. So now we abort active master so that</span>
<span class="source-line-no">1277</span><span id="line-1277"> // next active master init will not face any issues and all mandatory</span>
<span class="source-line-no">1278</span><span id="line-1278"> // services will be started during master init phase.</span>
<span class="source-line-no">1279</span><span id="line-1279"> throw new PleaseRestartMasterException("Aborting active master after missing"</span>
<span class="source-line-no">1280</span><span id="line-1280"> + " CFs are successfully added in meta. Subsequent active master "</span>
<span class="source-line-no">1281</span><span id="line-1281"> + "initialization should be uninterrupted");</span>
<span class="source-line-no">1282</span><span id="line-1282"> }</span>
<span class="source-line-no">1283</span><span id="line-1283"></span>
<span class="source-line-no">1284</span><span id="line-1284"> if (maintenanceMode) {</span>
<span class="source-line-no">1285</span><span id="line-1285"> LOG.info("Detected repair mode, skipping final initialization steps.");</span>
<span class="source-line-no">1286</span><span id="line-1286"> return;</span>
<span class="source-line-no">1287</span><span id="line-1287"> }</span>
<span class="source-line-no">1288</span><span id="line-1288"></span>
<span class="source-line-no">1289</span><span id="line-1289"> assignmentManager.checkIfShouldMoveSystemRegionAsync();</span>
<span class="source-line-no">1290</span><span id="line-1290"> status.setStatus("Starting quota manager");</span>
<span class="source-line-no">1291</span><span id="line-1291"> initQuotaManager();</span>
<span class="source-line-no">1292</span><span id="line-1292"> if (QuotaUtil.isQuotaEnabled(conf)) {</span>
<span class="source-line-no">1293</span><span id="line-1293"> // Create the quota snapshot notifier</span>
<span class="source-line-no">1294</span><span id="line-1294"> spaceQuotaSnapshotNotifier = createQuotaSnapshotNotifier();</span>
<span class="source-line-no">1295</span><span id="line-1295"> spaceQuotaSnapshotNotifier.initialize(getConnection());</span>
<span class="source-line-no">1296</span><span id="line-1296"> this.quotaObserverChore = new QuotaObserverChore(this, getMasterMetrics());</span>
<span class="source-line-no">1297</span><span id="line-1297"> // Start the chore to read the region FS space reports and act on them</span>
<span class="source-line-no">1298</span><span id="line-1298"> getChoreService().scheduleChore(quotaObserverChore);</span>
<span class="source-line-no">1299</span><span id="line-1299"></span>
<span class="source-line-no">1300</span><span id="line-1300"> this.snapshotQuotaChore = new SnapshotQuotaObserverChore(this, getMasterMetrics());</span>
<span class="source-line-no">1301</span><span id="line-1301"> // Start the chore to read snapshots and add their usage to table/NS quotas</span>
<span class="source-line-no">1302</span><span id="line-1302"> getChoreService().scheduleChore(snapshotQuotaChore);</span>
<span class="source-line-no">1303</span><span id="line-1303"> }</span>
<span class="source-line-no">1304</span><span id="line-1304"> final SlowLogMasterService slowLogMasterService = new SlowLogMasterService(conf, this);</span>
<span class="source-line-no">1305</span><span id="line-1305"> slowLogMasterService.init();</span>
<span class="source-line-no">1306</span><span id="line-1306"></span>
<span class="source-line-no">1307</span><span id="line-1307"> WALEventTrackerTableCreator.createIfNeededAndNotExists(conf, this);</span>
<span class="source-line-no">1308</span><span id="line-1308"> // Create REPLICATION.SINK_TRACKER table if needed.</span>
<span class="source-line-no">1309</span><span id="line-1309"> ReplicationSinkTrackerTableCreator.createIfNeededAndNotExists(conf, this);</span>
<span class="source-line-no">1310</span><span id="line-1310"></span>
<span class="source-line-no">1311</span><span id="line-1311"> // clear the dead servers with same host name and port of online server because we are not</span>
<span class="source-line-no">1312</span><span id="line-1312"> // removing dead server with same hostname and port of rs which is trying to check in before</span>
<span class="source-line-no">1313</span><span id="line-1313"> // master initialization. See HBASE-5916.</span>
<span class="source-line-no">1314</span><span id="line-1314"> this.serverManager.clearDeadServersWithSameHostNameAndPortOfOnlineServer();</span>
<span class="source-line-no">1315</span><span id="line-1315"></span>
<span class="source-line-no">1316</span><span id="line-1316"> // Check and set the znode ACLs if needed in case we are overtaking a non-secure configuration</span>
<span class="source-line-no">1317</span><span id="line-1317"> status.setStatus("Checking ZNode ACLs");</span>
<span class="source-line-no">1318</span><span id="line-1318"> zooKeeper.checkAndSetZNodeAcls();</span>
<span class="source-line-no">1319</span><span id="line-1319"></span>
<span class="source-line-no">1320</span><span id="line-1320"> status.setStatus("Initializing MOB Cleaner");</span>
<span class="source-line-no">1321</span><span id="line-1321"> initMobCleaner();</span>
<span class="source-line-no">1322</span><span id="line-1322"></span>
<span class="source-line-no">1323</span><span id="line-1323"> // delete the stale data for replication sync up tool if necessary</span>
<span class="source-line-no">1324</span><span id="line-1324"> status.setStatus("Cleanup ReplicationSyncUp status if necessary");</span>
<span class="source-line-no">1325</span><span id="line-1325"> Path replicationSyncUpInfoFile =</span>
<span class="source-line-no">1326</span><span id="line-1326"> new Path(new Path(dataRootDir, ReplicationSyncUp.INFO_DIR), ReplicationSyncUp.INFO_FILE);</span>
<span class="source-line-no">1327</span><span id="line-1327"> if (dataFs.exists(replicationSyncUpInfoFile)) {</span>
<span class="source-line-no">1328</span><span id="line-1328"> // info file is available, load the timestamp and use it to clean up stale data in replication</span>
<span class="source-line-no">1329</span><span id="line-1329"> // queue storage.</span>
<span class="source-line-no">1330</span><span id="line-1330"> byte[] data;</span>
<span class="source-line-no">1331</span><span id="line-1331"> try (FSDataInputStream in = dataFs.open(replicationSyncUpInfoFile)) {</span>
<span class="source-line-no">1332</span><span id="line-1332"> data = ByteStreams.toByteArray(in);</span>
<span class="source-line-no">1333</span><span id="line-1333"> }</span>
<span class="source-line-no">1334</span><span id="line-1334"> ReplicationSyncUpToolInfo info = null;</span>
<span class="source-line-no">1335</span><span id="line-1335"> try {</span>
<span class="source-line-no">1336</span><span id="line-1336"> info = JsonMapper.fromJson(Bytes.toString(data), ReplicationSyncUpToolInfo.class);</span>
<span class="source-line-no">1337</span><span id="line-1337"> } catch (JsonParseException e) {</span>
<span class="source-line-no">1338</span><span id="line-1338"> // usually this should be a partial file, which means the ReplicationSyncUp tool did not</span>
<span class="source-line-no">1339</span><span id="line-1339"> // finish properly, so not a problem. Here we do not clean up the status as we do not know</span>
<span class="source-line-no">1340</span><span id="line-1340"> // the reason why the tool did not finish properly, so let users clean the status up</span>
<span class="source-line-no">1341</span><span id="line-1341"> // manually</span>
<span class="source-line-no">1342</span><span id="line-1342"> LOG.warn("failed to parse replication sync up info file, ignore and continue...", e);</span>
<span class="source-line-no">1343</span><span id="line-1343"> }</span>
<span class="source-line-no">1344</span><span id="line-1344"> if (info != null) {</span>
<span class="source-line-no">1345</span><span id="line-1345"> LOG.info("Remove last sequence ids and hfile references which are written before {}({})",</span>
<span class="source-line-no">1346</span><span id="line-1346"> info.getStartTimeMs(), DateTimeFormatter.ISO_DATE_TIME.withZone(ZoneId.systemDefault())</span>
<span class="source-line-no">1347</span><span id="line-1347"> .format(Instant.ofEpochMilli(info.getStartTimeMs())));</span>
<span class="source-line-no">1348</span><span id="line-1348"> replicationPeerManager.getQueueStorage()</span>
<span class="source-line-no">1349</span><span id="line-1349"> .removeLastSequenceIdsAndHFileRefsBefore(info.getStartTimeMs());</span>
<span class="source-line-no">1350</span><span id="line-1350"> // delete the file after removing the stale data, so next time we do not need to do this</span>
<span class="source-line-no">1351</span><span id="line-1351"> // again.</span>
<span class="source-line-no">1352</span><span id="line-1352"> dataFs.delete(replicationSyncUpInfoFile, false);</span>
<span class="source-line-no">1353</span><span id="line-1353"> }</span>
<span class="source-line-no">1354</span><span id="line-1354"> }</span>
<span class="source-line-no">1355</span><span id="line-1355"> status.setStatus("Calling postStartMaster coprocessors");</span>
<span class="source-line-no">1356</span><span id="line-1356"> if (this.cpHost != null) {</span>
<span class="source-line-no">1357</span><span id="line-1357"> // don't let cp initialization errors kill the master</span>
<span class="source-line-no">1358</span><span id="line-1358"> try {</span>
<span class="source-line-no">1359</span><span id="line-1359"> this.cpHost.postStartMaster();</span>
<span class="source-line-no">1360</span><span id="line-1360"> } catch (IOException ioe) {</span>
<span class="source-line-no">1361</span><span id="line-1361"> LOG.error("Coprocessor postStartMaster() hook failed", ioe);</span>
<span class="source-line-no">1362</span><span id="line-1362"> }</span>
<span class="source-line-no">1363</span><span id="line-1363"> }</span>
<span class="source-line-no">1364</span><span id="line-1364"></span>
<span class="source-line-no">1365</span><span id="line-1365"> zombieDetector.interrupt();</span>
<span class="source-line-no">1366</span><span id="line-1366"></span>
<span class="source-line-no">1367</span><span id="line-1367"> /*</span>
<span class="source-line-no">1368</span><span id="line-1368"> * After master has started up, lets do balancer post startup initialization. Since this runs in</span>
<span class="source-line-no">1369</span><span id="line-1369"> * activeMasterManager thread, it should be fine.</span>
<span class="source-line-no">1370</span><span id="line-1370"> */</span>
<span class="source-line-no">1371</span><span id="line-1371"> long start = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">1372</span><span id="line-1372"> this.balancer.postMasterStartupInitialize();</span>
<span class="source-line-no">1373</span><span id="line-1373"> if (LOG.isDebugEnabled()) {</span>
<span class="source-line-no">1374</span><span id="line-1374"> LOG.debug("Balancer post startup initialization complete, took "</span>
<span class="source-line-no">1375</span><span id="line-1375"> + ((EnvironmentEdgeManager.currentTime() - start) / 1000) + " seconds");</span>
<span class="source-line-no">1376</span><span id="line-1376"> }</span>
<span class="source-line-no">1377</span><span id="line-1377"></span>
<span class="source-line-no">1378</span><span id="line-1378"> this.rollingUpgradeChore = new RollingUpgradeChore(this);</span>
<span class="source-line-no">1379</span><span id="line-1379"> getChoreService().scheduleChore(rollingUpgradeChore);</span>
<span class="source-line-no">1380</span><span id="line-1380"></span>
<span class="source-line-no">1381</span><span id="line-1381"> this.oldWALsDirSizeChore = new OldWALsDirSizeChore(this);</span>
<span class="source-line-no">1382</span><span id="line-1382"> getChoreService().scheduleChore(this.oldWALsDirSizeChore);</span>
<span class="source-line-no">1383</span><span id="line-1383"></span>
<span class="source-line-no">1384</span><span id="line-1384"> status.markComplete("Progress after master initialized complete");</span>
<span class="source-line-no">1385</span><span id="line-1385"> }</span>
<span class="source-line-no">1386</span><span id="line-1386"></span>
<span class="source-line-no">1387</span><span id="line-1387"> /**</span>
<span class="source-line-no">1388</span><span id="line-1388"> * Used for testing only to set Mock objects.</span>
<span class="source-line-no">1389</span><span id="line-1389"> * @param hbckChore hbckChore</span>
<span class="source-line-no">1390</span><span id="line-1390"> */</span>
<span class="source-line-no">1391</span><span id="line-1391"> public void setHbckChoreForTesting(HbckChore hbckChore) {</span>
<span class="source-line-no">1392</span><span id="line-1392"> this.hbckChore = hbckChore;</span>
<span class="source-line-no">1393</span><span id="line-1393"> }</span>
<span class="source-line-no">1394</span><span id="line-1394"></span>
<span class="source-line-no">1395</span><span id="line-1395"> /**</span>
<span class="source-line-no">1396</span><span id="line-1396"> * Used for testing only to set Mock objects.</span>
<span class="source-line-no">1397</span><span id="line-1397"> * @param catalogJanitorChore catalogJanitorChore</span>
<span class="source-line-no">1398</span><span id="line-1398"> */</span>
<span class="source-line-no">1399</span><span id="line-1399"> public void setCatalogJanitorChoreForTesting(CatalogJanitor catalogJanitorChore) {</span>
<span class="source-line-no">1400</span><span id="line-1400"> this.catalogJanitorChore = catalogJanitorChore;</span>
<span class="source-line-no">1401</span><span id="line-1401"> }</span>
<span class="source-line-no">1402</span><span id="line-1402"></span>
<span class="source-line-no">1403</span><span id="line-1403"> private void createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor)</span>
<span class="source-line-no">1404</span><span id="line-1404"> throws IOException {</span>
<span class="source-line-no">1405</span><span id="line-1405"> TableDescriptor newMetaDesc = TableDescriptorBuilder.newBuilder(metaDescriptor)</span>
<span class="source-line-no">1406</span><span id="line-1406"> .setColumnFamily(FSTableDescriptors.getTableFamilyDescForMeta(conf))</span>
<span class="source-line-no">1407</span><span id="line-1407"> .setColumnFamily(FSTableDescriptors.getReplBarrierFamilyDescForMeta()).build();</span>
<span class="source-line-no">1408</span><span id="line-1408"> long pid = this.modifyTable(TableName.META_TABLE_NAME, () -&gt; newMetaDesc, 0, 0, false);</span>
<span class="source-line-no">1409</span><span id="line-1409"> int tries = 30;</span>
<span class="source-line-no">1410</span><span id="line-1410"> while (</span>
<span class="source-line-no">1411</span><span id="line-1411"> !(getMasterProcedureExecutor().isFinished(pid)) &amp;&amp; getMasterProcedureExecutor().isRunning()</span>
<span class="source-line-no">1412</span><span id="line-1412"> &amp;&amp; tries &gt; 0</span>
<span class="source-line-no">1413</span><span id="line-1413"> ) {</span>
<span class="source-line-no">1414</span><span id="line-1414"> try {</span>
<span class="source-line-no">1415</span><span id="line-1415"> Thread.sleep(1000);</span>
<span class="source-line-no">1416</span><span id="line-1416"> } catch (InterruptedException e) {</span>
<span class="source-line-no">1417</span><span id="line-1417"> throw new IOException("Wait interrupted", e);</span>
<span class="source-line-no">1418</span><span id="line-1418"> }</span>
<span class="source-line-no">1419</span><span id="line-1419"> tries--;</span>
<span class="source-line-no">1420</span><span id="line-1420"> }</span>
<span class="source-line-no">1421</span><span id="line-1421"> if (tries &lt;= 0) {</span>
<span class="source-line-no">1422</span><span id="line-1422"> throw new HBaseIOException(</span>
<span class="source-line-no">1423</span><span id="line-1423"> "Failed to add table and rep_barrier CFs to meta in a given time.");</span>
<span class="source-line-no">1424</span><span id="line-1424"> } else {</span>
<span class="source-line-no">1425</span><span id="line-1425"> Procedure&lt;?&gt; result = getMasterProcedureExecutor().getResult(pid);</span>
<span class="source-line-no">1426</span><span id="line-1426"> if (result != null &amp;&amp; result.isFailed()) {</span>
<span class="source-line-no">1427</span><span id="line-1427"> throw new IOException("Failed to add table and rep_barrier CFs to meta. "</span>
<span class="source-line-no">1428</span><span id="line-1428"> + MasterProcedureUtil.unwrapRemoteIOException(result));</span>
<span class="source-line-no">1429</span><span id="line-1429"> }</span>
<span class="source-line-no">1430</span><span id="line-1430"> }</span>
<span class="source-line-no">1431</span><span id="line-1431"> }</span>
<span class="source-line-no">1432</span><span id="line-1432"></span>
<span class="source-line-no">1433</span><span id="line-1433"> /**</span>
<span class="source-line-no">1434</span><span id="line-1434"> * Check hbase:meta is up and ready for reading. For use during Master startup only.</span>
<span class="source-line-no">1435</span><span id="line-1435"> * @return True if meta is UP and online and startup can progress. Otherwise, meta is not online</span>
<span class="source-line-no">1436</span><span id="line-1436"> * and we will hold here until operator intervention.</span>
<span class="source-line-no">1437</span><span id="line-1437"> */</span>
<span class="source-line-no">1438</span><span id="line-1438"> @InterfaceAudience.Private</span>
<span class="source-line-no">1439</span><span id="line-1439"> public boolean waitForMetaOnline() {</span>
<span class="source-line-no">1440</span><span id="line-1440"> return isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO);</span>
<span class="source-line-no">1441</span><span id="line-1441"> }</span>
<span class="source-line-no">1442</span><span id="line-1442"></span>
<span class="source-line-no">1443</span><span id="line-1443"> /**</span>
<span class="source-line-no">1444</span><span id="line-1444"> * @return True if region is online and scannable else false if an error or shutdown (Otherwise we</span>
<span class="source-line-no">1445</span><span id="line-1445"> * just block in here holding up all forward-progess).</span>
<span class="source-line-no">1446</span><span id="line-1446"> */</span>
<span class="source-line-no">1447</span><span id="line-1447"> private boolean isRegionOnline(RegionInfo ri) {</span>
<span class="source-line-no">1448</span><span id="line-1448"> RetryCounter rc = null;</span>
<span class="source-line-no">1449</span><span id="line-1449"> while (!isStopped()) {</span>
<span class="source-line-no">1450</span><span id="line-1450"> RegionState rs = this.assignmentManager.getRegionStates().getRegionState(ri);</span>
<span class="source-line-no">1451</span><span id="line-1451"> if (rs != null &amp;&amp; rs.isOpened()) {</span>
<span class="source-line-no">1452</span><span id="line-1452"> if (this.getServerManager().isServerOnline(rs.getServerName())) {</span>
<span class="source-line-no">1453</span><span id="line-1453"> return true;</span>
<span class="source-line-no">1454</span><span id="line-1454"> }</span>
<span class="source-line-no">1455</span><span id="line-1455"> }</span>
<span class="source-line-no">1456</span><span id="line-1456"> // Region is not OPEN.</span>
<span class="source-line-no">1457</span><span id="line-1457"> Optional&lt;Procedure&lt;MasterProcedureEnv&gt;&gt; optProc = this.procedureExecutor.getProcedures()</span>
<span class="source-line-no">1458</span><span id="line-1458"> .stream().filter(p -&gt; p instanceof ServerCrashProcedure).findAny();</span>
<span class="source-line-no">1459</span><span id="line-1459"> // TODO: Add a page to refguide on how to do repair. Have this log message point to it.</span>
<span class="source-line-no">1460</span><span id="line-1460"> // Page will talk about loss of edits, how to schedule at least the meta WAL recovery, and</span>
<span class="source-line-no">1461</span><span id="line-1461"> // then how to assign including how to break region lock if one held.</span>
<span class="source-line-no">1462</span><span id="line-1462"> LOG.warn(</span>
<span class="source-line-no">1463</span><span id="line-1463"> "{} is NOT online; state={}; ServerCrashProcedures={}. Master startup cannot "</span>
<span class="source-line-no">1464</span><span id="line-1464"> + "progress, in holding-pattern until region onlined.",</span>
<span class="source-line-no">1465</span><span id="line-1465"> ri.getRegionNameAsString(), rs, optProc.isPresent());</span>
<span class="source-line-no">1466</span><span id="line-1466"> // Check once-a-minute.</span>
<span class="source-line-no">1467</span><span id="line-1467"> if (rc == null) {</span>
<span class="source-line-no">1468</span><span id="line-1468"> rc = new RetryCounterFactory(Integer.MAX_VALUE, 1000, 60_000).create();</span>
<span class="source-line-no">1469</span><span id="line-1469"> }</span>
<span class="source-line-no">1470</span><span id="line-1470"> Threads.sleep(rc.getBackoffTimeAndIncrementAttempts());</span>
<span class="source-line-no">1471</span><span id="line-1471"> }</span>
<span class="source-line-no">1472</span><span id="line-1472"> return false;</span>
<span class="source-line-no">1473</span><span id="line-1473"> }</span>
<span class="source-line-no">1474</span><span id="line-1474"></span>
<span class="source-line-no">1475</span><span id="line-1475"> /**</span>
<span class="source-line-no">1476</span><span id="line-1476"> * Check hbase:namespace table is assigned. If not, startup will hang looking for the ns table</span>
<span class="source-line-no">1477</span><span id="line-1477"> * &lt;p/&gt;</span>
<span class="source-line-no">1478</span><span id="line-1478"> * This is for rolling upgrading, later we will migrate the data in ns table to the ns family of</span>
<span class="source-line-no">1479</span><span id="line-1479"> * meta table. And if this is a new cluster, this method will return immediately as there will be</span>
<span class="source-line-no">1480</span><span id="line-1480"> * no namespace table/region.</span>
<span class="source-line-no">1481</span><span id="line-1481"> * @return True if namespace table is up/online.</span>
<span class="source-line-no">1482</span><span id="line-1482"> */</span>
<span class="source-line-no">1483</span><span id="line-1483"> private boolean waitForNamespaceOnline() throws IOException {</span>
<span class="source-line-no">1484</span><span id="line-1484"> TableState nsTableState =</span>
<span class="source-line-no">1485</span><span id="line-1485"> MetaTableAccessor.getTableState(getConnection(), TableName.NAMESPACE_TABLE_NAME);</span>
<span class="source-line-no">1486</span><span id="line-1486"> if (nsTableState == null || nsTableState.isDisabled()) {</span>
<span class="source-line-no">1487</span><span id="line-1487"> // this means we have already migrated the data and disabled or deleted the namespace table,</span>
<span class="source-line-no">1488</span><span id="line-1488"> // or this is a new deploy which does not have a namespace table from the beginning.</span>
<span class="source-line-no">1489</span><span id="line-1489"> return true;</span>
<span class="source-line-no">1490</span><span id="line-1490"> }</span>
<span class="source-line-no">1491</span><span id="line-1491"> List&lt;RegionInfo&gt; ris =</span>
<span class="source-line-no">1492</span><span id="line-1492"> this.assignmentManager.getRegionStates().getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME);</span>
<span class="source-line-no">1493</span><span id="line-1493"> if (ris.isEmpty()) {</span>
<span class="source-line-no">1494</span><span id="line-1494"> // maybe this will not happen any more, but anyway, no harm to add a check here...</span>
<span class="source-line-no">1495</span><span id="line-1495"> return true;</span>
<span class="source-line-no">1496</span><span id="line-1496"> }</span>
<span class="source-line-no">1497</span><span id="line-1497"> // Else there are namespace regions up in meta. Ensure they are assigned before we go on.</span>
<span class="source-line-no">1498</span><span id="line-1498"> for (RegionInfo ri : ris) {</span>
<span class="source-line-no">1499</span><span id="line-1499"> if (!isRegionOnline(ri)) {</span>
<span class="source-line-no">1500</span><span id="line-1500"> return false;</span>
<span class="source-line-no">1501</span><span id="line-1501"> }</span>
<span class="source-line-no">1502</span><span id="line-1502"> }</span>
<span class="source-line-no">1503</span><span id="line-1503"> return true;</span>
<span class="source-line-no">1504</span><span id="line-1504"> }</span>
<span class="source-line-no">1505</span><span id="line-1505"></span>
<span class="source-line-no">1506</span><span id="line-1506"> /**</span>
<span class="source-line-no">1507</span><span id="line-1507"> * Adds the {@code MasterQuotasObserver} to the list of configured Master observers to</span>
<span class="source-line-no">1508</span><span id="line-1508"> * automatically remove quotas for a table when that table is deleted.</span>
<span class="source-line-no">1509</span><span id="line-1509"> */</span>
<span class="source-line-no">1510</span><span id="line-1510"> @InterfaceAudience.Private</span>
<span class="source-line-no">1511</span><span id="line-1511"> public void updateConfigurationForQuotasObserver(Configuration conf) {</span>
<span class="source-line-no">1512</span><span id="line-1512"> // We're configured to not delete quotas on table deletion, so we don't need to add the obs.</span>
<span class="source-line-no">1513</span><span id="line-1513"> if (</span>
<span class="source-line-no">1514</span><span id="line-1514"> !conf.getBoolean(MasterQuotasObserver.REMOVE_QUOTA_ON_TABLE_DELETE,</span>
<span class="source-line-no">1515</span><span id="line-1515"> MasterQuotasObserver.REMOVE_QUOTA_ON_TABLE_DELETE_DEFAULT)</span>
<span class="source-line-no">1516</span><span id="line-1516"> ) {</span>
<span class="source-line-no">1517</span><span id="line-1517"> return;</span>
<span class="source-line-no">1518</span><span id="line-1518"> }</span>
<span class="source-line-no">1519</span><span id="line-1519"> String[] masterCoprocs = conf.getStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY);</span>
<span class="source-line-no">1520</span><span id="line-1520"> final int length = null == masterCoprocs ? 0 : masterCoprocs.length;</span>
<span class="source-line-no">1521</span><span id="line-1521"> String[] updatedCoprocs = new String[length + 1];</span>
<span class="source-line-no">1522</span><span id="line-1522"> if (length &gt; 0) {</span>
<span class="source-line-no">1523</span><span id="line-1523"> System.arraycopy(masterCoprocs, 0, updatedCoprocs, 0, masterCoprocs.length);</span>
<span class="source-line-no">1524</span><span id="line-1524"> }</span>
<span class="source-line-no">1525</span><span id="line-1525"> updatedCoprocs[length] = MasterQuotasObserver.class.getName();</span>
<span class="source-line-no">1526</span><span id="line-1526"> conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, updatedCoprocs);</span>
<span class="source-line-no">1527</span><span id="line-1527"> }</span>
<span class="source-line-no">1528</span><span id="line-1528"></span>
<span class="source-line-no">1529</span><span id="line-1529"> private void initMobCleaner() {</span>
<span class="source-line-no">1530</span><span id="line-1530"> this.mobFileCleanerChore = new MobFileCleanerChore(this);</span>
<span class="source-line-no">1531</span><span id="line-1531"> getChoreService().scheduleChore(mobFileCleanerChore);</span>
<span class="source-line-no">1532</span><span id="line-1532"> this.mobFileCompactionChore = new MobFileCompactionChore(this);</span>
<span class="source-line-no">1533</span><span id="line-1533"> getChoreService().scheduleChore(mobFileCompactionChore);</span>
<span class="source-line-no">1534</span><span id="line-1534"> }</span>
<span class="source-line-no">1535</span><span id="line-1535"></span>
<span class="source-line-no">1536</span><span id="line-1536"> /**</span>
<span class="source-line-no">1537</span><span id="line-1537"> * &lt;p&gt;</span>
<span class="source-line-no">1538</span><span id="line-1538"> * Create a {@link ServerManager} instance.</span>
<span class="source-line-no">1539</span><span id="line-1539"> * &lt;/p&gt;</span>
<span class="source-line-no">1540</span><span id="line-1540"> * &lt;p&gt;</span>
<span class="source-line-no">1541</span><span id="line-1541"> * Will be overridden in tests.</span>
<span class="source-line-no">1542</span><span id="line-1542"> * &lt;/p&gt;</span>
<span class="source-line-no">1543</span><span id="line-1543"> */</span>
<span class="source-line-no">1544</span><span id="line-1544"> @InterfaceAudience.Private</span>
<span class="source-line-no">1545</span><span id="line-1545"> protected ServerManager createServerManager(MasterServices master, RegionServerList storage)</span>
<span class="source-line-no">1546</span><span id="line-1546"> throws IOException {</span>
<span class="source-line-no">1547</span><span id="line-1547"> // We put this out here in a method so can do a Mockito.spy and stub it out</span>
<span class="source-line-no">1548</span><span id="line-1548"> // w/ a mocked up ServerManager.</span>
<span class="source-line-no">1549</span><span id="line-1549"> setupClusterConnection();</span>
<span class="source-line-no">1550</span><span id="line-1550"> return new ServerManager(master, storage);</span>
<span class="source-line-no">1551</span><span id="line-1551"> }</span>
<span class="source-line-no">1552</span><span id="line-1552"></span>
<span class="source-line-no">1553</span><span id="line-1553"> private void waitForRegionServers(final MonitoredTask status)</span>
<span class="source-line-no">1554</span><span id="line-1554"> throws IOException, InterruptedException {</span>
<span class="source-line-no">1555</span><span id="line-1555"> this.serverManager.waitForRegionServers(status);</span>
<span class="source-line-no">1556</span><span id="line-1556"> }</span>
<span class="source-line-no">1557</span><span id="line-1557"></span>
<span class="source-line-no">1558</span><span id="line-1558"> // Will be overridden in tests</span>
<span class="source-line-no">1559</span><span id="line-1559"> @InterfaceAudience.Private</span>
<span class="source-line-no">1560</span><span id="line-1560"> protected void initClusterSchemaService() throws IOException, InterruptedException {</span>
<span class="source-line-no">1561</span><span id="line-1561"> this.clusterSchemaService = new ClusterSchemaServiceImpl(this);</span>
<span class="source-line-no">1562</span><span id="line-1562"> this.clusterSchemaService.startAsync();</span>
<span class="source-line-no">1563</span><span id="line-1563"> try {</span>
<span class="source-line-no">1564</span><span id="line-1564"> this.clusterSchemaService</span>
<span class="source-line-no">1565</span><span id="line-1565"> .awaitRunning(getConfiguration().getInt(HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS,</span>
<span class="source-line-no">1566</span><span id="line-1566"> DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS), TimeUnit.SECONDS);</span>
<span class="source-line-no">1567</span><span id="line-1567"> } catch (TimeoutException toe) {</span>
<span class="source-line-no">1568</span><span id="line-1568"> throw new IOException("Timedout starting ClusterSchemaService", toe);</span>
<span class="source-line-no">1569</span><span id="line-1569"> }</span>
<span class="source-line-no">1570</span><span id="line-1570"> }</span>
<span class="source-line-no">1571</span><span id="line-1571"></span>
<span class="source-line-no">1572</span><span id="line-1572"> private void initQuotaManager() throws IOException {</span>
<span class="source-line-no">1573</span><span id="line-1573"> MasterQuotaManager quotaManager = new MasterQuotaManager(this);</span>
<span class="source-line-no">1574</span><span id="line-1574"> quotaManager.start();</span>
<span class="source-line-no">1575</span><span id="line-1575"> this.quotaManager = quotaManager;</span>
<span class="source-line-no">1576</span><span id="line-1576"> }</span>
<span class="source-line-no">1577</span><span id="line-1577"></span>
<span class="source-line-no">1578</span><span id="line-1578"> private SpaceQuotaSnapshotNotifier createQuotaSnapshotNotifier() {</span>
<span class="source-line-no">1579</span><span id="line-1579"> SpaceQuotaSnapshotNotifier notifier =</span>
<span class="source-line-no">1580</span><span id="line-1580"> SpaceQuotaSnapshotNotifierFactory.getInstance().create(getConfiguration());</span>
<span class="source-line-no">1581</span><span id="line-1581"> return notifier;</span>
<span class="source-line-no">1582</span><span id="line-1582"> }</span>
<span class="source-line-no">1583</span><span id="line-1583"></span>
<span class="source-line-no">1584</span><span id="line-1584"> public boolean isCatalogJanitorEnabled() {</span>
<span class="source-line-no">1585</span><span id="line-1585"> return catalogJanitorChore != null ? catalogJanitorChore.getEnabled() : false;</span>
<span class="source-line-no">1586</span><span id="line-1586"> }</span>
<span class="source-line-no">1587</span><span id="line-1587"></span>
<span class="source-line-no">1588</span><span id="line-1588"> boolean isCleanerChoreEnabled() {</span>
<span class="source-line-no">1589</span><span id="line-1589"> boolean hfileCleanerFlag = true, logCleanerFlag = true;</span>
<span class="source-line-no">1590</span><span id="line-1590"></span>
<span class="source-line-no">1591</span><span id="line-1591"> if (getHFileCleaner() != null) {</span>
<span class="source-line-no">1592</span><span id="line-1592"> hfileCleanerFlag = getHFileCleaner().getEnabled();</span>
<span class="source-line-no">1593</span><span id="line-1593"> }</span>
<span class="source-line-no">1594</span><span id="line-1594"></span>
<span class="source-line-no">1595</span><span id="line-1595"> if (logCleaner != null) {</span>
<span class="source-line-no">1596</span><span id="line-1596"> logCleanerFlag = logCleaner.getEnabled();</span>
<span class="source-line-no">1597</span><span id="line-1597"> }</span>
<span class="source-line-no">1598</span><span id="line-1598"></span>
<span class="source-line-no">1599</span><span id="line-1599"> return (hfileCleanerFlag &amp;&amp; logCleanerFlag);</span>
<span class="source-line-no">1600</span><span id="line-1600"> }</span>
<span class="source-line-no">1601</span><span id="line-1601"></span>
<span class="source-line-no">1602</span><span id="line-1602"> @Override</span>
<span class="source-line-no">1603</span><span id="line-1603"> public ServerManager getServerManager() {</span>
<span class="source-line-no">1604</span><span id="line-1604"> return this.serverManager;</span>
<span class="source-line-no">1605</span><span id="line-1605"> }</span>
<span class="source-line-no">1606</span><span id="line-1606"></span>
<span class="source-line-no">1607</span><span id="line-1607"> @Override</span>
<span class="source-line-no">1608</span><span id="line-1608"> public MasterFileSystem getMasterFileSystem() {</span>
<span class="source-line-no">1609</span><span id="line-1609"> return this.fileSystemManager;</span>
<span class="source-line-no">1610</span><span id="line-1610"> }</span>
<span class="source-line-no">1611</span><span id="line-1611"></span>
<span class="source-line-no">1612</span><span id="line-1612"> @Override</span>
<span class="source-line-no">1613</span><span id="line-1613"> public MasterWalManager getMasterWalManager() {</span>
<span class="source-line-no">1614</span><span id="line-1614"> return this.walManager;</span>
<span class="source-line-no">1615</span><span id="line-1615"> }</span>
<span class="source-line-no">1616</span><span id="line-1616"></span>
<span class="source-line-no">1617</span><span id="line-1617"> @Override</span>
<span class="source-line-no">1618</span><span id="line-1618"> public SplitWALManager getSplitWALManager() {</span>
<span class="source-line-no">1619</span><span id="line-1619"> return splitWALManager;</span>
<span class="source-line-no">1620</span><span id="line-1620"> }</span>
<span class="source-line-no">1621</span><span id="line-1621"></span>
<span class="source-line-no">1622</span><span id="line-1622"> @Override</span>
<span class="source-line-no">1623</span><span id="line-1623"> public TableStateManager getTableStateManager() {</span>
<span class="source-line-no">1624</span><span id="line-1624"> return tableStateManager;</span>
<span class="source-line-no">1625</span><span id="line-1625"> }</span>
<span class="source-line-no">1626</span><span id="line-1626"></span>
<span class="source-line-no">1627</span><span id="line-1627"> /*</span>
<span class="source-line-no">1628</span><span id="line-1628"> * Start up all services. If any of these threads gets an unhandled exception then they just die</span>
<span class="source-line-no">1629</span><span id="line-1629"> * with a logged message. This should be fine because in general, we do not expect the master to</span>
<span class="source-line-no">1630</span><span id="line-1630"> * get such unhandled exceptions as OOMEs; it should be lightly loaded. See what HRegionServer</span>
<span class="source-line-no">1631</span><span id="line-1631"> * does if need to install an unexpected exception handler.</span>
<span class="source-line-no">1632</span><span id="line-1632"> */</span>
<span class="source-line-no">1633</span><span id="line-1633"> private void startServiceThreads() throws IOException {</span>
<span class="source-line-no">1634</span><span id="line-1634"> // Start the executor service pools</span>
<span class="source-line-no">1635</span><span id="line-1635"> final int masterOpenRegionPoolSize = conf.getInt(HConstants.MASTER_OPEN_REGION_THREADS,</span>
<span class="source-line-no">1636</span><span id="line-1636"> HConstants.MASTER_OPEN_REGION_THREADS_DEFAULT);</span>
<span class="source-line-no">1637</span><span id="line-1637"> executorService.startExecutorService(executorService.new ExecutorConfig()</span>
<span class="source-line-no">1638</span><span id="line-1638"> .setExecutorType(ExecutorType.MASTER_OPEN_REGION).setCorePoolSize(masterOpenRegionPoolSize));</span>
<span class="source-line-no">1639</span><span id="line-1639"> final int masterCloseRegionPoolSize = conf.getInt(HConstants.MASTER_CLOSE_REGION_THREADS,</span>
<span class="source-line-no">1640</span><span id="line-1640"> HConstants.MASTER_CLOSE_REGION_THREADS_DEFAULT);</span>
<span class="source-line-no">1641</span><span id="line-1641"> executorService.startExecutorService(</span>
<span class="source-line-no">1642</span><span id="line-1642"> executorService.new ExecutorConfig().setExecutorType(ExecutorType.MASTER_CLOSE_REGION)</span>
<span class="source-line-no">1643</span><span id="line-1643"> .setCorePoolSize(masterCloseRegionPoolSize));</span>
<span class="source-line-no">1644</span><span id="line-1644"> final int masterServerOpThreads = conf.getInt(HConstants.MASTER_SERVER_OPERATIONS_THREADS,</span>
<span class="source-line-no">1645</span><span id="line-1645"> HConstants.MASTER_SERVER_OPERATIONS_THREADS_DEFAULT);</span>
<span class="source-line-no">1646</span><span id="line-1646"> executorService.startExecutorService(</span>
<span class="source-line-no">1647</span><span id="line-1647"> executorService.new ExecutorConfig().setExecutorType(ExecutorType.MASTER_SERVER_OPERATIONS)</span>
<span class="source-line-no">1648</span><span id="line-1648"> .setCorePoolSize(masterServerOpThreads));</span>
<span class="source-line-no">1649</span><span id="line-1649"> final int masterServerMetaOpsThreads =</span>
<span class="source-line-no">1650</span><span id="line-1650"> conf.getInt(HConstants.MASTER_META_SERVER_OPERATIONS_THREADS,</span>
<span class="source-line-no">1651</span><span id="line-1651"> HConstants.MASTER_META_SERVER_OPERATIONS_THREADS_DEFAULT);</span>
<span class="source-line-no">1652</span><span id="line-1652"> executorService.startExecutorService(executorService.new ExecutorConfig()</span>
<span class="source-line-no">1653</span><span id="line-1653"> .setExecutorType(ExecutorType.MASTER_META_SERVER_OPERATIONS)</span>
<span class="source-line-no">1654</span><span id="line-1654"> .setCorePoolSize(masterServerMetaOpsThreads));</span>
<span class="source-line-no">1655</span><span id="line-1655"> final int masterLogReplayThreads = conf.getInt(HConstants.MASTER_LOG_REPLAY_OPS_THREADS,</span>
<span class="source-line-no">1656</span><span id="line-1656"> HConstants.MASTER_LOG_REPLAY_OPS_THREADS_DEFAULT);</span>
<span class="source-line-no">1657</span><span id="line-1657"> executorService.startExecutorService(executorService.new ExecutorConfig()</span>
<span class="source-line-no">1658</span><span id="line-1658"> .setExecutorType(ExecutorType.M_LOG_REPLAY_OPS).setCorePoolSize(masterLogReplayThreads));</span>
<span class="source-line-no">1659</span><span id="line-1659"> final int masterSnapshotThreads = conf.getInt(SnapshotManager.SNAPSHOT_POOL_THREADS_KEY,</span>
<span class="source-line-no">1660</span><span id="line-1660"> SnapshotManager.SNAPSHOT_POOL_THREADS_DEFAULT);</span>
<span class="source-line-no">1661</span><span id="line-1661"> executorService.startExecutorService(</span>
<span class="source-line-no">1662</span><span id="line-1662"> executorService.new ExecutorConfig().setExecutorType(ExecutorType.MASTER_SNAPSHOT_OPERATIONS)</span>
<span class="source-line-no">1663</span><span id="line-1663"> .setCorePoolSize(masterSnapshotThreads).setAllowCoreThreadTimeout(true));</span>
<span class="source-line-no">1664</span><span id="line-1664"> final int masterMergeDispatchThreads = conf.getInt(HConstants.MASTER_MERGE_DISPATCH_THREADS,</span>
<span class="source-line-no">1665</span><span id="line-1665"> HConstants.MASTER_MERGE_DISPATCH_THREADS_DEFAULT);</span>
<span class="source-line-no">1666</span><span id="line-1666"> executorService.startExecutorService(</span>
<span class="source-line-no">1667</span><span id="line-1667"> executorService.new ExecutorConfig().setExecutorType(ExecutorType.MASTER_MERGE_OPERATIONS)</span>
<span class="source-line-no">1668</span><span id="line-1668"> .setCorePoolSize(masterMergeDispatchThreads).setAllowCoreThreadTimeout(true));</span>
<span class="source-line-no">1669</span><span id="line-1669"></span>
<span class="source-line-no">1670</span><span id="line-1670"> // We depend on there being only one instance of this executor running</span>
<span class="source-line-no">1671</span><span id="line-1671"> // at a time. To do concurrency, would need fencing of enable/disable of</span>
<span class="source-line-no">1672</span><span id="line-1672"> // tables.</span>
<span class="source-line-no">1673</span><span id="line-1673"> // Any time changing this maxThreads to &gt; 1, pls see the comment at</span>
<span class="source-line-no">1674</span><span id="line-1674"> // AccessController#postCompletedCreateTableAction</span>
<span class="source-line-no">1675</span><span id="line-1675"> executorService.startExecutorService(executorService.new ExecutorConfig()</span>
<span class="source-line-no">1676</span><span id="line-1676"> .setExecutorType(ExecutorType.MASTER_TABLE_OPERATIONS).setCorePoolSize(1));</span>
<span class="source-line-no">1677</span><span id="line-1677"> startProcedureExecutor();</span>
<span class="source-line-no">1678</span><span id="line-1678"></span>
<span class="source-line-no">1679</span><span id="line-1679"> // Create log cleaner thread pool</span>
<span class="source-line-no">1680</span><span id="line-1680"> logCleanerPool = DirScanPool.getLogCleanerScanPool(conf);</span>
<span class="source-line-no">1681</span><span id="line-1681"> Map&lt;String, Object&gt; params = new HashMap&lt;&gt;();</span>
<span class="source-line-no">1682</span><span id="line-1682"> params.put(MASTER, this);</span>
<span class="source-line-no">1683</span><span id="line-1683"> // Start log cleaner thread</span>
<span class="source-line-no">1684</span><span id="line-1684"> int cleanerInterval =</span>
<span class="source-line-no">1685</span><span id="line-1685"> conf.getInt(HBASE_MASTER_CLEANER_INTERVAL, DEFAULT_HBASE_MASTER_CLEANER_INTERVAL);</span>
<span class="source-line-no">1686</span><span id="line-1686"> this.logCleaner =</span>
<span class="source-line-no">1687</span><span id="line-1687"> new LogCleaner(cleanerInterval, this, conf, getMasterWalManager().getFileSystem(),</span>
<span class="source-line-no">1688</span><span id="line-1688"> getMasterWalManager().getOldLogDir(), logCleanerPool, params);</span>
<span class="source-line-no">1689</span><span id="line-1689"> getChoreService().scheduleChore(logCleaner);</span>
<span class="source-line-no">1690</span><span id="line-1690"></span>
<span class="source-line-no">1691</span><span id="line-1691"> Path archiveDir = HFileArchiveUtil.getArchivePath(conf);</span>
<span class="source-line-no">1692</span><span id="line-1692"></span>
<span class="source-line-no">1693</span><span id="line-1693"> // Create custom archive hfile cleaners</span>
<span class="source-line-no">1694</span><span id="line-1694"> String[] paths = conf.getStrings(HFileCleaner.HFILE_CLEANER_CUSTOM_PATHS);</span>
<span class="source-line-no">1695</span><span id="line-1695"> // todo: handle the overlap issues for the custom paths</span>
<span class="source-line-no">1696</span><span id="line-1696"></span>
<span class="source-line-no">1697</span><span id="line-1697"> if (paths != null &amp;&amp; paths.length &gt; 0) {</span>
<span class="source-line-no">1698</span><span id="line-1698"> if (conf.getStrings(HFileCleaner.HFILE_CLEANER_CUSTOM_PATHS_PLUGINS) == null) {</span>
<span class="source-line-no">1699</span><span id="line-1699"> Set&lt;String&gt; cleanerClasses = new HashSet&lt;&gt;();</span>
<span class="source-line-no">1700</span><span id="line-1700"> String[] cleaners = conf.getStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS);</span>
<span class="source-line-no">1701</span><span id="line-1701"> if (cleaners != null) {</span>
<span class="source-line-no">1702</span><span id="line-1702"> Collections.addAll(cleanerClasses, cleaners);</span>
<span class="source-line-no">1703</span><span id="line-1703"> }</span>
<span class="source-line-no">1704</span><span id="line-1704"> conf.setStrings(HFileCleaner.HFILE_CLEANER_CUSTOM_PATHS_PLUGINS,</span>
<span class="source-line-no">1705</span><span id="line-1705"> cleanerClasses.toArray(new String[cleanerClasses.size()]));</span>
<span class="source-line-no">1706</span><span id="line-1706"> LOG.info("Archive custom cleaner paths: {}, plugins: {}", Arrays.asList(paths),</span>
<span class="source-line-no">1707</span><span id="line-1707"> cleanerClasses);</span>
<span class="source-line-no">1708</span><span id="line-1708"> }</span>
<span class="source-line-no">1709</span><span id="line-1709"> // share the hfile cleaner pool in custom paths</span>
<span class="source-line-no">1710</span><span id="line-1710"> sharedHFileCleanerPool = DirScanPool.getHFileCleanerScanPool(conf.get(CUSTOM_POOL_SIZE, "6"));</span>
<span class="source-line-no">1711</span><span id="line-1711"> for (int i = 0; i &lt; paths.length; i++) {</span>
<span class="source-line-no">1712</span><span id="line-1712"> Path path = new Path(paths[i].trim());</span>
<span class="source-line-no">1713</span><span id="line-1713"> HFileCleaner cleaner =</span>
<span class="source-line-no">1714</span><span id="line-1714"> new HFileCleaner("ArchiveCustomHFileCleaner-" + path.getName(), cleanerInterval, this,</span>
<span class="source-line-no">1715</span><span id="line-1715"> conf, getMasterFileSystem().getFileSystem(), new Path(archiveDir, path),</span>
<span class="source-line-no">1716</span><span id="line-1716"> HFileCleaner.HFILE_CLEANER_CUSTOM_PATHS_PLUGINS, sharedHFileCleanerPool, params, null);</span>
<span class="source-line-no">1717</span><span id="line-1717"> hfileCleaners.add(cleaner);</span>
<span class="source-line-no">1718</span><span id="line-1718"> hfileCleanerPaths.add(path);</span>
<span class="source-line-no">1719</span><span id="line-1719"> }</span>
<span class="source-line-no">1720</span><span id="line-1720"> }</span>
<span class="source-line-no">1721</span><span id="line-1721"></span>
<span class="source-line-no">1722</span><span id="line-1722"> // Create the whole archive dir cleaner thread pool</span>
<span class="source-line-no">1723</span><span id="line-1723"> exclusiveHFileCleanerPool = DirScanPool.getHFileCleanerScanPool(conf);</span>
<span class="source-line-no">1724</span><span id="line-1724"> hfileCleaners.add(0,</span>
<span class="source-line-no">1725</span><span id="line-1725"> new HFileCleaner(cleanerInterval, this, conf, getMasterFileSystem().getFileSystem(),</span>
<span class="source-line-no">1726</span><span id="line-1726"> archiveDir, exclusiveHFileCleanerPool, params, hfileCleanerPaths));</span>
<span class="source-line-no">1727</span><span id="line-1727"> hfileCleanerPaths.add(0, archiveDir);</span>
<span class="source-line-no">1728</span><span id="line-1728"> // Schedule all the hfile cleaners</span>
<span class="source-line-no">1729</span><span id="line-1729"> for (HFileCleaner hFileCleaner : hfileCleaners) {</span>
<span class="source-line-no">1730</span><span id="line-1730"> getChoreService().scheduleChore(hFileCleaner);</span>
<span class="source-line-no">1731</span><span id="line-1731"> }</span>
<span class="source-line-no">1732</span><span id="line-1732"></span>
<span class="source-line-no">1733</span><span id="line-1733"> // Regions Reopen based on very high storeFileRefCount is considered enabled</span>
<span class="source-line-no">1734</span><span id="line-1734"> // only if hbase.regions.recovery.store.file.ref.count has value &gt; 0</span>
<span class="source-line-no">1735</span><span id="line-1735"> final int maxStoreFileRefCount = conf.getInt(HConstants.STORE_FILE_REF_COUNT_THRESHOLD,</span>
<span class="source-line-no">1736</span><span id="line-1736"> HConstants.DEFAULT_STORE_FILE_REF_COUNT_THRESHOLD);</span>
<span class="source-line-no">1737</span><span id="line-1737"> if (maxStoreFileRefCount &gt; 0) {</span>
<span class="source-line-no">1738</span><span id="line-1738"> this.regionsRecoveryChore = new RegionsRecoveryChore(this, conf, this);</span>
<span class="source-line-no">1739</span><span id="line-1739"> getChoreService().scheduleChore(this.regionsRecoveryChore);</span>
<span class="source-line-no">1740</span><span id="line-1740"> } else {</span>
<span class="source-line-no">1741</span><span id="line-1741"> LOG.info(</span>
<span class="source-line-no">1742</span><span id="line-1742"> "Reopening regions with very high storeFileRefCount is disabled. "</span>
<span class="source-line-no">1743</span><span id="line-1743"> + "Provide threshold value &gt; 0 for {} to enable it.",</span>
<span class="source-line-no">1744</span><span id="line-1744"> HConstants.STORE_FILE_REF_COUNT_THRESHOLD);</span>
<span class="source-line-no">1745</span><span id="line-1745"> }</span>
<span class="source-line-no">1746</span><span id="line-1746"></span>
<span class="source-line-no">1747</span><span id="line-1747"> this.regionsRecoveryConfigManager = new RegionsRecoveryConfigManager(this);</span>
<span class="source-line-no">1748</span><span id="line-1748"></span>
<span class="source-line-no">1749</span><span id="line-1749"> replicationBarrierCleaner =</span>
<span class="source-line-no">1750</span><span id="line-1750"> new ReplicationBarrierCleaner(conf, this, getConnection(), replicationPeerManager);</span>
<span class="source-line-no">1751</span><span id="line-1751"> getChoreService().scheduleChore(replicationBarrierCleaner);</span>
<span class="source-line-no">1752</span><span id="line-1752"></span>
<span class="source-line-no">1753</span><span id="line-1753"> final boolean isSnapshotChoreEnabled = this.snapshotCleanupStateStore.get();</span>
<span class="source-line-no">1754</span><span id="line-1754"> this.snapshotCleanerChore = new SnapshotCleanerChore(this, conf, getSnapshotManager());</span>
<span class="source-line-no">1755</span><span id="line-1755"> if (isSnapshotChoreEnabled) {</span>
<span class="source-line-no">1756</span><span id="line-1756"> getChoreService().scheduleChore(this.snapshotCleanerChore);</span>
<span class="source-line-no">1757</span><span id="line-1757"> } else {</span>
<span class="source-line-no">1758</span><span id="line-1758"> if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">1759</span><span id="line-1759"> LOG.trace("Snapshot Cleaner Chore is disabled. Not starting up the chore..");</span>
<span class="source-line-no">1760</span><span id="line-1760"> }</span>
<span class="source-line-no">1761</span><span id="line-1761"> }</span>
<span class="source-line-no">1762</span><span id="line-1762"> serviceStarted = true;</span>
<span class="source-line-no">1763</span><span id="line-1763"> if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">1764</span><span id="line-1764"> LOG.trace("Started service threads");</span>
<span class="source-line-no">1765</span><span id="line-1765"> }</span>
<span class="source-line-no">1766</span><span id="line-1766"> }</span>
<span class="source-line-no">1767</span><span id="line-1767"></span>
<span class="source-line-no">1768</span><span id="line-1768"> protected void stopServiceThreads() {</span>
<span class="source-line-no">1769</span><span id="line-1769"> if (masterJettyServer != null) {</span>
<span class="source-line-no">1770</span><span id="line-1770"> LOG.info("Stopping master jetty server");</span>
<span class="source-line-no">1771</span><span id="line-1771"> try {</span>
<span class="source-line-no">1772</span><span id="line-1772"> masterJettyServer.stop();</span>
<span class="source-line-no">1773</span><span id="line-1773"> } catch (Exception e) {</span>
<span class="source-line-no">1774</span><span id="line-1774"> LOG.error("Failed to stop master jetty server", e);</span>
<span class="source-line-no">1775</span><span id="line-1775"> }</span>
<span class="source-line-no">1776</span><span id="line-1776"> }</span>
<span class="source-line-no">1777</span><span id="line-1777"> stopChoreService();</span>
<span class="source-line-no">1778</span><span id="line-1778"> stopExecutorService();</span>
<span class="source-line-no">1779</span><span id="line-1779"> if (exclusiveHFileCleanerPool != null) {</span>
<span class="source-line-no">1780</span><span id="line-1780"> exclusiveHFileCleanerPool.shutdownNow();</span>
<span class="source-line-no">1781</span><span id="line-1781"> exclusiveHFileCleanerPool = null;</span>
<span class="source-line-no">1782</span><span id="line-1782"> }</span>
<span class="source-line-no">1783</span><span id="line-1783"> if (logCleanerPool != null) {</span>
<span class="source-line-no">1784</span><span id="line-1784"> logCleanerPool.shutdownNow();</span>
<span class="source-line-no">1785</span><span id="line-1785"> logCleanerPool = null;</span>
<span class="source-line-no">1786</span><span id="line-1786"> }</span>
<span class="source-line-no">1787</span><span id="line-1787"> if (sharedHFileCleanerPool != null) {</span>
<span class="source-line-no">1788</span><span id="line-1788"> sharedHFileCleanerPool.shutdownNow();</span>
<span class="source-line-no">1789</span><span id="line-1789"> sharedHFileCleanerPool = null;</span>
<span class="source-line-no">1790</span><span id="line-1790"> }</span>
<span class="source-line-no">1791</span><span id="line-1791"> if (maintenanceRegionServer != null) {</span>
<span class="source-line-no">1792</span><span id="line-1792"> maintenanceRegionServer.getRegionServer().stop(HBASE_MASTER_CLEANER_INTERVAL);</span>
<span class="source-line-no">1793</span><span id="line-1793"> }</span>
<span class="source-line-no">1794</span><span id="line-1794"></span>
<span class="source-line-no">1795</span><span id="line-1795"> LOG.debug("Stopping service threads");</span>
<span class="source-line-no">1796</span><span id="line-1796"> // stop procedure executor prior to other services such as server manager and assignment</span>
<span class="source-line-no">1797</span><span id="line-1797"> // manager, as these services are important for some running procedures. See HBASE-24117 for</span>
<span class="source-line-no">1798</span><span id="line-1798"> // example.</span>
<span class="source-line-no">1799</span><span id="line-1799"> stopProcedureExecutor();</span>
<span class="source-line-no">1800</span><span id="line-1800"></span>
<span class="source-line-no">1801</span><span id="line-1801"> if (regionNormalizerManager != null) {</span>
<span class="source-line-no">1802</span><span id="line-1802"> regionNormalizerManager.stop();</span>
<span class="source-line-no">1803</span><span id="line-1803"> }</span>
<span class="source-line-no">1804</span><span id="line-1804"> if (this.quotaManager != null) {</span>
<span class="source-line-no">1805</span><span id="line-1805"> this.quotaManager.stop();</span>
<span class="source-line-no">1806</span><span id="line-1806"> }</span>
<span class="source-line-no">1807</span><span id="line-1807"></span>
<span class="source-line-no">1808</span><span id="line-1808"> if (this.activeMasterManager != null) {</span>
<span class="source-line-no">1809</span><span id="line-1809"> this.activeMasterManager.stop();</span>
<span class="source-line-no">1810</span><span id="line-1810"> }</span>
<span class="source-line-no">1811</span><span id="line-1811"> if (this.serverManager != null) {</span>
<span class="source-line-no">1812</span><span id="line-1812"> this.serverManager.stop();</span>
<span class="source-line-no">1813</span><span id="line-1813"> }</span>
<span class="source-line-no">1814</span><span id="line-1814"> if (this.assignmentManager != null) {</span>
<span class="source-line-no">1815</span><span id="line-1815"> this.assignmentManager.stop();</span>
<span class="source-line-no">1816</span><span id="line-1816"> }</span>
<span class="source-line-no">1817</span><span id="line-1817"></span>
<span class="source-line-no">1818</span><span id="line-1818"> if (masterRegion != null) {</span>
<span class="source-line-no">1819</span><span id="line-1819"> masterRegion.close(isAborted());</span>
<span class="source-line-no">1820</span><span id="line-1820"> }</span>
<span class="source-line-no">1821</span><span id="line-1821"> if (this.walManager != null) {</span>
<span class="source-line-no">1822</span><span id="line-1822"> this.walManager.stop();</span>
<span class="source-line-no">1823</span><span id="line-1823"> }</span>
<span class="source-line-no">1824</span><span id="line-1824"> if (this.fileSystemManager != null) {</span>
<span class="source-line-no">1825</span><span id="line-1825"> this.fileSystemManager.stop();</span>
<span class="source-line-no">1826</span><span id="line-1826"> }</span>
<span class="source-line-no">1827</span><span id="line-1827"> if (this.mpmHost != null) {</span>
<span class="source-line-no">1828</span><span id="line-1828"> this.mpmHost.stop("server shutting down.");</span>
<span class="source-line-no">1829</span><span id="line-1829"> }</span>
<span class="source-line-no">1830</span><span id="line-1830"> if (this.regionServerTracker != null) {</span>
<span class="source-line-no">1831</span><span id="line-1831"> this.regionServerTracker.stop();</span>
<span class="source-line-no">1832</span><span id="line-1832"> }</span>
<span class="source-line-no">1833</span><span id="line-1833"> }</span>
<span class="source-line-no">1834</span><span id="line-1834"></span>
<span class="source-line-no">1835</span><span id="line-1835"> private void createProcedureExecutor() throws IOException {</span>
<span class="source-line-no">1836</span><span id="line-1836"> MasterProcedureEnv procEnv = new MasterProcedureEnv(this);</span>
<span class="source-line-no">1837</span><span id="line-1837"> procedureStore = new RegionProcedureStore(this, masterRegion,</span>
<span class="source-line-no">1838</span><span id="line-1838"> new MasterProcedureEnv.FsUtilsLeaseRecovery(this));</span>
<span class="source-line-no">1839</span><span id="line-1839"> procedureStore.registerListener(new ProcedureStoreListener() {</span>
<span class="source-line-no">1840</span><span id="line-1840"></span>
<span class="source-line-no">1841</span><span id="line-1841"> @Override</span>
<span class="source-line-no">1842</span><span id="line-1842"> public void abortProcess() {</span>
<span class="source-line-no">1843</span><span id="line-1843"> abort("The Procedure Store lost the lease", null);</span>
<span class="source-line-no">1844</span><span id="line-1844"> }</span>
<span class="source-line-no">1845</span><span id="line-1845"> });</span>
<span class="source-line-no">1846</span><span id="line-1846"> MasterProcedureScheduler procedureScheduler = procEnv.getProcedureScheduler();</span>
<span class="source-line-no">1847</span><span id="line-1847"> procedureExecutor = new ProcedureExecutor&lt;&gt;(conf, procEnv, procedureStore, procedureScheduler);</span>
<span class="source-line-no">1848</span><span id="line-1848"> configurationManager.registerObserver(procEnv);</span>
<span class="source-line-no">1849</span><span id="line-1849"></span>
<span class="source-line-no">1850</span><span id="line-1850"> int cpus = Runtime.getRuntime().availableProcessors();</span>
<span class="source-line-no">1851</span><span id="line-1851"> final int numThreads = conf.getInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, Math.max(</span>
<span class="source-line-no">1852</span><span id="line-1852"> (cpus &gt; 0 ? cpus / 4 : 0), MasterProcedureConstants.DEFAULT_MIN_MASTER_PROCEDURE_THREADS));</span>
<span class="source-line-no">1853</span><span id="line-1853"> final boolean abortOnCorruption =</span>
<span class="source-line-no">1854</span><span id="line-1854"> conf.getBoolean(MasterProcedureConstants.EXECUTOR_ABORT_ON_CORRUPTION,</span>
<span class="source-line-no">1855</span><span id="line-1855"> MasterProcedureConstants.DEFAULT_EXECUTOR_ABORT_ON_CORRUPTION);</span>
<span class="source-line-no">1856</span><span id="line-1856"> procedureStore.start(numThreads);</span>
<span class="source-line-no">1857</span><span id="line-1857"> // Just initialize it but do not start the workers, we will start the workers later by calling</span>
<span class="source-line-no">1858</span><span id="line-1858"> // startProcedureExecutor. See the javadoc for finishActiveMasterInitialization for more</span>
<span class="source-line-no">1859</span><span id="line-1859"> // details.</span>
<span class="source-line-no">1860</span><span id="line-1860"> procedureExecutor.init(numThreads, abortOnCorruption);</span>
<span class="source-line-no">1861</span><span id="line-1861"> if (!procEnv.getRemoteDispatcher().start()) {</span>
<span class="source-line-no">1862</span><span id="line-1862"> throw new HBaseIOException("Failed start of remote dispatcher");</span>
<span class="source-line-no">1863</span><span id="line-1863"> }</span>
<span class="source-line-no">1864</span><span id="line-1864"> }</span>
<span class="source-line-no">1865</span><span id="line-1865"></span>
<span class="source-line-no">1866</span><span id="line-1866"> // will be override in UT</span>
<span class="source-line-no">1867</span><span id="line-1867"> protected void startProcedureExecutor() throws IOException {</span>
<span class="source-line-no">1868</span><span id="line-1868"> procedureExecutor.startWorkers();</span>
<span class="source-line-no">1869</span><span id="line-1869"> }</span>
<span class="source-line-no">1870</span><span id="line-1870"></span>
<span class="source-line-no">1871</span><span id="line-1871"> /**</span>
<span class="source-line-no">1872</span><span id="line-1872"> * Turn on/off Snapshot Cleanup Chore</span>
<span class="source-line-no">1873</span><span id="line-1873"> * @param on indicates whether Snapshot Cleanup Chore is to be run</span>
<span class="source-line-no">1874</span><span id="line-1874"> */</span>
<span class="source-line-no">1875</span><span id="line-1875"> void switchSnapshotCleanup(final boolean on, final boolean synchronous) throws IOException {</span>
<span class="source-line-no">1876</span><span id="line-1876"> if (synchronous) {</span>
<span class="source-line-no">1877</span><span id="line-1877"> synchronized (this.snapshotCleanerChore) {</span>
<span class="source-line-no">1878</span><span id="line-1878"> switchSnapshotCleanup(on);</span>
<span class="source-line-no">1879</span><span id="line-1879"> }</span>
<span class="source-line-no">1880</span><span id="line-1880"> } else {</span>
<span class="source-line-no">1881</span><span id="line-1881"> switchSnapshotCleanup(on);</span>
<span class="source-line-no">1882</span><span id="line-1882"> }</span>
<span class="source-line-no">1883</span><span id="line-1883"> }</span>
<span class="source-line-no">1884</span><span id="line-1884"></span>
<span class="source-line-no">1885</span><span id="line-1885"> private void switchSnapshotCleanup(final boolean on) throws IOException {</span>
<span class="source-line-no">1886</span><span id="line-1886"> snapshotCleanupStateStore.set(on);</span>
<span class="source-line-no">1887</span><span id="line-1887"> if (on) {</span>
<span class="source-line-no">1888</span><span id="line-1888"> getChoreService().scheduleChore(this.snapshotCleanerChore);</span>
<span class="source-line-no">1889</span><span id="line-1889"> } else {</span>
<span class="source-line-no">1890</span><span id="line-1890"> this.snapshotCleanerChore.cancel();</span>
<span class="source-line-no">1891</span><span id="line-1891"> }</span>
<span class="source-line-no">1892</span><span id="line-1892"> }</span>
<span class="source-line-no">1893</span><span id="line-1893"></span>
<span class="source-line-no">1894</span><span id="line-1894"> private void stopProcedureExecutor() {</span>
<span class="source-line-no">1895</span><span id="line-1895"> if (procedureExecutor != null) {</span>
<span class="source-line-no">1896</span><span id="line-1896"> configurationManager.deregisterObserver(procedureExecutor.getEnvironment());</span>
<span class="source-line-no">1897</span><span id="line-1897"> procedureExecutor.getEnvironment().getRemoteDispatcher().stop();</span>
<span class="source-line-no">1898</span><span id="line-1898"> procedureExecutor.stop();</span>
<span class="source-line-no">1899</span><span id="line-1899"> procedureExecutor.join();</span>
<span class="source-line-no">1900</span><span id="line-1900"> procedureExecutor = null;</span>
<span class="source-line-no">1901</span><span id="line-1901"> }</span>
<span class="source-line-no">1902</span><span id="line-1902"></span>
<span class="source-line-no">1903</span><span id="line-1903"> if (procedureStore != null) {</span>
<span class="source-line-no">1904</span><span id="line-1904"> procedureStore.stop(isAborted());</span>
<span class="source-line-no">1905</span><span id="line-1905"> procedureStore = null;</span>
<span class="source-line-no">1906</span><span id="line-1906"> }</span>
<span class="source-line-no">1907</span><span id="line-1907"> }</span>
<span class="source-line-no">1908</span><span id="line-1908"></span>
<span class="source-line-no">1909</span><span id="line-1909"> protected void stopChores() {</span>
<span class="source-line-no">1910</span><span id="line-1910"> shutdownChore(mobFileCleanerChore);</span>
<span class="source-line-no">1911</span><span id="line-1911"> shutdownChore(mobFileCompactionChore);</span>
<span class="source-line-no">1912</span><span id="line-1912"> shutdownChore(balancerChore);</span>
<span class="source-line-no">1913</span><span id="line-1913"> if (regionNormalizerManager != null) {</span>
<span class="source-line-no">1914</span><span id="line-1914"> shutdownChore(regionNormalizerManager.getRegionNormalizerChore());</span>
<span class="source-line-no">1915</span><span id="line-1915"> }</span>
<span class="source-line-no">1916</span><span id="line-1916"> shutdownChore(clusterStatusChore);</span>
<span class="source-line-no">1917</span><span id="line-1917"> shutdownChore(catalogJanitorChore);</span>
<span class="source-line-no">1918</span><span id="line-1918"> shutdownChore(clusterStatusPublisherChore);</span>
<span class="source-line-no">1919</span><span id="line-1919"> shutdownChore(snapshotQuotaChore);</span>
<span class="source-line-no">1920</span><span id="line-1920"> shutdownChore(logCleaner);</span>
<span class="source-line-no">1921</span><span id="line-1921"> if (hfileCleaners != null) {</span>
<span class="source-line-no">1922</span><span id="line-1922"> for (ScheduledChore chore : hfileCleaners) {</span>
<span class="source-line-no">1923</span><span id="line-1923"> chore.shutdown();</span>
<span class="source-line-no">1924</span><span id="line-1924"> }</span>
<span class="source-line-no">1925</span><span id="line-1925"> hfileCleaners = null;</span>
<span class="source-line-no">1926</span><span id="line-1926"> }</span>
<span class="source-line-no">1927</span><span id="line-1927"> shutdownChore(replicationBarrierCleaner);</span>
<span class="source-line-no">1928</span><span id="line-1928"> shutdownChore(snapshotCleanerChore);</span>
<span class="source-line-no">1929</span><span id="line-1929"> shutdownChore(hbckChore);</span>
<span class="source-line-no">1930</span><span id="line-1930"> shutdownChore(regionsRecoveryChore);</span>
<span class="source-line-no">1931</span><span id="line-1931"> shutdownChore(rollingUpgradeChore);</span>
<span class="source-line-no">1932</span><span id="line-1932"> shutdownChore(oldWALsDirSizeChore);</span>
<span class="source-line-no">1933</span><span id="line-1933"> }</span>
<span class="source-line-no">1934</span><span id="line-1934"></span>
<span class="source-line-no">1935</span><span id="line-1935"> /** Returns Get remote side's InetAddress */</span>
<span class="source-line-no">1936</span><span id="line-1936"> InetAddress getRemoteInetAddress(final int port, final long serverStartCode)</span>
<span class="source-line-no">1937</span><span id="line-1937"> throws UnknownHostException {</span>
<span class="source-line-no">1938</span><span id="line-1938"> // Do it out here in its own little method so can fake an address when</span>
<span class="source-line-no">1939</span><span id="line-1939"> // mocking up in tests.</span>
<span class="source-line-no">1940</span><span id="line-1940"> InetAddress ia = RpcServer.getRemoteIp();</span>
<span class="source-line-no">1941</span><span id="line-1941"></span>
<span class="source-line-no">1942</span><span id="line-1942"> // The call could be from the local regionserver,</span>
<span class="source-line-no">1943</span><span id="line-1943"> // in which case, there is no remote address.</span>
<span class="source-line-no">1944</span><span id="line-1944"> if (ia == null &amp;&amp; serverStartCode == startcode) {</span>
<span class="source-line-no">1945</span><span id="line-1945"> InetSocketAddress isa = rpcServices.getSocketAddress();</span>
<span class="source-line-no">1946</span><span id="line-1946"> if (isa != null &amp;&amp; isa.getPort() == port) {</span>
<span class="source-line-no">1947</span><span id="line-1947"> ia = isa.getAddress();</span>
<span class="source-line-no">1948</span><span id="line-1948"> }</span>
<span class="source-line-no">1949</span><span id="line-1949"> }</span>
<span class="source-line-no">1950</span><span id="line-1950"> return ia;</span>
<span class="source-line-no">1951</span><span id="line-1951"> }</span>
<span class="source-line-no">1952</span><span id="line-1952"></span>
<span class="source-line-no">1953</span><span id="line-1953"> /** Returns Maximum time we should run balancer for */</span>
<span class="source-line-no">1954</span><span id="line-1954"> private int getMaxBalancingTime() {</span>
<span class="source-line-no">1955</span><span id="line-1955"> // if max balancing time isn't set, defaulting it to period time</span>
<span class="source-line-no">1956</span><span id="line-1956"> int maxBalancingTime =</span>
<span class="source-line-no">1957</span><span id="line-1957"> getConfiguration().getInt(HConstants.HBASE_BALANCER_MAX_BALANCING, getConfiguration()</span>
<span class="source-line-no">1958</span><span id="line-1958"> .getInt(HConstants.HBASE_BALANCER_PERIOD, HConstants.DEFAULT_HBASE_BALANCER_PERIOD));</span>
<span class="source-line-no">1959</span><span id="line-1959"> return maxBalancingTime;</span>
<span class="source-line-no">1960</span><span id="line-1960"> }</span>
<span class="source-line-no">1961</span><span id="line-1961"></span>
<span class="source-line-no">1962</span><span id="line-1962"> /** Returns Maximum number of regions in transition */</span>
<span class="source-line-no">1963</span><span id="line-1963"> private int getMaxRegionsInTransition() {</span>
<span class="source-line-no">1964</span><span id="line-1964"> int numRegions = this.assignmentManager.getRegionStates().getRegionAssignments().size();</span>
<span class="source-line-no">1965</span><span id="line-1965"> return Math.max((int) Math.floor(numRegions * this.maxRitPercent), 1);</span>
<span class="source-line-no">1966</span><span id="line-1966"> }</span>
<span class="source-line-no">1967</span><span id="line-1967"></span>
<span class="source-line-no">1968</span><span id="line-1968"> /**</span>
<span class="source-line-no">1969</span><span id="line-1969"> * It first sleep to the next balance plan start time. Meanwhile, throttling by the max number</span>
<span class="source-line-no">1970</span><span id="line-1970"> * regions in transition to protect availability.</span>
<span class="source-line-no">1971</span><span id="line-1971"> * @param nextBalanceStartTime The next balance plan start time</span>
<span class="source-line-no">1972</span><span id="line-1972"> * @param maxRegionsInTransition max number of regions in transition</span>
<span class="source-line-no">1973</span><span id="line-1973"> * @param cutoffTime when to exit balancer</span>
<span class="source-line-no">1974</span><span id="line-1974"> */</span>
<span class="source-line-no">1975</span><span id="line-1975"> private void balanceThrottling(long nextBalanceStartTime, int maxRegionsInTransition,</span>
<span class="source-line-no">1976</span><span id="line-1976"> long cutoffTime) {</span>
<span class="source-line-no">1977</span><span id="line-1977"> boolean interrupted = false;</span>
<span class="source-line-no">1978</span><span id="line-1978"></span>
<span class="source-line-no">1979</span><span id="line-1979"> // Sleep to next balance plan start time</span>
<span class="source-line-no">1980</span><span id="line-1980"> // But if there are zero regions in transition, it can skip sleep to speed up.</span>
<span class="source-line-no">1981</span><span id="line-1981"> while (</span>
<span class="source-line-no">1982</span><span id="line-1982"> !interrupted &amp;&amp; EnvironmentEdgeManager.currentTime() &lt; nextBalanceStartTime</span>
<span class="source-line-no">1983</span><span id="line-1983"> &amp;&amp; this.assignmentManager.getRegionStates().hasRegionsInTransition()</span>
<span class="source-line-no">1984</span><span id="line-1984"> ) {</span>
<span class="source-line-no">1985</span><span id="line-1985"> try {</span>
<span class="source-line-no">1986</span><span id="line-1986"> Thread.sleep(100);</span>
<span class="source-line-no">1987</span><span id="line-1987"> } catch (InterruptedException ie) {</span>
<span class="source-line-no">1988</span><span id="line-1988"> interrupted = true;</span>
<span class="source-line-no">1989</span><span id="line-1989"> }</span>
<span class="source-line-no">1990</span><span id="line-1990"> }</span>
<span class="source-line-no">1991</span><span id="line-1991"></span>
<span class="source-line-no">1992</span><span id="line-1992"> // Throttling by max number regions in transition</span>
<span class="source-line-no">1993</span><span id="line-1993"> while (</span>
<span class="source-line-no">1994</span><span id="line-1994"> !interrupted &amp;&amp; maxRegionsInTransition &gt; 0</span>
<span class="source-line-no">1995</span><span id="line-1995"> &amp;&amp; this.assignmentManager.getRegionStates().getRegionsInTransitionCount()</span>
<span class="source-line-no">1996</span><span id="line-1996"> &gt;= maxRegionsInTransition</span>
<span class="source-line-no">1997</span><span id="line-1997"> &amp;&amp; EnvironmentEdgeManager.currentTime() &lt;= cutoffTime</span>
<span class="source-line-no">1998</span><span id="line-1998"> ) {</span>
<span class="source-line-no">1999</span><span id="line-1999"> try {</span>
<span class="source-line-no">2000</span><span id="line-2000"> // sleep if the number of regions in transition exceeds the limit</span>
<span class="source-line-no">2001</span><span id="line-2001"> Thread.sleep(100);</span>
<span class="source-line-no">2002</span><span id="line-2002"> } catch (InterruptedException ie) {</span>
<span class="source-line-no">2003</span><span id="line-2003"> interrupted = true;</span>
<span class="source-line-no">2004</span><span id="line-2004"> }</span>
<span class="source-line-no">2005</span><span id="line-2005"> }</span>
<span class="source-line-no">2006</span><span id="line-2006"></span>
<span class="source-line-no">2007</span><span id="line-2007"> if (interrupted) Thread.currentThread().interrupt();</span>
<span class="source-line-no">2008</span><span id="line-2008"> }</span>
<span class="source-line-no">2009</span><span id="line-2009"></span>
<span class="source-line-no">2010</span><span id="line-2010"> public BalanceResponse balance() throws IOException {</span>
<span class="source-line-no">2011</span><span id="line-2011"> return balance(BalanceRequest.defaultInstance());</span>
<span class="source-line-no">2012</span><span id="line-2012"> }</span>
<span class="source-line-no">2013</span><span id="line-2013"></span>
<span class="source-line-no">2014</span><span id="line-2014"> /**</span>
<span class="source-line-no">2015</span><span id="line-2015"> * Trigger a normal balance, see {@link HMaster#balance()} . If the balance is not executed this</span>
<span class="source-line-no">2016</span><span id="line-2016"> * time, the metrics related to the balance will be updated. When balance is running, related</span>
<span class="source-line-no">2017</span><span id="line-2017"> * metrics will be updated at the same time. But if some checking logic failed and cause the</span>
<span class="source-line-no">2018</span><span id="line-2018"> * balancer exit early, we lost the chance to update balancer metrics. This will lead to user</span>
<span class="source-line-no">2019</span><span id="line-2019"> * missing the latest balancer info.</span>
<span class="source-line-no">2020</span><span id="line-2020"> */</span>
<span class="source-line-no">2021</span><span id="line-2021"> public BalanceResponse balanceOrUpdateMetrics() throws IOException {</span>
<span class="source-line-no">2022</span><span id="line-2022"> synchronized (this.balancer) {</span>
<span class="source-line-no">2023</span><span id="line-2023"> BalanceResponse response = balance();</span>
<span class="source-line-no">2024</span><span id="line-2024"> if (!response.isBalancerRan()) {</span>
<span class="source-line-no">2025</span><span id="line-2025"> Map&lt;TableName, Map&lt;ServerName, List&lt;RegionInfo&gt;&gt;&gt; assignments =</span>
<span class="source-line-no">2026</span><span id="line-2026"> this.assignmentManager.getRegionStates().getAssignmentsForBalancer(this.tableStateManager,</span>
<span class="source-line-no">2027</span><span id="line-2027"> this.serverManager.getOnlineServersList());</span>
<span class="source-line-no">2028</span><span id="line-2028"> for (Map&lt;ServerName, List&lt;RegionInfo&gt;&gt; serverMap : assignments.values()) {</span>
<span class="source-line-no">2029</span><span id="line-2029"> serverMap.keySet().removeAll(this.serverManager.getDrainingServersList());</span>
<span class="source-line-no">2030</span><span id="line-2030"> }</span>
<span class="source-line-no">2031</span><span id="line-2031"> this.balancer.updateBalancerLoadInfo(assignments);</span>
<span class="source-line-no">2032</span><span id="line-2032"> }</span>
<span class="source-line-no">2033</span><span id="line-2033"> return response;</span>
<span class="source-line-no">2034</span><span id="line-2034"> }</span>
<span class="source-line-no">2035</span><span id="line-2035"> }</span>
<span class="source-line-no">2036</span><span id="line-2036"></span>
<span class="source-line-no">2037</span><span id="line-2037"> /**</span>
<span class="source-line-no">2038</span><span id="line-2038"> * Checks master state before initiating action over region topology.</span>
<span class="source-line-no">2039</span><span id="line-2039"> * @param action the name of the action under consideration, for logging.</span>
<span class="source-line-no">2040</span><span id="line-2040"> * @return {@code true} when the caller should exit early, {@code false} otherwise.</span>
<span class="source-line-no">2041</span><span id="line-2041"> */</span>
<span class="source-line-no">2042</span><span id="line-2042"> @Override</span>
<span class="source-line-no">2043</span><span id="line-2043"> public boolean skipRegionManagementAction(final String action) {</span>
<span class="source-line-no">2044</span><span id="line-2044"> // Note: this method could be `default` on MasterServices if but for logging.</span>
<span class="source-line-no">2045</span><span id="line-2045"> if (!isInitialized()) {</span>
<span class="source-line-no">2046</span><span id="line-2046"> LOG.debug("Master has not been initialized, don't run {}.", action);</span>
<span class="source-line-no">2047</span><span id="line-2047"> return true;</span>
<span class="source-line-no">2048</span><span id="line-2048"> }</span>
<span class="source-line-no">2049</span><span id="line-2049"> if (this.getServerManager().isClusterShutdown()) {</span>
<span class="source-line-no">2050</span><span id="line-2050"> LOG.info("Cluster is shutting down, don't run {}.", action);</span>
<span class="source-line-no">2051</span><span id="line-2051"> return true;</span>
<span class="source-line-no">2052</span><span id="line-2052"> }</span>
<span class="source-line-no">2053</span><span id="line-2053"> if (isInMaintenanceMode()) {</span>
<span class="source-line-no">2054</span><span id="line-2054"> LOG.info("Master is in maintenance mode, don't run {}.", action);</span>
<span class="source-line-no">2055</span><span id="line-2055"> return true;</span>
<span class="source-line-no">2056</span><span id="line-2056"> }</span>
<span class="source-line-no">2057</span><span id="line-2057"> return false;</span>
<span class="source-line-no">2058</span><span id="line-2058"> }</span>
<span class="source-line-no">2059</span><span id="line-2059"></span>
<span class="source-line-no">2060</span><span id="line-2060"> public BalanceResponse balance(BalanceRequest request) throws IOException {</span>
<span class="source-line-no">2061</span><span id="line-2061"> checkInitialized();</span>
<span class="source-line-no">2062</span><span id="line-2062"></span>
<span class="source-line-no">2063</span><span id="line-2063"> BalanceResponse.Builder responseBuilder = BalanceResponse.newBuilder();</span>
<span class="source-line-no">2064</span><span id="line-2064"></span>
<span class="source-line-no">2065</span><span id="line-2065"> if (loadBalancerStateStore == null || !(loadBalancerStateStore.get() || request.isDryRun())) {</span>
<span class="source-line-no">2066</span><span id="line-2066"> return responseBuilder.build();</span>
<span class="source-line-no">2067</span><span id="line-2067"> }</span>
<span class="source-line-no">2068</span><span id="line-2068"></span>
<span class="source-line-no">2069</span><span id="line-2069"> if (skipRegionManagementAction("balancer")) {</span>
<span class="source-line-no">2070</span><span id="line-2070"> return responseBuilder.build();</span>
<span class="source-line-no">2071</span><span id="line-2071"> }</span>
<span class="source-line-no">2072</span><span id="line-2072"></span>
<span class="source-line-no">2073</span><span id="line-2073"> synchronized (this.balancer) {</span>
<span class="source-line-no">2074</span><span id="line-2074"> // Only allow one balance run at at time.</span>
<span class="source-line-no">2075</span><span id="line-2075"> if (this.assignmentManager.hasRegionsInTransition()) {</span>
<span class="source-line-no">2076</span><span id="line-2076"> List&lt;RegionStateNode&gt; regionsInTransition = assignmentManager.getRegionsInTransition();</span>
<span class="source-line-no">2077</span><span id="line-2077"> // if hbase:meta region is in transition, result of assignment cannot be recorded</span>
<span class="source-line-no">2078</span><span id="line-2078"> // ignore the force flag in that case</span>
<span class="source-line-no">2079</span><span id="line-2079"> boolean metaInTransition = assignmentManager.isMetaRegionInTransition();</span>
<span class="source-line-no">2080</span><span id="line-2080"> List&lt;RegionStateNode&gt; toPrint = regionsInTransition;</span>
<span class="source-line-no">2081</span><span id="line-2081"> int max = 5;</span>
<span class="source-line-no">2082</span><span id="line-2082"> boolean truncated = false;</span>
<span class="source-line-no">2083</span><span id="line-2083"> if (regionsInTransition.size() &gt; max) {</span>
<span class="source-line-no">2084</span><span id="line-2084"> toPrint = regionsInTransition.subList(0, max);</span>
<span class="source-line-no">2085</span><span id="line-2085"> truncated = true;</span>
<span class="source-line-no">2086</span><span id="line-2086"> }</span>
<span class="source-line-no">2087</span><span id="line-2087"></span>
<span class="source-line-no">2088</span><span id="line-2088"> if (!request.isIgnoreRegionsInTransition() || metaInTransition) {</span>
<span class="source-line-no">2089</span><span id="line-2089"> LOG.info("Not running balancer (ignoreRIT=false" + ", metaRIT=" + metaInTransition</span>
<span class="source-line-no">2090</span><span id="line-2090"> + ") because " + regionsInTransition.size() + " region(s) in transition: " + toPrint</span>
<span class="source-line-no">2091</span><span id="line-2091"> + (truncated ? "(truncated list)" : ""));</span>
<span class="source-line-no">2092</span><span id="line-2092"> return responseBuilder.build();</span>
<span class="source-line-no">2093</span><span id="line-2093"> }</span>
<span class="source-line-no">2094</span><span id="line-2094"> }</span>
<span class="source-line-no">2095</span><span id="line-2095"> if (this.serverManager.areDeadServersInProgress()) {</span>
<span class="source-line-no">2096</span><span id="line-2096"> LOG.info("Not running balancer because processing dead regionserver(s): "</span>
<span class="source-line-no">2097</span><span id="line-2097"> + this.serverManager.getDeadServers());</span>
<span class="source-line-no">2098</span><span id="line-2098"> return responseBuilder.build();</span>
<span class="source-line-no">2099</span><span id="line-2099"> }</span>
<span class="source-line-no">2100</span><span id="line-2100"></span>
<span class="source-line-no">2101</span><span id="line-2101"> if (this.cpHost != null) {</span>
<span class="source-line-no">2102</span><span id="line-2102"> try {</span>
<span class="source-line-no">2103</span><span id="line-2103"> if (this.cpHost.preBalance(request)) {</span>
<span class="source-line-no">2104</span><span id="line-2104"> LOG.debug("Coprocessor bypassing balancer request");</span>
<span class="source-line-no">2105</span><span id="line-2105"> return responseBuilder.build();</span>
<span class="source-line-no">2106</span><span id="line-2106"> }</span>
<span class="source-line-no">2107</span><span id="line-2107"> } catch (IOException ioe) {</span>
<span class="source-line-no">2108</span><span id="line-2108"> LOG.error("Error invoking master coprocessor preBalance()", ioe);</span>
<span class="source-line-no">2109</span><span id="line-2109"> return responseBuilder.build();</span>
<span class="source-line-no">2110</span><span id="line-2110"> }</span>
<span class="source-line-no">2111</span><span id="line-2111"> }</span>
<span class="source-line-no">2112</span><span id="line-2112"></span>
<span class="source-line-no">2113</span><span id="line-2113"> Map&lt;TableName, Map&lt;ServerName, List&lt;RegionInfo&gt;&gt;&gt; assignments =</span>
<span class="source-line-no">2114</span><span id="line-2114"> this.assignmentManager.getRegionStates().getAssignmentsForBalancer(tableStateManager,</span>
<span class="source-line-no">2115</span><span id="line-2115"> this.serverManager.getOnlineServersList());</span>
<span class="source-line-no">2116</span><span id="line-2116"> for (Map&lt;ServerName, List&lt;RegionInfo&gt;&gt; serverMap : assignments.values()) {</span>
<span class="source-line-no">2117</span><span id="line-2117"> serverMap.keySet().removeAll(this.serverManager.getDrainingServersList());</span>
<span class="source-line-no">2118</span><span id="line-2118"> }</span>
<span class="source-line-no">2119</span><span id="line-2119"></span>
<span class="source-line-no">2120</span><span id="line-2120"> // Give the balancer the current cluster state.</span>
<span class="source-line-no">2121</span><span id="line-2121"> this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());</span>
<span class="source-line-no">2122</span><span id="line-2122"></span>
<span class="source-line-no">2123</span><span id="line-2123"> List&lt;RegionPlan&gt; plans = this.balancer.balanceCluster(assignments);</span>
<span class="source-line-no">2124</span><span id="line-2124"></span>
<span class="source-line-no">2125</span><span id="line-2125"> responseBuilder.setBalancerRan(true).setMovesCalculated(plans == null ? 0 : plans.size());</span>
<span class="source-line-no">2126</span><span id="line-2126"></span>
<span class="source-line-no">2127</span><span id="line-2127"> if (skipRegionManagementAction("balancer")) {</span>
<span class="source-line-no">2128</span><span id="line-2128"> // make one last check that the cluster isn't shutting down before proceeding.</span>
<span class="source-line-no">2129</span><span id="line-2129"> return responseBuilder.build();</span>
<span class="source-line-no">2130</span><span id="line-2130"> }</span>
<span class="source-line-no">2131</span><span id="line-2131"></span>
<span class="source-line-no">2132</span><span id="line-2132"> // For dry run we don't actually want to execute the moves, but we do want</span>
<span class="source-line-no">2133</span><span id="line-2133"> // to execute the coprocessor below</span>
<span class="source-line-no">2134</span><span id="line-2134"> List&lt;RegionPlan&gt; sucRPs =</span>
<span class="source-line-no">2135</span><span id="line-2135"> request.isDryRun() ? Collections.emptyList() : executeRegionPlansWithThrottling(plans);</span>
<span class="source-line-no">2136</span><span id="line-2136"></span>
<span class="source-line-no">2137</span><span id="line-2137"> if (this.cpHost != null) {</span>
<span class="source-line-no">2138</span><span id="line-2138"> try {</span>
<span class="source-line-no">2139</span><span id="line-2139"> this.cpHost.postBalance(request, sucRPs);</span>
<span class="source-line-no">2140</span><span id="line-2140"> } catch (IOException ioe) {</span>
<span class="source-line-no">2141</span><span id="line-2141"> // balancing already succeeded so don't change the result</span>
<span class="source-line-no">2142</span><span id="line-2142"> LOG.error("Error invoking master coprocessor postBalance()", ioe);</span>
<span class="source-line-no">2143</span><span id="line-2143"> }</span>
<span class="source-line-no">2144</span><span id="line-2144"> }</span>
<span class="source-line-no">2145</span><span id="line-2145"></span>
<span class="source-line-no">2146</span><span id="line-2146"> responseBuilder.setMovesExecuted(sucRPs.size());</span>
<span class="source-line-no">2147</span><span id="line-2147"> }</span>
<span class="source-line-no">2148</span><span id="line-2148"></span>
<span class="source-line-no">2149</span><span id="line-2149"> // If LoadBalancer did not generate any plans, it means the cluster is already balanced.</span>
<span class="source-line-no">2150</span><span id="line-2150"> // Return true indicating a success.</span>
<span class="source-line-no">2151</span><span id="line-2151"> return responseBuilder.build();</span>
<span class="source-line-no">2152</span><span id="line-2152"> }</span>
<span class="source-line-no">2153</span><span id="line-2153"></span>
<span class="source-line-no">2154</span><span id="line-2154"> /**</span>
<span class="source-line-no">2155</span><span id="line-2155"> * Execute region plans with throttling</span>
<span class="source-line-no">2156</span><span id="line-2156"> * @param plans to execute</span>
<span class="source-line-no">2157</span><span id="line-2157"> * @return succeeded plans</span>
<span class="source-line-no">2158</span><span id="line-2158"> */</span>
<span class="source-line-no">2159</span><span id="line-2159"> public List&lt;RegionPlan&gt; executeRegionPlansWithThrottling(List&lt;RegionPlan&gt; plans) {</span>
<span class="source-line-no">2160</span><span id="line-2160"> List&lt;RegionPlan&gt; successRegionPlans = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">2161</span><span id="line-2161"> int maxRegionsInTransition = getMaxRegionsInTransition();</span>
<span class="source-line-no">2162</span><span id="line-2162"> long balanceStartTime = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">2163</span><span id="line-2163"> long cutoffTime = balanceStartTime + this.maxBalancingTime;</span>
<span class="source-line-no">2164</span><span id="line-2164"> int rpCount = 0; // number of RegionPlans balanced so far</span>
<span class="source-line-no">2165</span><span id="line-2165"> if (plans != null &amp;&amp; !plans.isEmpty()) {</span>
<span class="source-line-no">2166</span><span id="line-2166"> int balanceInterval = this.maxBalancingTime / plans.size();</span>
<span class="source-line-no">2167</span><span id="line-2167"> LOG.info(</span>
<span class="source-line-no">2168</span><span id="line-2168"> "Balancer plans size is " + plans.size() + ", the balance interval is " + balanceInterval</span>
<span class="source-line-no">2169</span><span id="line-2169"> + " ms, and the max number regions in transition is " + maxRegionsInTransition);</span>
<span class="source-line-no">2170</span><span id="line-2170"></span>
<span class="source-line-no">2171</span><span id="line-2171"> for (RegionPlan plan : plans) {</span>
<span class="source-line-no">2172</span><span id="line-2172"> LOG.info("balance " + plan);</span>
<span class="source-line-no">2173</span><span id="line-2173"> // TODO: bulk assign</span>
<span class="source-line-no">2174</span><span id="line-2174"> try {</span>
<span class="source-line-no">2175</span><span id="line-2175"> this.assignmentManager.balance(plan);</span>
<span class="source-line-no">2176</span><span id="line-2176"> } catch (HBaseIOException hioe) {</span>
<span class="source-line-no">2177</span><span id="line-2177"> // should ignore failed plans here, avoiding the whole balance plans be aborted</span>
<span class="source-line-no">2178</span><span id="line-2178"> // later calls of balance() can fetch up the failed and skipped plans</span>
<span class="source-line-no">2179</span><span id="line-2179"> LOG.warn("Failed balance plan {}, skipping...", plan, hioe);</span>
<span class="source-line-no">2180</span><span id="line-2180"> }</span>
<span class="source-line-no">2181</span><span id="line-2181"> // rpCount records balance plans processed, does not care if a plan succeeds</span>
<span class="source-line-no">2182</span><span id="line-2182"> rpCount++;</span>
<span class="source-line-no">2183</span><span id="line-2183"> successRegionPlans.add(plan);</span>
<span class="source-line-no">2184</span><span id="line-2184"></span>
<span class="source-line-no">2185</span><span id="line-2185"> if (this.maxBalancingTime &gt; 0) {</span>
<span class="source-line-no">2186</span><span id="line-2186"> balanceThrottling(balanceStartTime + rpCount * balanceInterval, maxRegionsInTransition,</span>
<span class="source-line-no">2187</span><span id="line-2187"> cutoffTime);</span>
<span class="source-line-no">2188</span><span id="line-2188"> }</span>
<span class="source-line-no">2189</span><span id="line-2189"></span>
<span class="source-line-no">2190</span><span id="line-2190"> // if performing next balance exceeds cutoff time, exit the loop</span>
<span class="source-line-no">2191</span><span id="line-2191"> if (</span>
<span class="source-line-no">2192</span><span id="line-2192"> this.maxBalancingTime &gt; 0 &amp;&amp; rpCount &lt; plans.size()</span>
<span class="source-line-no">2193</span><span id="line-2193"> &amp;&amp; EnvironmentEdgeManager.currentTime() &gt; cutoffTime</span>
<span class="source-line-no">2194</span><span id="line-2194"> ) {</span>
<span class="source-line-no">2195</span><span id="line-2195"> // TODO: After balance, there should not be a cutoff time (keeping it as</span>
<span class="source-line-no">2196</span><span id="line-2196"> // a security net for now)</span>
<span class="source-line-no">2197</span><span id="line-2197"> LOG.debug(</span>
<span class="source-line-no">2198</span><span id="line-2198"> "No more balancing till next balance run; maxBalanceTime=" + this.maxBalancingTime);</span>
<span class="source-line-no">2199</span><span id="line-2199"> break;</span>
<span class="source-line-no">2200</span><span id="line-2200"> }</span>
<span class="source-line-no">2201</span><span id="line-2201"> }</span>
<span class="source-line-no">2202</span><span id="line-2202"> }</span>
<span class="source-line-no">2203</span><span id="line-2203"> LOG.debug("Balancer is going into sleep until next period in {}ms", getConfiguration()</span>
<span class="source-line-no">2204</span><span id="line-2204"> .getInt(HConstants.HBASE_BALANCER_PERIOD, HConstants.DEFAULT_HBASE_BALANCER_PERIOD));</span>
<span class="source-line-no">2205</span><span id="line-2205"> return successRegionPlans;</span>
<span class="source-line-no">2206</span><span id="line-2206"> }</span>
<span class="source-line-no">2207</span><span id="line-2207"></span>
<span class="source-line-no">2208</span><span id="line-2208"> @Override</span>
<span class="source-line-no">2209</span><span id="line-2209"> public RegionNormalizerManager getRegionNormalizerManager() {</span>
<span class="source-line-no">2210</span><span id="line-2210"> return regionNormalizerManager;</span>
<span class="source-line-no">2211</span><span id="line-2211"> }</span>
<span class="source-line-no">2212</span><span id="line-2212"></span>
<span class="source-line-no">2213</span><span id="line-2213"> @Override</span>
<span class="source-line-no">2214</span><span id="line-2214"> public boolean normalizeRegions(final NormalizeTableFilterParams ntfp,</span>
<span class="source-line-no">2215</span><span id="line-2215"> final boolean isHighPriority) throws IOException {</span>
<span class="source-line-no">2216</span><span id="line-2216"> if (regionNormalizerManager == null || !regionNormalizerManager.isNormalizerOn()) {</span>
<span class="source-line-no">2217</span><span id="line-2217"> LOG.debug("Region normalization is disabled, don't run region normalizer.");</span>
<span class="source-line-no">2218</span><span id="line-2218"> return false;</span>
<span class="source-line-no">2219</span><span id="line-2219"> }</span>
<span class="source-line-no">2220</span><span id="line-2220"> if (skipRegionManagementAction("region normalizer")) {</span>
<span class="source-line-no">2221</span><span id="line-2221"> return false;</span>
<span class="source-line-no">2222</span><span id="line-2222"> }</span>
<span class="source-line-no">2223</span><span id="line-2223"> if (assignmentManager.hasRegionsInTransition()) {</span>
<span class="source-line-no">2224</span><span id="line-2224"> return false;</span>
<span class="source-line-no">2225</span><span id="line-2225"> }</span>
<span class="source-line-no">2226</span><span id="line-2226"></span>
<span class="source-line-no">2227</span><span id="line-2227"> final Set&lt;TableName&gt; matchingTables = getTableDescriptors(new LinkedList&lt;&gt;(),</span>
<span class="source-line-no">2228</span><span id="line-2228"> ntfp.getNamespace(), ntfp.getRegex(), ntfp.getTableNames(), false).stream()</span>
<span class="source-line-no">2229</span><span id="line-2229"> .map(TableDescriptor::getTableName).collect(Collectors.toSet());</span>
<span class="source-line-no">2230</span><span id="line-2230"> final Set&lt;TableName&gt; allEnabledTables =</span>
<span class="source-line-no">2231</span><span id="line-2231"> tableStateManager.getTablesInStates(TableState.State.ENABLED);</span>
<span class="source-line-no">2232</span><span id="line-2232"> final List&lt;TableName&gt; targetTables =</span>
<span class="source-line-no">2233</span><span id="line-2233"> new ArrayList&lt;&gt;(Sets.intersection(matchingTables, allEnabledTables));</span>
<span class="source-line-no">2234</span><span id="line-2234"> Collections.shuffle(targetTables);</span>
<span class="source-line-no">2235</span><span id="line-2235"> return regionNormalizerManager.normalizeRegions(targetTables, isHighPriority);</span>
<span class="source-line-no">2236</span><span id="line-2236"> }</span>
<span class="source-line-no">2237</span><span id="line-2237"></span>
<span class="source-line-no">2238</span><span id="line-2238"> /** Returns Client info for use as prefix on an audit log string; who did an action */</span>
<span class="source-line-no">2239</span><span id="line-2239"> @Override</span>
<span class="source-line-no">2240</span><span id="line-2240"> public String getClientIdAuditPrefix() {</span>
<span class="source-line-no">2241</span><span id="line-2241"> return "Client=" + RpcServer.getRequestUserName().orElse(null) + "/"</span>
<span class="source-line-no">2242</span><span id="line-2242"> + RpcServer.getRemoteAddress().orElse(null);</span>
<span class="source-line-no">2243</span><span id="line-2243"> }</span>
<span class="source-line-no">2244</span><span id="line-2244"></span>
<span class="source-line-no">2245</span><span id="line-2245"> /**</span>
<span class="source-line-no">2246</span><span id="line-2246"> * Switch for the background CatalogJanitor thread. Used for testing. The thread will continue to</span>
<span class="source-line-no">2247</span><span id="line-2247"> * run. It will just be a noop if disabled.</span>
<span class="source-line-no">2248</span><span id="line-2248"> * @param b If false, the catalog janitor won't do anything.</span>
<span class="source-line-no">2249</span><span id="line-2249"> */</span>
<span class="source-line-no">2250</span><span id="line-2250"> public void setCatalogJanitorEnabled(final boolean b) {</span>
<span class="source-line-no">2251</span><span id="line-2251"> this.catalogJanitorChore.setEnabled(b);</span>
<span class="source-line-no">2252</span><span id="line-2252"> }</span>
<span class="source-line-no">2253</span><span id="line-2253"></span>
<span class="source-line-no">2254</span><span id="line-2254"> @Override</span>
<span class="source-line-no">2255</span><span id="line-2255"> public long mergeRegions(final RegionInfo[] regionsToMerge, final boolean forcible, final long ng,</span>
<span class="source-line-no">2256</span><span id="line-2256"> final long nonce) throws IOException {</span>
<span class="source-line-no">2257</span><span id="line-2257"> checkInitialized();</span>
<span class="source-line-no">2258</span><span id="line-2258"></span>
<span class="source-line-no">2259</span><span id="line-2259"> final String regionNamesToLog = RegionInfo.getShortNameToLog(regionsToMerge);</span>
<span class="source-line-no">2260</span><span id="line-2260"></span>
<span class="source-line-no">2261</span><span id="line-2261"> if (!isSplitOrMergeEnabled(MasterSwitchType.MERGE)) {</span>
<span class="source-line-no">2262</span><span id="line-2262"> LOG.warn("Merge switch is off! skip merge of " + regionNamesToLog);</span>
<span class="source-line-no">2263</span><span id="line-2263"> throw new DoNotRetryIOException(</span>
<span class="source-line-no">2264</span><span id="line-2264"> "Merge of " + regionNamesToLog + " failed because merge switch is off");</span>
<span class="source-line-no">2265</span><span id="line-2265"> }</span>
<span class="source-line-no">2266</span><span id="line-2266"></span>
<span class="source-line-no">2267</span><span id="line-2267"> if (!getTableDescriptors().get(regionsToMerge[0].getTable()).isMergeEnabled()) {</span>
<span class="source-line-no">2268</span><span id="line-2268"> LOG.warn("Merge is disabled for the table! Skipping merge of {}", regionNamesToLog);</span>
<span class="source-line-no">2269</span><span id="line-2269"> throw new DoNotRetryIOException(</span>
<span class="source-line-no">2270</span><span id="line-2270"> "Merge of " + regionNamesToLog + " failed as region merge is disabled for the table");</span>
<span class="source-line-no">2271</span><span id="line-2271"> }</span>
<span class="source-line-no">2272</span><span id="line-2272"></span>
<span class="source-line-no">2273</span><span id="line-2273"> return MasterProcedureUtil.submitProcedure(new NonceProcedureRunnable(this, ng, nonce) {</span>
<span class="source-line-no">2274</span><span id="line-2274"> @Override</span>
<span class="source-line-no">2275</span><span id="line-2275"> protected void run() throws IOException {</span>
<span class="source-line-no">2276</span><span id="line-2276"> getMaster().getMasterCoprocessorHost().preMergeRegions(regionsToMerge);</span>
<span class="source-line-no">2277</span><span id="line-2277"> String aid = getClientIdAuditPrefix();</span>
<span class="source-line-no">2278</span><span id="line-2278"> LOG.info("{} merge regions {}", aid, regionNamesToLog);</span>
<span class="source-line-no">2279</span><span id="line-2279"> submitProcedure(new MergeTableRegionsProcedure(procedureExecutor.getEnvironment(),</span>
<span class="source-line-no">2280</span><span id="line-2280"> regionsToMerge, forcible));</span>
<span class="source-line-no">2281</span><span id="line-2281"> getMaster().getMasterCoprocessorHost().postMergeRegions(regionsToMerge);</span>
<span class="source-line-no">2282</span><span id="line-2282"> }</span>
<span class="source-line-no">2283</span><span id="line-2283"></span>
<span class="source-line-no">2284</span><span id="line-2284"> @Override</span>
<span class="source-line-no">2285</span><span id="line-2285"> protected String getDescription() {</span>
<span class="source-line-no">2286</span><span id="line-2286"> return "MergeTableProcedure";</span>
<span class="source-line-no">2287</span><span id="line-2287"> }</span>
<span class="source-line-no">2288</span><span id="line-2288"> });</span>
<span class="source-line-no">2289</span><span id="line-2289"> }</span>
<span class="source-line-no">2290</span><span id="line-2290"></span>
<span class="source-line-no">2291</span><span id="line-2291"> @Override</span>
<span class="source-line-no">2292</span><span id="line-2292"> public long splitRegion(final RegionInfo regionInfo, final byte[] splitRow, final long nonceGroup,</span>
<span class="source-line-no">2293</span><span id="line-2293"> final long nonce) throws IOException {</span>
<span class="source-line-no">2294</span><span id="line-2294"> checkInitialized();</span>
<span class="source-line-no">2295</span><span id="line-2295"></span>
<span class="source-line-no">2296</span><span id="line-2296"> if (!isSplitOrMergeEnabled(MasterSwitchType.SPLIT)) {</span>
<span class="source-line-no">2297</span><span id="line-2297"> LOG.warn("Split switch is off! skip split of " + regionInfo);</span>
<span class="source-line-no">2298</span><span id="line-2298"> throw new DoNotRetryIOException(</span>
<span class="source-line-no">2299</span><span id="line-2299"> "Split region " + regionInfo.getRegionNameAsString() + " failed due to split switch off");</span>
<span class="source-line-no">2300</span><span id="line-2300"> }</span>
<span class="source-line-no">2301</span><span id="line-2301"></span>
<span class="source-line-no">2302</span><span id="line-2302"> if (!getTableDescriptors().get(regionInfo.getTable()).isSplitEnabled()) {</span>
<span class="source-line-no">2303</span><span id="line-2303"> LOG.warn("Split is disabled for the table! Skipping split of {}", regionInfo);</span>
<span class="source-line-no">2304</span><span id="line-2304"> throw new DoNotRetryIOException("Split region " + regionInfo.getRegionNameAsString()</span>
<span class="source-line-no">2305</span><span id="line-2305"> + " failed as region split is disabled for the table");</span>
<span class="source-line-no">2306</span><span id="line-2306"> }</span>
<span class="source-line-no">2307</span><span id="line-2307"></span>
<span class="source-line-no">2308</span><span id="line-2308"> return MasterProcedureUtil</span>
<span class="source-line-no">2309</span><span id="line-2309"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">2310</span><span id="line-2310"> @Override</span>
<span class="source-line-no">2311</span><span id="line-2311"> protected void run() throws IOException {</span>
<span class="source-line-no">2312</span><span id="line-2312"> getMaster().getMasterCoprocessorHost().preSplitRegion(regionInfo.getTable(), splitRow);</span>
<span class="source-line-no">2313</span><span id="line-2313"> LOG.info(getClientIdAuditPrefix() + " split " + regionInfo.getRegionNameAsString());</span>
<span class="source-line-no">2314</span><span id="line-2314"></span>
<span class="source-line-no">2315</span><span id="line-2315"> // Execute the operation asynchronously</span>
<span class="source-line-no">2316</span><span id="line-2316"> submitProcedure(getAssignmentManager().createSplitProcedure(regionInfo, splitRow));</span>
<span class="source-line-no">2317</span><span id="line-2317"> }</span>
<span class="source-line-no">2318</span><span id="line-2318"></span>
<span class="source-line-no">2319</span><span id="line-2319"> @Override</span>
<span class="source-line-no">2320</span><span id="line-2320"> protected String getDescription() {</span>
<span class="source-line-no">2321</span><span id="line-2321"> return "SplitTableProcedure";</span>
<span class="source-line-no">2322</span><span id="line-2322"> }</span>
<span class="source-line-no">2323</span><span id="line-2323"> });</span>
<span class="source-line-no">2324</span><span id="line-2324"> }</span>
<span class="source-line-no">2325</span><span id="line-2325"></span>
<span class="source-line-no">2326</span><span id="line-2326"> private void warmUpRegion(ServerName server, RegionInfo region) {</span>
<span class="source-line-no">2327</span><span id="line-2327"> FutureUtils.addListener(asyncClusterConnection.getRegionServerAdmin(server)</span>
<span class="source-line-no">2328</span><span id="line-2328"> .warmupRegion(RequestConverter.buildWarmupRegionRequest(region)), (r, e) -&gt; {</span>
<span class="source-line-no">2329</span><span id="line-2329"> if (e != null) {</span>
<span class="source-line-no">2330</span><span id="line-2330"> LOG.warn("Failed to warm up region {} on server {}", region, server, e);</span>
<span class="source-line-no">2331</span><span id="line-2331"> }</span>
<span class="source-line-no">2332</span><span id="line-2332"> });</span>
<span class="source-line-no">2333</span><span id="line-2333"> }</span>
<span class="source-line-no">2334</span><span id="line-2334"></span>
<span class="source-line-no">2335</span><span id="line-2335"> // Public so can be accessed by tests. Blocks until move is done.</span>
<span class="source-line-no">2336</span><span id="line-2336"> // Replace with an async implementation from which you can get</span>
<span class="source-line-no">2337</span><span id="line-2337"> // a success/failure result.</span>
<span class="source-line-no">2338</span><span id="line-2338"> @InterfaceAudience.Private</span>
<span class="source-line-no">2339</span><span id="line-2339"> public void move(final byte[] encodedRegionName, byte[] destServerName) throws IOException {</span>
<span class="source-line-no">2340</span><span id="line-2340"> RegionState regionState =</span>
<span class="source-line-no">2341</span><span id="line-2341"> assignmentManager.getRegionStates().getRegionState(Bytes.toString(encodedRegionName));</span>
<span class="source-line-no">2342</span><span id="line-2342"></span>
<span class="source-line-no">2343</span><span id="line-2343"> RegionInfo hri;</span>
<span class="source-line-no">2344</span><span id="line-2344"> if (regionState != null) {</span>
<span class="source-line-no">2345</span><span id="line-2345"> hri = regionState.getRegion();</span>
<span class="source-line-no">2346</span><span id="line-2346"> } else {</span>
<span class="source-line-no">2347</span><span id="line-2347"> throw new UnknownRegionException(Bytes.toStringBinary(encodedRegionName));</span>
<span class="source-line-no">2348</span><span id="line-2348"> }</span>
<span class="source-line-no">2349</span><span id="line-2349"></span>
<span class="source-line-no">2350</span><span id="line-2350"> ServerName dest;</span>
<span class="source-line-no">2351</span><span id="line-2351"> List&lt;ServerName&gt; exclude = hri.getTable().isSystemTable()</span>
<span class="source-line-no">2352</span><span id="line-2352"> ? assignmentManager.getExcludedServersForSystemTable()</span>
<span class="source-line-no">2353</span><span id="line-2353"> : new ArrayList&lt;&gt;(1);</span>
<span class="source-line-no">2354</span><span id="line-2354"> if (</span>
<span class="source-line-no">2355</span><span id="line-2355"> destServerName != null &amp;&amp; exclude.contains(ServerName.valueOf(Bytes.toString(destServerName)))</span>
<span class="source-line-no">2356</span><span id="line-2356"> ) {</span>
<span class="source-line-no">2357</span><span id="line-2357"> LOG.info(Bytes.toString(encodedRegionName) + " can not move to "</span>
<span class="source-line-no">2358</span><span id="line-2358"> + Bytes.toString(destServerName) + " because the server is in exclude list");</span>
<span class="source-line-no">2359</span><span id="line-2359"> destServerName = null;</span>
<span class="source-line-no">2360</span><span id="line-2360"> }</span>
<span class="source-line-no">2361</span><span id="line-2361"> if (destServerName == null || destServerName.length == 0) {</span>
<span class="source-line-no">2362</span><span id="line-2362"> LOG.info("Passed destination servername is null/empty so " + "choosing a server at random");</span>
<span class="source-line-no">2363</span><span id="line-2363"> exclude.add(regionState.getServerName());</span>
<span class="source-line-no">2364</span><span id="line-2364"> final List&lt;ServerName&gt; destServers = this.serverManager.createDestinationServersList(exclude);</span>
<span class="source-line-no">2365</span><span id="line-2365"> dest = balancer.randomAssignment(hri, destServers);</span>
<span class="source-line-no">2366</span><span id="line-2366"> if (dest == null) {</span>
<span class="source-line-no">2367</span><span id="line-2367"> LOG.debug("Unable to determine a plan to assign " + hri);</span>
<span class="source-line-no">2368</span><span id="line-2368"> return;</span>
<span class="source-line-no">2369</span><span id="line-2369"> }</span>
<span class="source-line-no">2370</span><span id="line-2370"> } else {</span>
<span class="source-line-no">2371</span><span id="line-2371"> ServerName candidate = ServerName.valueOf(Bytes.toString(destServerName));</span>
<span class="source-line-no">2372</span><span id="line-2372"> dest = balancer.randomAssignment(hri, Lists.newArrayList(candidate));</span>
<span class="source-line-no">2373</span><span id="line-2373"> if (dest == null) {</span>
<span class="source-line-no">2374</span><span id="line-2374"> LOG.debug("Unable to determine a plan to assign " + hri);</span>
<span class="source-line-no">2375</span><span id="line-2375"> return;</span>
<span class="source-line-no">2376</span><span id="line-2376"> }</span>
<span class="source-line-no">2377</span><span id="line-2377"> // TODO: deal with table on master for rs group.</span>
<span class="source-line-no">2378</span><span id="line-2378"> if (dest.equals(serverName)) {</span>
<span class="source-line-no">2379</span><span id="line-2379"> // To avoid unnecessary region moving later by balancer. Don't put user</span>
<span class="source-line-no">2380</span><span id="line-2380"> // regions on master.</span>
<span class="source-line-no">2381</span><span id="line-2381"> LOG.debug("Skipping move of region " + hri.getRegionNameAsString()</span>
<span class="source-line-no">2382</span><span id="line-2382"> + " to avoid unnecessary region moving later by load balancer,"</span>
<span class="source-line-no">2383</span><span id="line-2383"> + " because it should not be on master");</span>
<span class="source-line-no">2384</span><span id="line-2384"> return;</span>
<span class="source-line-no">2385</span><span id="line-2385"> }</span>
<span class="source-line-no">2386</span><span id="line-2386"> }</span>
<span class="source-line-no">2387</span><span id="line-2387"></span>
<span class="source-line-no">2388</span><span id="line-2388"> if (dest.equals(regionState.getServerName())) {</span>
<span class="source-line-no">2389</span><span id="line-2389"> LOG.debug("Skipping move of region " + hri.getRegionNameAsString()</span>
<span class="source-line-no">2390</span><span id="line-2390"> + " because region already assigned to the same server " + dest + ".");</span>
<span class="source-line-no">2391</span><span id="line-2391"> return;</span>
<span class="source-line-no">2392</span><span id="line-2392"> }</span>
<span class="source-line-no">2393</span><span id="line-2393"></span>
<span class="source-line-no">2394</span><span id="line-2394"> // Now we can do the move</span>
<span class="source-line-no">2395</span><span id="line-2395"> RegionPlan rp = new RegionPlan(hri, regionState.getServerName(), dest);</span>
<span class="source-line-no">2396</span><span id="line-2396"> assert rp.getDestination() != null : rp.toString() + " " + dest;</span>
<span class="source-line-no">2397</span><span id="line-2397"></span>
<span class="source-line-no">2398</span><span id="line-2398"> try {</span>
<span class="source-line-no">2399</span><span id="line-2399"> checkInitialized();</span>
<span class="source-line-no">2400</span><span id="line-2400"> if (this.cpHost != null) {</span>
<span class="source-line-no">2401</span><span id="line-2401"> this.cpHost.preMove(hri, rp.getSource(), rp.getDestination());</span>
<span class="source-line-no">2402</span><span id="line-2402"> }</span>
<span class="source-line-no">2403</span><span id="line-2403"></span>
<span class="source-line-no">2404</span><span id="line-2404"> TransitRegionStateProcedure proc =</span>
<span class="source-line-no">2405</span><span id="line-2405"> this.assignmentManager.createMoveRegionProcedure(rp.getRegionInfo(), rp.getDestination());</span>
<span class="source-line-no">2406</span><span id="line-2406"> if (conf.getBoolean(WARMUP_BEFORE_MOVE, DEFAULT_WARMUP_BEFORE_MOVE)) {</span>
<span class="source-line-no">2407</span><span id="line-2407"> // Warmup the region on the destination before initiating the move.</span>
<span class="source-line-no">2408</span><span id="line-2408"> // A region server could reject the close request because it either does not</span>
<span class="source-line-no">2409</span><span id="line-2409"> // have the specified region or the region is being split.</span>
<span class="source-line-no">2410</span><span id="line-2410"> LOG.info(getClientIdAuditPrefix() + " move " + rp + ", warming up region on "</span>
<span class="source-line-no">2411</span><span id="line-2411"> + rp.getDestination());</span>
<span class="source-line-no">2412</span><span id="line-2412"> warmUpRegion(rp.getDestination(), hri);</span>
<span class="source-line-no">2413</span><span id="line-2413"> }</span>
<span class="source-line-no">2414</span><span id="line-2414"> LOG.info(getClientIdAuditPrefix() + " move " + rp + ", running balancer");</span>
<span class="source-line-no">2415</span><span id="line-2415"> Future&lt;byte[]&gt; future = ProcedureSyncWait.submitProcedure(this.procedureExecutor, proc);</span>
<span class="source-line-no">2416</span><span id="line-2416"> try {</span>
<span class="source-line-no">2417</span><span id="line-2417"> // Is this going to work? Will we throw exception on error?</span>
<span class="source-line-no">2418</span><span id="line-2418"> // TODO: CompletableFuture rather than this stunted Future.</span>
<span class="source-line-no">2419</span><span id="line-2419"> future.get();</span>
<span class="source-line-no">2420</span><span id="line-2420"> } catch (InterruptedException | ExecutionException e) {</span>
<span class="source-line-no">2421</span><span id="line-2421"> throw new HBaseIOException(e);</span>
<span class="source-line-no">2422</span><span id="line-2422"> }</span>
<span class="source-line-no">2423</span><span id="line-2423"> if (this.cpHost != null) {</span>
<span class="source-line-no">2424</span><span id="line-2424"> this.cpHost.postMove(hri, rp.getSource(), rp.getDestination());</span>
<span class="source-line-no">2425</span><span id="line-2425"> }</span>
<span class="source-line-no">2426</span><span id="line-2426"> } catch (IOException ioe) {</span>
<span class="source-line-no">2427</span><span id="line-2427"> if (ioe instanceof HBaseIOException) {</span>
<span class="source-line-no">2428</span><span id="line-2428"> throw (HBaseIOException) ioe;</span>
<span class="source-line-no">2429</span><span id="line-2429"> }</span>
<span class="source-line-no">2430</span><span id="line-2430"> throw new HBaseIOException(ioe);</span>
<span class="source-line-no">2431</span><span id="line-2431"> }</span>
<span class="source-line-no">2432</span><span id="line-2432"> }</span>
<span class="source-line-no">2433</span><span id="line-2433"></span>
<span class="source-line-no">2434</span><span id="line-2434"> @Override</span>
<span class="source-line-no">2435</span><span id="line-2435"> public long createTable(final TableDescriptor tableDescriptor, final byte[][] splitKeys,</span>
<span class="source-line-no">2436</span><span id="line-2436"> final long nonceGroup, final long nonce) throws IOException {</span>
<span class="source-line-no">2437</span><span id="line-2437"> checkInitialized();</span>
<span class="source-line-no">2438</span><span id="line-2438"> TableDescriptor desc = getMasterCoprocessorHost().preCreateTableRegionsInfos(tableDescriptor);</span>
<span class="source-line-no">2439</span><span id="line-2439"> if (desc == null) {</span>
<span class="source-line-no">2440</span><span id="line-2440"> throw new IOException("Creation for " + tableDescriptor + " is canceled by CP");</span>
<span class="source-line-no">2441</span><span id="line-2441"> }</span>
<span class="source-line-no">2442</span><span id="line-2442"> String namespace = desc.getTableName().getNamespaceAsString();</span>
<span class="source-line-no">2443</span><span id="line-2443"> this.clusterSchemaService.getNamespace(namespace);</span>
<span class="source-line-no">2444</span><span id="line-2444"></span>
<span class="source-line-no">2445</span><span id="line-2445"> RegionInfo[] newRegions = ModifyRegionUtils.createRegionInfos(desc, splitKeys);</span>
<span class="source-line-no">2446</span><span id="line-2446"> TableDescriptorChecker.sanityCheck(conf, desc);</span>
<span class="source-line-no">2447</span><span id="line-2447"></span>
<span class="source-line-no">2448</span><span id="line-2448"> return MasterProcedureUtil</span>
<span class="source-line-no">2449</span><span id="line-2449"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">2450</span><span id="line-2450"> @Override</span>
<span class="source-line-no">2451</span><span id="line-2451"> protected void run() throws IOException {</span>
<span class="source-line-no">2452</span><span id="line-2452"> getMaster().getMasterCoprocessorHost().preCreateTable(desc, newRegions);</span>
<span class="source-line-no">2453</span><span id="line-2453"></span>
<span class="source-line-no">2454</span><span id="line-2454"> LOG.info(getClientIdAuditPrefix() + " create " + desc);</span>
<span class="source-line-no">2455</span><span id="line-2455"></span>
<span class="source-line-no">2456</span><span id="line-2456"> // TODO: We can handle/merge duplicate requests, and differentiate the case of</span>
<span class="source-line-no">2457</span><span id="line-2457"> // TableExistsException by saying if the schema is the same or not.</span>
<span class="source-line-no">2458</span><span id="line-2458"> //</span>
<span class="source-line-no">2459</span><span id="line-2459"> // We need to wait for the procedure to potentially fail due to "prepare" sanity</span>
<span class="source-line-no">2460</span><span id="line-2460"> // checks. This will block only the beginning of the procedure. See HBASE-19953.</span>
<span class="source-line-no">2461</span><span id="line-2461"> ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();</span>
<span class="source-line-no">2462</span><span id="line-2462"> submitProcedure(</span>
<span class="source-line-no">2463</span><span id="line-2463"> new CreateTableProcedure(procedureExecutor.getEnvironment(), desc, newRegions, latch));</span>
<span class="source-line-no">2464</span><span id="line-2464"> latch.await();</span>
<span class="source-line-no">2465</span><span id="line-2465"></span>
<span class="source-line-no">2466</span><span id="line-2466"> getMaster().getMasterCoprocessorHost().postCreateTable(desc, newRegions);</span>
<span class="source-line-no">2467</span><span id="line-2467"> }</span>
<span class="source-line-no">2468</span><span id="line-2468"></span>
<span class="source-line-no">2469</span><span id="line-2469"> @Override</span>
<span class="source-line-no">2470</span><span id="line-2470"> protected String getDescription() {</span>
<span class="source-line-no">2471</span><span id="line-2471"> return "CreateTableProcedure";</span>
<span class="source-line-no">2472</span><span id="line-2472"> }</span>
<span class="source-line-no">2473</span><span id="line-2473"> });</span>
<span class="source-line-no">2474</span><span id="line-2474"> }</span>
<span class="source-line-no">2475</span><span id="line-2475"></span>
<span class="source-line-no">2476</span><span id="line-2476"> @Override</span>
<span class="source-line-no">2477</span><span id="line-2477"> public long createSystemTable(final TableDescriptor tableDescriptor) throws IOException {</span>
<span class="source-line-no">2478</span><span id="line-2478"> if (isStopped()) {</span>
<span class="source-line-no">2479</span><span id="line-2479"> throw new MasterNotRunningException();</span>
<span class="source-line-no">2480</span><span id="line-2480"> }</span>
<span class="source-line-no">2481</span><span id="line-2481"></span>
<span class="source-line-no">2482</span><span id="line-2482"> TableName tableName = tableDescriptor.getTableName();</span>
<span class="source-line-no">2483</span><span id="line-2483"> if (!(tableName.isSystemTable())) {</span>
<span class="source-line-no">2484</span><span id="line-2484"> throw new IllegalArgumentException(</span>
<span class="source-line-no">2485</span><span id="line-2485"> "Only system table creation can use this createSystemTable API");</span>
<span class="source-line-no">2486</span><span id="line-2486"> }</span>
<span class="source-line-no">2487</span><span id="line-2487"></span>
<span class="source-line-no">2488</span><span id="line-2488"> RegionInfo[] newRegions = ModifyRegionUtils.createRegionInfos(tableDescriptor, null);</span>
<span class="source-line-no">2489</span><span id="line-2489"></span>
<span class="source-line-no">2490</span><span id="line-2490"> LOG.info(getClientIdAuditPrefix() + " create " + tableDescriptor);</span>
<span class="source-line-no">2491</span><span id="line-2491"></span>
<span class="source-line-no">2492</span><span id="line-2492"> // This special create table is called locally to master. Therefore, no RPC means no need</span>
<span class="source-line-no">2493</span><span id="line-2493"> // to use nonce to detect duplicated RPC call.</span>
<span class="source-line-no">2494</span><span id="line-2494"> long procId = this.procedureExecutor.submitProcedure(</span>
<span class="source-line-no">2495</span><span id="line-2495"> new CreateTableProcedure(procedureExecutor.getEnvironment(), tableDescriptor, newRegions));</span>
<span class="source-line-no">2496</span><span id="line-2496"></span>
<span class="source-line-no">2497</span><span id="line-2497"> return procId;</span>
<span class="source-line-no">2498</span><span id="line-2498"> }</span>
<span class="source-line-no">2499</span><span id="line-2499"></span>
<span class="source-line-no">2500</span><span id="line-2500"> private void startActiveMasterManager(int infoPort) throws KeeperException {</span>
<span class="source-line-no">2501</span><span id="line-2501"> String backupZNode = ZNodePaths.joinZNode(zooKeeper.getZNodePaths().backupMasterAddressesZNode,</span>
<span class="source-line-no">2502</span><span id="line-2502"> serverName.toString());</span>
<span class="source-line-no">2503</span><span id="line-2503"> /*</span>
<span class="source-line-no">2504</span><span id="line-2504"> * Add a ZNode for ourselves in the backup master directory since we may not become the active</span>
<span class="source-line-no">2505</span><span id="line-2505"> * master. If so, we want the actual active master to know we are backup masters, so that it</span>
<span class="source-line-no">2506</span><span id="line-2506"> * won't assign regions to us if so configured. If we become the active master later,</span>
<span class="source-line-no">2507</span><span id="line-2507"> * ActiveMasterManager will delete this node explicitly. If we crash before then, ZooKeeper will</span>
<span class="source-line-no">2508</span><span id="line-2508"> * delete this node for us since it is ephemeral.</span>
<span class="source-line-no">2509</span><span id="line-2509"> */</span>
<span class="source-line-no">2510</span><span id="line-2510"> LOG.info("Adding backup master ZNode " + backupZNode);</span>
<span class="source-line-no">2511</span><span id="line-2511"> if (!MasterAddressTracker.setMasterAddress(zooKeeper, backupZNode, serverName, infoPort)) {</span>
<span class="source-line-no">2512</span><span id="line-2512"> LOG.warn("Failed create of " + backupZNode + " by " + serverName);</span>
<span class="source-line-no">2513</span><span id="line-2513"> }</span>
<span class="source-line-no">2514</span><span id="line-2514"> this.activeMasterManager.setInfoPort(infoPort);</span>
<span class="source-line-no">2515</span><span id="line-2515"> int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);</span>
<span class="source-line-no">2516</span><span id="line-2516"> // If we're a backup master, stall until a primary to write this address</span>
<span class="source-line-no">2517</span><span id="line-2517"> if (conf.getBoolean(HConstants.MASTER_TYPE_BACKUP, HConstants.DEFAULT_MASTER_TYPE_BACKUP)) {</span>
<span class="source-line-no">2518</span><span id="line-2518"> LOG.debug("HMaster started in backup mode. Stalling until master znode is written.");</span>
<span class="source-line-no">2519</span><span id="line-2519"> // This will only be a minute or so while the cluster starts up,</span>
<span class="source-line-no">2520</span><span id="line-2520"> // so don't worry about setting watches on the parent znode</span>
<span class="source-line-no">2521</span><span id="line-2521"> while (!activeMasterManager.hasActiveMaster()) {</span>
<span class="source-line-no">2522</span><span id="line-2522"> LOG.debug("Waiting for master address and cluster state znode to be written.");</span>
<span class="source-line-no">2523</span><span id="line-2523"> Threads.sleep(timeout);</span>
<span class="source-line-no">2524</span><span id="line-2524"> }</span>
<span class="source-line-no">2525</span><span id="line-2525"> }</span>
<span class="source-line-no">2526</span><span id="line-2526"></span>
<span class="source-line-no">2527</span><span id="line-2527"> // Here for the master startup process, we use TaskGroup to monitor the whole progress.</span>
<span class="source-line-no">2528</span><span id="line-2528"> // The UI is similar to how Hadoop designed the startup page for the NameNode.</span>
<span class="source-line-no">2529</span><span id="line-2529"> // See HBASE-21521 for more details.</span>
<span class="source-line-no">2530</span><span id="line-2530"> // We do not cleanup the startupTaskGroup, let the startup progress information</span>
<span class="source-line-no">2531</span><span id="line-2531"> // be permanent in the MEM.</span>
<span class="source-line-no">2532</span><span id="line-2532"> startupTaskGroup = TaskMonitor.createTaskGroup(true, "Master startup");</span>
<span class="source-line-no">2533</span><span id="line-2533"> try {</span>
<span class="source-line-no">2534</span><span id="line-2534"> if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, startupTaskGroup)) {</span>
<span class="source-line-no">2535</span><span id="line-2535"> finishActiveMasterInitialization();</span>
<span class="source-line-no">2536</span><span id="line-2536"> }</span>
<span class="source-line-no">2537</span><span id="line-2537"> } catch (Throwable t) {</span>
<span class="source-line-no">2538</span><span id="line-2538"> startupTaskGroup.abort("Failed to become active master due to:" + t.getMessage());</span>
<span class="source-line-no">2539</span><span id="line-2539"> LOG.error(HBaseMarkers.FATAL, "Failed to become active master", t);</span>
<span class="source-line-no">2540</span><span id="line-2540"> // HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility</span>
<span class="source-line-no">2541</span><span id="line-2541"> if (</span>
<span class="source-line-no">2542</span><span id="line-2542"> t instanceof NoClassDefFoundError</span>
<span class="source-line-no">2543</span><span id="line-2543"> &amp;&amp; t.getMessage().contains("org/apache/hadoop/hdfs/protocol/HdfsConstants$SafeModeAction")</span>
<span class="source-line-no">2544</span><span id="line-2544"> ) {</span>
<span class="source-line-no">2545</span><span id="line-2545"> // improved error message for this special case</span>
<span class="source-line-no">2546</span><span id="line-2546"> abort("HBase is having a problem with its Hadoop jars. You may need to recompile "</span>
<span class="source-line-no">2547</span><span id="line-2547"> + "HBase against Hadoop version " + org.apache.hadoop.util.VersionInfo.getVersion()</span>
<span class="source-line-no">2548</span><span id="line-2548"> + " or change your hadoop jars to start properly", t);</span>
<span class="source-line-no">2549</span><span id="line-2549"> } else {</span>
<span class="source-line-no">2550</span><span id="line-2550"> abort("Unhandled exception. Starting shutdown.", t);</span>
<span class="source-line-no">2551</span><span id="line-2551"> }</span>
<span class="source-line-no">2552</span><span id="line-2552"> }</span>
<span class="source-line-no">2553</span><span id="line-2553"> }</span>
<span class="source-line-no">2554</span><span id="line-2554"></span>
<span class="source-line-no">2555</span><span id="line-2555"> private static boolean isCatalogTable(final TableName tableName) {</span>
<span class="source-line-no">2556</span><span id="line-2556"> return tableName.equals(TableName.META_TABLE_NAME);</span>
<span class="source-line-no">2557</span><span id="line-2557"> }</span>
<span class="source-line-no">2558</span><span id="line-2558"></span>
<span class="source-line-no">2559</span><span id="line-2559"> @Override</span>
<span class="source-line-no">2560</span><span id="line-2560"> public long deleteTable(final TableName tableName, final long nonceGroup, final long nonce)</span>
<span class="source-line-no">2561</span><span id="line-2561"> throws IOException {</span>
<span class="source-line-no">2562</span><span id="line-2562"> checkInitialized();</span>
<span class="source-line-no">2563</span><span id="line-2563"></span>
<span class="source-line-no">2564</span><span id="line-2564"> return MasterProcedureUtil</span>
<span class="source-line-no">2565</span><span id="line-2565"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">2566</span><span id="line-2566"> @Override</span>
<span class="source-line-no">2567</span><span id="line-2567"> protected void run() throws IOException {</span>
<span class="source-line-no">2568</span><span id="line-2568"> getMaster().getMasterCoprocessorHost().preDeleteTable(tableName);</span>
<span class="source-line-no">2569</span><span id="line-2569"></span>
<span class="source-line-no">2570</span><span id="line-2570"> LOG.info(getClientIdAuditPrefix() + " delete " + tableName);</span>
<span class="source-line-no">2571</span><span id="line-2571"></span>
<span class="source-line-no">2572</span><span id="line-2572"> // TODO: We can handle/merge duplicate request</span>
<span class="source-line-no">2573</span><span id="line-2573"> //</span>
<span class="source-line-no">2574</span><span id="line-2574"> // We need to wait for the procedure to potentially fail due to "prepare" sanity</span>
<span class="source-line-no">2575</span><span id="line-2575"> // checks. This will block only the beginning of the procedure. See HBASE-19953.</span>
<span class="source-line-no">2576</span><span id="line-2576"> ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();</span>
<span class="source-line-no">2577</span><span id="line-2577"> submitProcedure(</span>
<span class="source-line-no">2578</span><span id="line-2578"> new DeleteTableProcedure(procedureExecutor.getEnvironment(), tableName, latch));</span>
<span class="source-line-no">2579</span><span id="line-2579"> latch.await();</span>
<span class="source-line-no">2580</span><span id="line-2580"></span>
<span class="source-line-no">2581</span><span id="line-2581"> getMaster().getMasterCoprocessorHost().postDeleteTable(tableName);</span>
<span class="source-line-no">2582</span><span id="line-2582"> }</span>
<span class="source-line-no">2583</span><span id="line-2583"></span>
<span class="source-line-no">2584</span><span id="line-2584"> @Override</span>
<span class="source-line-no">2585</span><span id="line-2585"> protected String getDescription() {</span>
<span class="source-line-no">2586</span><span id="line-2586"> return "DeleteTableProcedure";</span>
<span class="source-line-no">2587</span><span id="line-2587"> }</span>
<span class="source-line-no">2588</span><span id="line-2588"> });</span>
<span class="source-line-no">2589</span><span id="line-2589"> }</span>
<span class="source-line-no">2590</span><span id="line-2590"></span>
<span class="source-line-no">2591</span><span id="line-2591"> @Override</span>
<span class="source-line-no">2592</span><span id="line-2592"> public long truncateTable(final TableName tableName, final boolean preserveSplits,</span>
<span class="source-line-no">2593</span><span id="line-2593"> final long nonceGroup, final long nonce) throws IOException {</span>
<span class="source-line-no">2594</span><span id="line-2594"> checkInitialized();</span>
<span class="source-line-no">2595</span><span id="line-2595"></span>
<span class="source-line-no">2596</span><span id="line-2596"> return MasterProcedureUtil</span>
<span class="source-line-no">2597</span><span id="line-2597"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">2598</span><span id="line-2598"> @Override</span>
<span class="source-line-no">2599</span><span id="line-2599"> protected void run() throws IOException {</span>
<span class="source-line-no">2600</span><span id="line-2600"> getMaster().getMasterCoprocessorHost().preTruncateTable(tableName);</span>
<span class="source-line-no">2601</span><span id="line-2601"></span>
<span class="source-line-no">2602</span><span id="line-2602"> LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);</span>
<span class="source-line-no">2603</span><span id="line-2603"> ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);</span>
<span class="source-line-no">2604</span><span id="line-2604"> submitProcedure(new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName,</span>
<span class="source-line-no">2605</span><span id="line-2605"> preserveSplits, latch));</span>
<span class="source-line-no">2606</span><span id="line-2606"> latch.await();</span>
<span class="source-line-no">2607</span><span id="line-2607"></span>
<span class="source-line-no">2608</span><span id="line-2608"> getMaster().getMasterCoprocessorHost().postTruncateTable(tableName);</span>
<span class="source-line-no">2609</span><span id="line-2609"> }</span>
<span class="source-line-no">2610</span><span id="line-2610"></span>
<span class="source-line-no">2611</span><span id="line-2611"> @Override</span>
<span class="source-line-no">2612</span><span id="line-2612"> protected String getDescription() {</span>
<span class="source-line-no">2613</span><span id="line-2613"> return "TruncateTableProcedure";</span>
<span class="source-line-no">2614</span><span id="line-2614"> }</span>
<span class="source-line-no">2615</span><span id="line-2615"> });</span>
<span class="source-line-no">2616</span><span id="line-2616"> }</span>
<span class="source-line-no">2617</span><span id="line-2617"></span>
<span class="source-line-no">2618</span><span id="line-2618"> @Override</span>
<span class="source-line-no">2619</span><span id="line-2619"> public long truncateRegion(final RegionInfo regionInfo, final long nonceGroup, final long nonce)</span>
<span class="source-line-no">2620</span><span id="line-2620"> throws IOException {</span>
<span class="source-line-no">2621</span><span id="line-2621"> checkInitialized();</span>
<span class="source-line-no">2622</span><span id="line-2622"></span>
<span class="source-line-no">2623</span><span id="line-2623"> return MasterProcedureUtil</span>
<span class="source-line-no">2624</span><span id="line-2624"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">2625</span><span id="line-2625"> @Override</span>
<span class="source-line-no">2626</span><span id="line-2626"> protected void run() throws IOException {</span>
<span class="source-line-no">2627</span><span id="line-2627"> getMaster().getMasterCoprocessorHost().preTruncateRegion(regionInfo);</span>
<span class="source-line-no">2628</span><span id="line-2628"></span>
<span class="source-line-no">2629</span><span id="line-2629"> LOG.info(</span>
<span class="source-line-no">2630</span><span id="line-2630"> getClientIdAuditPrefix() + " truncate region " + regionInfo.getRegionNameAsString());</span>
<span class="source-line-no">2631</span><span id="line-2631"></span>
<span class="source-line-no">2632</span><span id="line-2632"> // Execute the operation asynchronously</span>
<span class="source-line-no">2633</span><span id="line-2633"> ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);</span>
<span class="source-line-no">2634</span><span id="line-2634"> submitProcedure(</span>
<span class="source-line-no">2635</span><span id="line-2635"> new TruncateRegionProcedure(procedureExecutor.getEnvironment(), regionInfo, latch));</span>
<span class="source-line-no">2636</span><span id="line-2636"> latch.await();</span>
<span class="source-line-no">2637</span><span id="line-2637"></span>
<span class="source-line-no">2638</span><span id="line-2638"> getMaster().getMasterCoprocessorHost().postTruncateRegion(regionInfo);</span>
<span class="source-line-no">2639</span><span id="line-2639"> }</span>
<span class="source-line-no">2640</span><span id="line-2640"></span>
<span class="source-line-no">2641</span><span id="line-2641"> @Override</span>
<span class="source-line-no">2642</span><span id="line-2642"> protected String getDescription() {</span>
<span class="source-line-no">2643</span><span id="line-2643"> return "TruncateRegionProcedure";</span>
<span class="source-line-no">2644</span><span id="line-2644"> }</span>
<span class="source-line-no">2645</span><span id="line-2645"> });</span>
<span class="source-line-no">2646</span><span id="line-2646"> }</span>
<span class="source-line-no">2647</span><span id="line-2647"></span>
<span class="source-line-no">2648</span><span id="line-2648"> @Override</span>
<span class="source-line-no">2649</span><span id="line-2649"> public long addColumn(final TableName tableName, final ColumnFamilyDescriptor column,</span>
<span class="source-line-no">2650</span><span id="line-2650"> final long nonceGroup, final long nonce) throws IOException {</span>
<span class="source-line-no">2651</span><span id="line-2651"> checkInitialized();</span>
<span class="source-line-no">2652</span><span id="line-2652"> checkTableExists(tableName);</span>
<span class="source-line-no">2653</span><span id="line-2653"></span>
<span class="source-line-no">2654</span><span id="line-2654"> return modifyTable(tableName, new TableDescriptorGetter() {</span>
<span class="source-line-no">2655</span><span id="line-2655"></span>
<span class="source-line-no">2656</span><span id="line-2656"> @Override</span>
<span class="source-line-no">2657</span><span id="line-2657"> public TableDescriptor get() throws IOException {</span>
<span class="source-line-no">2658</span><span id="line-2658"> TableDescriptor old = getTableDescriptors().get(tableName);</span>
<span class="source-line-no">2659</span><span id="line-2659"> if (old.hasColumnFamily(column.getName())) {</span>
<span class="source-line-no">2660</span><span id="line-2660"> throw new InvalidFamilyOperationException("Column family '" + column.getNameAsString()</span>
<span class="source-line-no">2661</span><span id="line-2661"> + "' in table '" + tableName + "' already exists so cannot be added");</span>
<span class="source-line-no">2662</span><span id="line-2662"> }</span>
<span class="source-line-no">2663</span><span id="line-2663"></span>
<span class="source-line-no">2664</span><span id="line-2664"> return TableDescriptorBuilder.newBuilder(old).setColumnFamily(column).build();</span>
<span class="source-line-no">2665</span><span id="line-2665"> }</span>
<span class="source-line-no">2666</span><span id="line-2666"> }, nonceGroup, nonce, true);</span>
<span class="source-line-no">2667</span><span id="line-2667"> }</span>
<span class="source-line-no">2668</span><span id="line-2668"></span>
<span class="source-line-no">2669</span><span id="line-2669"> /**</span>
<span class="source-line-no">2670</span><span id="line-2670"> * Implement to return TableDescriptor after pre-checks</span>
<span class="source-line-no">2671</span><span id="line-2671"> */</span>
<span class="source-line-no">2672</span><span id="line-2672"> protected interface TableDescriptorGetter {</span>
<span class="source-line-no">2673</span><span id="line-2673"> TableDescriptor get() throws IOException;</span>
<span class="source-line-no">2674</span><span id="line-2674"> }</span>
<span class="source-line-no">2675</span><span id="line-2675"></span>
<span class="source-line-no">2676</span><span id="line-2676"> @Override</span>
<span class="source-line-no">2677</span><span id="line-2677"> public long modifyColumn(final TableName tableName, final ColumnFamilyDescriptor descriptor,</span>
<span class="source-line-no">2678</span><span id="line-2678"> final long nonceGroup, final long nonce) throws IOException {</span>
<span class="source-line-no">2679</span><span id="line-2679"> checkInitialized();</span>
<span class="source-line-no">2680</span><span id="line-2680"> checkTableExists(tableName);</span>
<span class="source-line-no">2681</span><span id="line-2681"> return modifyTable(tableName, new TableDescriptorGetter() {</span>
<span class="source-line-no">2682</span><span id="line-2682"></span>
<span class="source-line-no">2683</span><span id="line-2683"> @Override</span>
<span class="source-line-no">2684</span><span id="line-2684"> public TableDescriptor get() throws IOException {</span>
<span class="source-line-no">2685</span><span id="line-2685"> TableDescriptor old = getTableDescriptors().get(tableName);</span>
<span class="source-line-no">2686</span><span id="line-2686"> if (!old.hasColumnFamily(descriptor.getName())) {</span>
<span class="source-line-no">2687</span><span id="line-2687"> throw new InvalidFamilyOperationException("Family '" + descriptor.getNameAsString()</span>
<span class="source-line-no">2688</span><span id="line-2688"> + "' does not exist, so it cannot be modified");</span>
<span class="source-line-no">2689</span><span id="line-2689"> }</span>
<span class="source-line-no">2690</span><span id="line-2690"></span>
<span class="source-line-no">2691</span><span id="line-2691"> return TableDescriptorBuilder.newBuilder(old).modifyColumnFamily(descriptor).build();</span>
<span class="source-line-no">2692</span><span id="line-2692"> }</span>
<span class="source-line-no">2693</span><span id="line-2693"> }, nonceGroup, nonce, true);</span>
<span class="source-line-no">2694</span><span id="line-2694"> }</span>
<span class="source-line-no">2695</span><span id="line-2695"></span>
<span class="source-line-no">2696</span><span id="line-2696"> @Override</span>
<span class="source-line-no">2697</span><span id="line-2697"> public long modifyColumnStoreFileTracker(TableName tableName, byte[] family, String dstSFT,</span>
<span class="source-line-no">2698</span><span id="line-2698"> long nonceGroup, long nonce) throws IOException {</span>
<span class="source-line-no">2699</span><span id="line-2699"> checkInitialized();</span>
<span class="source-line-no">2700</span><span id="line-2700"> return MasterProcedureUtil</span>
<span class="source-line-no">2701</span><span id="line-2701"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">2702</span><span id="line-2702"></span>
<span class="source-line-no">2703</span><span id="line-2703"> @Override</span>
<span class="source-line-no">2704</span><span id="line-2704"> protected void run() throws IOException {</span>
<span class="source-line-no">2705</span><span id="line-2705"> String sft = getMaster().getMasterCoprocessorHost()</span>
<span class="source-line-no">2706</span><span id="line-2706"> .preModifyColumnFamilyStoreFileTracker(tableName, family, dstSFT);</span>
<span class="source-line-no">2707</span><span id="line-2707"> LOG.info("{} modify column {} store file tracker of table {} to {}",</span>
<span class="source-line-no">2708</span><span id="line-2708"> getClientIdAuditPrefix(), Bytes.toStringBinary(family), tableName, sft);</span>
<span class="source-line-no">2709</span><span id="line-2709"> submitProcedure(new ModifyColumnFamilyStoreFileTrackerProcedure(</span>
<span class="source-line-no">2710</span><span id="line-2710"> procedureExecutor.getEnvironment(), tableName, family, sft));</span>
<span class="source-line-no">2711</span><span id="line-2711"> getMaster().getMasterCoprocessorHost().postModifyColumnFamilyStoreFileTracker(tableName,</span>
<span class="source-line-no">2712</span><span id="line-2712"> family, dstSFT);</span>
<span class="source-line-no">2713</span><span id="line-2713"> }</span>
<span class="source-line-no">2714</span><span id="line-2714"></span>
<span class="source-line-no">2715</span><span id="line-2715"> @Override</span>
<span class="source-line-no">2716</span><span id="line-2716"> protected String getDescription() {</span>
<span class="source-line-no">2717</span><span id="line-2717"> return "ModifyColumnFamilyStoreFileTrackerProcedure";</span>
<span class="source-line-no">2718</span><span id="line-2718"> }</span>
<span class="source-line-no">2719</span><span id="line-2719"> });</span>
<span class="source-line-no">2720</span><span id="line-2720"> }</span>
<span class="source-line-no">2721</span><span id="line-2721"></span>
<span class="source-line-no">2722</span><span id="line-2722"> @Override</span>
<span class="source-line-no">2723</span><span id="line-2723"> public long deleteColumn(final TableName tableName, final byte[] columnName,</span>
<span class="source-line-no">2724</span><span id="line-2724"> final long nonceGroup, final long nonce) throws IOException {</span>
<span class="source-line-no">2725</span><span id="line-2725"> checkInitialized();</span>
<span class="source-line-no">2726</span><span id="line-2726"> checkTableExists(tableName);</span>
<span class="source-line-no">2727</span><span id="line-2727"></span>
<span class="source-line-no">2728</span><span id="line-2728"> return modifyTable(tableName, new TableDescriptorGetter() {</span>
<span class="source-line-no">2729</span><span id="line-2729"></span>
<span class="source-line-no">2730</span><span id="line-2730"> @Override</span>
<span class="source-line-no">2731</span><span id="line-2731"> public TableDescriptor get() throws IOException {</span>
<span class="source-line-no">2732</span><span id="line-2732"> TableDescriptor old = getTableDescriptors().get(tableName);</span>
<span class="source-line-no">2733</span><span id="line-2733"></span>
<span class="source-line-no">2734</span><span id="line-2734"> if (!old.hasColumnFamily(columnName)) {</span>
<span class="source-line-no">2735</span><span id="line-2735"> throw new InvalidFamilyOperationException(</span>
<span class="source-line-no">2736</span><span id="line-2736"> "Family '" + Bytes.toString(columnName) + "' does not exist, so it cannot be deleted");</span>
<span class="source-line-no">2737</span><span id="line-2737"> }</span>
<span class="source-line-no">2738</span><span id="line-2738"> if (old.getColumnFamilyCount() == 1) {</span>
<span class="source-line-no">2739</span><span id="line-2739"> throw new InvalidFamilyOperationException("Family '" + Bytes.toString(columnName)</span>
<span class="source-line-no">2740</span><span id="line-2740"> + "' is the only column family in the table, so it cannot be deleted");</span>
<span class="source-line-no">2741</span><span id="line-2741"> }</span>
<span class="source-line-no">2742</span><span id="line-2742"> return TableDescriptorBuilder.newBuilder(old).removeColumnFamily(columnName).build();</span>
<span class="source-line-no">2743</span><span id="line-2743"> }</span>
<span class="source-line-no">2744</span><span id="line-2744"> }, nonceGroup, nonce, true);</span>
<span class="source-line-no">2745</span><span id="line-2745"> }</span>
<span class="source-line-no">2746</span><span id="line-2746"></span>
<span class="source-line-no">2747</span><span id="line-2747"> @Override</span>
<span class="source-line-no">2748</span><span id="line-2748"> public long enableTable(final TableName tableName, final long nonceGroup, final long nonce)</span>
<span class="source-line-no">2749</span><span id="line-2749"> throws IOException {</span>
<span class="source-line-no">2750</span><span id="line-2750"> checkInitialized();</span>
<span class="source-line-no">2751</span><span id="line-2751"></span>
<span class="source-line-no">2752</span><span id="line-2752"> return MasterProcedureUtil</span>
<span class="source-line-no">2753</span><span id="line-2753"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">2754</span><span id="line-2754"> @Override</span>
<span class="source-line-no">2755</span><span id="line-2755"> protected void run() throws IOException {</span>
<span class="source-line-no">2756</span><span id="line-2756"> getMaster().getMasterCoprocessorHost().preEnableTable(tableName);</span>
<span class="source-line-no">2757</span><span id="line-2757"></span>
<span class="source-line-no">2758</span><span id="line-2758"> // Normally, it would make sense for this authorization check to exist inside</span>
<span class="source-line-no">2759</span><span id="line-2759"> // AccessController, but because the authorization check is done based on internal state</span>
<span class="source-line-no">2760</span><span id="line-2760"> // (rather than explicit permissions) we'll do the check here instead of in the</span>
<span class="source-line-no">2761</span><span id="line-2761"> // coprocessor.</span>
<span class="source-line-no">2762</span><span id="line-2762"> MasterQuotaManager quotaManager = getMasterQuotaManager();</span>
<span class="source-line-no">2763</span><span id="line-2763"> if (quotaManager != null) {</span>
<span class="source-line-no">2764</span><span id="line-2764"> if (quotaManager.isQuotaInitialized()) {</span>
<span class="source-line-no">2765</span><span id="line-2765"> // skip checking quotas for system tables, see:</span>
<span class="source-line-no">2766</span><span id="line-2766"> // https://issues.apache.org/jira/browse/HBASE-28183</span>
<span class="source-line-no">2767</span><span id="line-2767"> if (!tableName.isSystemTable()) {</span>
<span class="source-line-no">2768</span><span id="line-2768"> SpaceQuotaSnapshot currSnapshotOfTable =</span>
<span class="source-line-no">2769</span><span id="line-2769"> QuotaTableUtil.getCurrentSnapshotFromQuotaTable(getConnection(), tableName);</span>
<span class="source-line-no">2770</span><span id="line-2770"> if (currSnapshotOfTable != null) {</span>
<span class="source-line-no">2771</span><span id="line-2771"> SpaceQuotaStatus quotaStatus = currSnapshotOfTable.getQuotaStatus();</span>
<span class="source-line-no">2772</span><span id="line-2772"> if (</span>
<span class="source-line-no">2773</span><span id="line-2773"> quotaStatus.isInViolation()</span>
<span class="source-line-no">2774</span><span id="line-2774"> &amp;&amp; SpaceViolationPolicy.DISABLE == quotaStatus.getPolicy().orElse(null)</span>
<span class="source-line-no">2775</span><span id="line-2775"> ) {</span>
<span class="source-line-no">2776</span><span id="line-2776"> throw new AccessDeniedException("Enabling the table '" + tableName</span>
<span class="source-line-no">2777</span><span id="line-2777"> + "' is disallowed due to a violated space quota.");</span>
<span class="source-line-no">2778</span><span id="line-2778"> }</span>
<span class="source-line-no">2779</span><span id="line-2779"> }</span>
<span class="source-line-no">2780</span><span id="line-2780"> }</span>
<span class="source-line-no">2781</span><span id="line-2781"> } else if (LOG.isTraceEnabled()) {</span>
<span class="source-line-no">2782</span><span id="line-2782"> LOG</span>
<span class="source-line-no">2783</span><span id="line-2783"> .trace("Unable to check for space quotas as the MasterQuotaManager is not enabled");</span>
<span class="source-line-no">2784</span><span id="line-2784"> }</span>
<span class="source-line-no">2785</span><span id="line-2785"> }</span>
<span class="source-line-no">2786</span><span id="line-2786"></span>
<span class="source-line-no">2787</span><span id="line-2787"> LOG.info(getClientIdAuditPrefix() + " enable " + tableName);</span>
<span class="source-line-no">2788</span><span id="line-2788"></span>
<span class="source-line-no">2789</span><span id="line-2789"> // Execute the operation asynchronously - client will check the progress of the operation</span>
<span class="source-line-no">2790</span><span id="line-2790"> // In case the request is from a &lt;1.1 client before returning,</span>
<span class="source-line-no">2791</span><span id="line-2791"> // we want to make sure that the table is prepared to be</span>
<span class="source-line-no">2792</span><span id="line-2792"> // enabled (the table is locked and the table state is set).</span>
<span class="source-line-no">2793</span><span id="line-2793"> // Note: if the procedure throws exception, we will catch it and rethrow.</span>
<span class="source-line-no">2794</span><span id="line-2794"> final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();</span>
<span class="source-line-no">2795</span><span id="line-2795"> submitProcedure(</span>
<span class="source-line-no">2796</span><span id="line-2796"> new EnableTableProcedure(procedureExecutor.getEnvironment(), tableName, prepareLatch));</span>
<span class="source-line-no">2797</span><span id="line-2797"> prepareLatch.await();</span>
<span class="source-line-no">2798</span><span id="line-2798"></span>
<span class="source-line-no">2799</span><span id="line-2799"> getMaster().getMasterCoprocessorHost().postEnableTable(tableName);</span>
<span class="source-line-no">2800</span><span id="line-2800"> }</span>
<span class="source-line-no">2801</span><span id="line-2801"></span>
<span class="source-line-no">2802</span><span id="line-2802"> @Override</span>
<span class="source-line-no">2803</span><span id="line-2803"> protected String getDescription() {</span>
<span class="source-line-no">2804</span><span id="line-2804"> return "EnableTableProcedure";</span>
<span class="source-line-no">2805</span><span id="line-2805"> }</span>
<span class="source-line-no">2806</span><span id="line-2806"> });</span>
<span class="source-line-no">2807</span><span id="line-2807"> }</span>
<span class="source-line-no">2808</span><span id="line-2808"></span>
<span class="source-line-no">2809</span><span id="line-2809"> @Override</span>
<span class="source-line-no">2810</span><span id="line-2810"> public long disableTable(final TableName tableName, final long nonceGroup, final long nonce)</span>
<span class="source-line-no">2811</span><span id="line-2811"> throws IOException {</span>
<span class="source-line-no">2812</span><span id="line-2812"> checkInitialized();</span>
<span class="source-line-no">2813</span><span id="line-2813"></span>
<span class="source-line-no">2814</span><span id="line-2814"> return MasterProcedureUtil</span>
<span class="source-line-no">2815</span><span id="line-2815"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">2816</span><span id="line-2816"> @Override</span>
<span class="source-line-no">2817</span><span id="line-2817"> protected void run() throws IOException {</span>
<span class="source-line-no">2818</span><span id="line-2818"> getMaster().getMasterCoprocessorHost().preDisableTable(tableName);</span>
<span class="source-line-no">2819</span><span id="line-2819"></span>
<span class="source-line-no">2820</span><span id="line-2820"> LOG.info(getClientIdAuditPrefix() + " disable " + tableName);</span>
<span class="source-line-no">2821</span><span id="line-2821"></span>
<span class="source-line-no">2822</span><span id="line-2822"> // Execute the operation asynchronously - client will check the progress of the operation</span>
<span class="source-line-no">2823</span><span id="line-2823"> // In case the request is from a &lt;1.1 client before returning,</span>
<span class="source-line-no">2824</span><span id="line-2824"> // we want to make sure that the table is prepared to be</span>
<span class="source-line-no">2825</span><span id="line-2825"> // enabled (the table is locked and the table state is set).</span>
<span class="source-line-no">2826</span><span id="line-2826"> // Note: if the procedure throws exception, we will catch it and rethrow.</span>
<span class="source-line-no">2827</span><span id="line-2827"> //</span>
<span class="source-line-no">2828</span><span id="line-2828"> // We need to wait for the procedure to potentially fail due to "prepare" sanity</span>
<span class="source-line-no">2829</span><span id="line-2829"> // checks. This will block only the beginning of the procedure. See HBASE-19953.</span>
<span class="source-line-no">2830</span><span id="line-2830"> final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createBlockingLatch();</span>
<span class="source-line-no">2831</span><span id="line-2831"> submitProcedure(new DisableTableProcedure(procedureExecutor.getEnvironment(), tableName,</span>
<span class="source-line-no">2832</span><span id="line-2832"> false, prepareLatch));</span>
<span class="source-line-no">2833</span><span id="line-2833"> prepareLatch.await();</span>
<span class="source-line-no">2834</span><span id="line-2834"></span>
<span class="source-line-no">2835</span><span id="line-2835"> getMaster().getMasterCoprocessorHost().postDisableTable(tableName);</span>
<span class="source-line-no">2836</span><span id="line-2836"> }</span>
<span class="source-line-no">2837</span><span id="line-2837"></span>
<span class="source-line-no">2838</span><span id="line-2838"> @Override</span>
<span class="source-line-no">2839</span><span id="line-2839"> protected String getDescription() {</span>
<span class="source-line-no">2840</span><span id="line-2840"> return "DisableTableProcedure";</span>
<span class="source-line-no">2841</span><span id="line-2841"> }</span>
<span class="source-line-no">2842</span><span id="line-2842"> });</span>
<span class="source-line-no">2843</span><span id="line-2843"> }</span>
<span class="source-line-no">2844</span><span id="line-2844"></span>
<span class="source-line-no">2845</span><span id="line-2845"> private long modifyTable(final TableName tableName,</span>
<span class="source-line-no">2846</span><span id="line-2846"> final TableDescriptorGetter newDescriptorGetter, final long nonceGroup, final long nonce,</span>
<span class="source-line-no">2847</span><span id="line-2847"> final boolean shouldCheckDescriptor) throws IOException {</span>
<span class="source-line-no">2848</span><span id="line-2848"> return modifyTable(tableName, newDescriptorGetter, nonceGroup, nonce, shouldCheckDescriptor,</span>
<span class="source-line-no">2849</span><span id="line-2849"> true);</span>
<span class="source-line-no">2850</span><span id="line-2850"> }</span>
<span class="source-line-no">2851</span><span id="line-2851"></span>
<span class="source-line-no">2852</span><span id="line-2852"> private long modifyTable(final TableName tableName,</span>
<span class="source-line-no">2853</span><span id="line-2853"> final TableDescriptorGetter newDescriptorGetter, final long nonceGroup, final long nonce,</span>
<span class="source-line-no">2854</span><span id="line-2854"> final boolean shouldCheckDescriptor, final boolean reopenRegions) throws IOException {</span>
<span class="source-line-no">2855</span><span id="line-2855"> return MasterProcedureUtil</span>
<span class="source-line-no">2856</span><span id="line-2856"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">2857</span><span id="line-2857"> @Override</span>
<span class="source-line-no">2858</span><span id="line-2858"> protected void run() throws IOException {</span>
<span class="source-line-no">2859</span><span id="line-2859"> TableDescriptor oldDescriptor = getMaster().getTableDescriptors().get(tableName);</span>
<span class="source-line-no">2860</span><span id="line-2860"> TableDescriptor newDescriptor = getMaster().getMasterCoprocessorHost()</span>
<span class="source-line-no">2861</span><span id="line-2861"> .preModifyTable(tableName, oldDescriptor, newDescriptorGetter.get());</span>
<span class="source-line-no">2862</span><span id="line-2862"> TableDescriptorChecker.sanityCheck(conf, newDescriptor);</span>
<span class="source-line-no">2863</span><span id="line-2863"> LOG.info("{} modify table {} from {} to {}", getClientIdAuditPrefix(), tableName,</span>
<span class="source-line-no">2864</span><span id="line-2864"> oldDescriptor, newDescriptor);</span>
<span class="source-line-no">2865</span><span id="line-2865"></span>
<span class="source-line-no">2866</span><span id="line-2866"> // Execute the operation synchronously - wait for the operation completes before</span>
<span class="source-line-no">2867</span><span id="line-2867"> // continuing.</span>
<span class="source-line-no">2868</span><span id="line-2868"> //</span>
<span class="source-line-no">2869</span><span id="line-2869"> // We need to wait for the procedure to potentially fail due to "prepare" sanity</span>
<span class="source-line-no">2870</span><span id="line-2870"> // checks. This will block only the beginning of the procedure. See HBASE-19953.</span>
<span class="source-line-no">2871</span><span id="line-2871"> ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();</span>
<span class="source-line-no">2872</span><span id="line-2872"> submitProcedure(new ModifyTableProcedure(procedureExecutor.getEnvironment(),</span>
<span class="source-line-no">2873</span><span id="line-2873"> newDescriptor, latch, oldDescriptor, shouldCheckDescriptor, reopenRegions));</span>
<span class="source-line-no">2874</span><span id="line-2874"> latch.await();</span>
<span class="source-line-no">2875</span><span id="line-2875"></span>
<span class="source-line-no">2876</span><span id="line-2876"> getMaster().getMasterCoprocessorHost().postModifyTable(tableName, oldDescriptor,</span>
<span class="source-line-no">2877</span><span id="line-2877"> newDescriptor);</span>
<span class="source-line-no">2878</span><span id="line-2878"> }</span>
<span class="source-line-no">2879</span><span id="line-2879"></span>
<span class="source-line-no">2880</span><span id="line-2880"> @Override</span>
<span class="source-line-no">2881</span><span id="line-2881"> protected String getDescription() {</span>
<span class="source-line-no">2882</span><span id="line-2882"> return "ModifyTableProcedure";</span>
<span class="source-line-no">2883</span><span id="line-2883"> }</span>
<span class="source-line-no">2884</span><span id="line-2884"> });</span>
<span class="source-line-no">2885</span><span id="line-2885"></span>
<span class="source-line-no">2886</span><span id="line-2886"> }</span>
<span class="source-line-no">2887</span><span id="line-2887"></span>
<span class="source-line-no">2888</span><span id="line-2888"> @Override</span>
<span class="source-line-no">2889</span><span id="line-2889"> public long modifyTable(final TableName tableName, final TableDescriptor newDescriptor,</span>
<span class="source-line-no">2890</span><span id="line-2890"> final long nonceGroup, final long nonce, final boolean reopenRegions) throws IOException {</span>
<span class="source-line-no">2891</span><span id="line-2891"> checkInitialized();</span>
<span class="source-line-no">2892</span><span id="line-2892"> return modifyTable(tableName, new TableDescriptorGetter() {</span>
<span class="source-line-no">2893</span><span id="line-2893"> @Override</span>
<span class="source-line-no">2894</span><span id="line-2894"> public TableDescriptor get() throws IOException {</span>
<span class="source-line-no">2895</span><span id="line-2895"> return newDescriptor;</span>
<span class="source-line-no">2896</span><span id="line-2896"> }</span>
<span class="source-line-no">2897</span><span id="line-2897"> }, nonceGroup, nonce, false, reopenRegions);</span>
<span class="source-line-no">2898</span><span id="line-2898"></span>
<span class="source-line-no">2899</span><span id="line-2899"> }</span>
<span class="source-line-no">2900</span><span id="line-2900"></span>
<span class="source-line-no">2901</span><span id="line-2901"> @Override</span>
<span class="source-line-no">2902</span><span id="line-2902"> public long modifyTableStoreFileTracker(TableName tableName, String dstSFT, long nonceGroup,</span>
<span class="source-line-no">2903</span><span id="line-2903"> long nonce) throws IOException {</span>
<span class="source-line-no">2904</span><span id="line-2904"> checkInitialized();</span>
<span class="source-line-no">2905</span><span id="line-2905"> return MasterProcedureUtil</span>
<span class="source-line-no">2906</span><span id="line-2906"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">2907</span><span id="line-2907"></span>
<span class="source-line-no">2908</span><span id="line-2908"> @Override</span>
<span class="source-line-no">2909</span><span id="line-2909"> protected void run() throws IOException {</span>
<span class="source-line-no">2910</span><span id="line-2910"> String sft = getMaster().getMasterCoprocessorHost()</span>
<span class="source-line-no">2911</span><span id="line-2911"> .preModifyTableStoreFileTracker(tableName, dstSFT);</span>
<span class="source-line-no">2912</span><span id="line-2912"> LOG.info("{} modify table store file tracker of table {} to {}", getClientIdAuditPrefix(),</span>
<span class="source-line-no">2913</span><span id="line-2913"> tableName, sft);</span>
<span class="source-line-no">2914</span><span id="line-2914"> submitProcedure(new ModifyTableStoreFileTrackerProcedure(</span>
<span class="source-line-no">2915</span><span id="line-2915"> procedureExecutor.getEnvironment(), tableName, sft));</span>
<span class="source-line-no">2916</span><span id="line-2916"> getMaster().getMasterCoprocessorHost().postModifyTableStoreFileTracker(tableName, sft);</span>
<span class="source-line-no">2917</span><span id="line-2917"> }</span>
<span class="source-line-no">2918</span><span id="line-2918"></span>
<span class="source-line-no">2919</span><span id="line-2919"> @Override</span>
<span class="source-line-no">2920</span><span id="line-2920"> protected String getDescription() {</span>
<span class="source-line-no">2921</span><span id="line-2921"> return "ModifyTableStoreFileTrackerProcedure";</span>
<span class="source-line-no">2922</span><span id="line-2922"> }</span>
<span class="source-line-no">2923</span><span id="line-2923"> });</span>
<span class="source-line-no">2924</span><span id="line-2924"> }</span>
<span class="source-line-no">2925</span><span id="line-2925"></span>
<span class="source-line-no">2926</span><span id="line-2926"> public long restoreSnapshot(final SnapshotDescription snapshotDesc, final long nonceGroup,</span>
<span class="source-line-no">2927</span><span id="line-2927"> final long nonce, final boolean restoreAcl, final String customSFT) throws IOException {</span>
<span class="source-line-no">2928</span><span id="line-2928"> checkInitialized();</span>
<span class="source-line-no">2929</span><span id="line-2929"> getSnapshotManager().checkSnapshotSupport();</span>
<span class="source-line-no">2930</span><span id="line-2930"></span>
<span class="source-line-no">2931</span><span id="line-2931"> // Ensure namespace exists. Will throw exception if non-known NS.</span>
<span class="source-line-no">2932</span><span id="line-2932"> final TableName dstTable = TableName.valueOf(snapshotDesc.getTable());</span>
<span class="source-line-no">2933</span><span id="line-2933"> getClusterSchema().getNamespace(dstTable.getNamespaceAsString());</span>
<span class="source-line-no">2934</span><span id="line-2934"></span>
<span class="source-line-no">2935</span><span id="line-2935"> return MasterProcedureUtil</span>
<span class="source-line-no">2936</span><span id="line-2936"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">2937</span><span id="line-2937"> @Override</span>
<span class="source-line-no">2938</span><span id="line-2938"> protected void run() throws IOException {</span>
<span class="source-line-no">2939</span><span id="line-2939"> setProcId(getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc, getNonceKey(),</span>
<span class="source-line-no">2940</span><span id="line-2940"> restoreAcl, customSFT));</span>
<span class="source-line-no">2941</span><span id="line-2941"> }</span>
<span class="source-line-no">2942</span><span id="line-2942"></span>
<span class="source-line-no">2943</span><span id="line-2943"> @Override</span>
<span class="source-line-no">2944</span><span id="line-2944"> protected String getDescription() {</span>
<span class="source-line-no">2945</span><span id="line-2945"> return "RestoreSnapshotProcedure";</span>
<span class="source-line-no">2946</span><span id="line-2946"> }</span>
<span class="source-line-no">2947</span><span id="line-2947"> });</span>
<span class="source-line-no">2948</span><span id="line-2948"> }</span>
<span class="source-line-no">2949</span><span id="line-2949"></span>
<span class="source-line-no">2950</span><span id="line-2950"> private void checkTableExists(final TableName tableName)</span>
<span class="source-line-no">2951</span><span id="line-2951"> throws IOException, TableNotFoundException {</span>
<span class="source-line-no">2952</span><span id="line-2952"> if (!tableDescriptors.exists(tableName)) {</span>
<span class="source-line-no">2953</span><span id="line-2953"> throw new TableNotFoundException(tableName);</span>
<span class="source-line-no">2954</span><span id="line-2954"> }</span>
<span class="source-line-no">2955</span><span id="line-2955"> }</span>
<span class="source-line-no">2956</span><span id="line-2956"></span>
<span class="source-line-no">2957</span><span id="line-2957"> @Override</span>
<span class="source-line-no">2958</span><span id="line-2958"> public void checkTableModifiable(final TableName tableName)</span>
<span class="source-line-no">2959</span><span id="line-2959"> throws IOException, TableNotFoundException, TableNotDisabledException {</span>
<span class="source-line-no">2960</span><span id="line-2960"> if (isCatalogTable(tableName)) {</span>
<span class="source-line-no">2961</span><span id="line-2961"> throw new IOException("Can't modify catalog tables");</span>
<span class="source-line-no">2962</span><span id="line-2962"> }</span>
<span class="source-line-no">2963</span><span id="line-2963"> checkTableExists(tableName);</span>
<span class="source-line-no">2964</span><span id="line-2964"> TableState ts = getTableStateManager().getTableState(tableName);</span>
<span class="source-line-no">2965</span><span id="line-2965"> if (!ts.isDisabled()) {</span>
<span class="source-line-no">2966</span><span id="line-2966"> throw new TableNotDisabledException("Not DISABLED; " + ts);</span>
<span class="source-line-no">2967</span><span id="line-2967"> }</span>
<span class="source-line-no">2968</span><span id="line-2968"> }</span>
<span class="source-line-no">2969</span><span id="line-2969"></span>
<span class="source-line-no">2970</span><span id="line-2970"> public ClusterMetrics getClusterMetricsWithoutCoprocessor() throws InterruptedIOException {</span>
<span class="source-line-no">2971</span><span id="line-2971"> return getClusterMetricsWithoutCoprocessor(EnumSet.allOf(Option.class));</span>
<span class="source-line-no">2972</span><span id="line-2972"> }</span>
<span class="source-line-no">2973</span><span id="line-2973"></span>
<span class="source-line-no">2974</span><span id="line-2974"> public ClusterMetrics getClusterMetricsWithoutCoprocessor(EnumSet&lt;Option&gt; options)</span>
<span class="source-line-no">2975</span><span id="line-2975"> throws InterruptedIOException {</span>
<span class="source-line-no">2976</span><span id="line-2976"> ClusterMetricsBuilder builder = ClusterMetricsBuilder.newBuilder();</span>
<span class="source-line-no">2977</span><span id="line-2977"> // given that hbase1 can't submit the request with Option,</span>
<span class="source-line-no">2978</span><span id="line-2978"> // we return all information to client if the list of Option is empty.</span>
<span class="source-line-no">2979</span><span id="line-2979"> if (options.isEmpty()) {</span>
<span class="source-line-no">2980</span><span id="line-2980"> options = EnumSet.allOf(Option.class);</span>
<span class="source-line-no">2981</span><span id="line-2981"> }</span>
<span class="source-line-no">2982</span><span id="line-2982"></span>
<span class="source-line-no">2983</span><span id="line-2983"> // TASKS and/or LIVE_SERVERS will populate this map, which will be given to the builder if</span>
<span class="source-line-no">2984</span><span id="line-2984"> // not null after option processing completes.</span>
<span class="source-line-no">2985</span><span id="line-2985"> Map&lt;ServerName, ServerMetrics&gt; serverMetricsMap = null;</span>
<span class="source-line-no">2986</span><span id="line-2986"></span>
<span class="source-line-no">2987</span><span id="line-2987"> for (Option opt : options) {</span>
<span class="source-line-no">2988</span><span id="line-2988"> switch (opt) {</span>
<span class="source-line-no">2989</span><span id="line-2989"> case HBASE_VERSION:</span>
<span class="source-line-no">2990</span><span id="line-2990"> builder.setHBaseVersion(VersionInfo.getVersion());</span>
<span class="source-line-no">2991</span><span id="line-2991"> break;</span>
<span class="source-line-no">2992</span><span id="line-2992"> case CLUSTER_ID:</span>
<span class="source-line-no">2993</span><span id="line-2993"> builder.setClusterId(getClusterId());</span>
<span class="source-line-no">2994</span><span id="line-2994"> break;</span>
<span class="source-line-no">2995</span><span id="line-2995"> case MASTER:</span>
<span class="source-line-no">2996</span><span id="line-2996"> builder.setMasterName(getServerName());</span>
<span class="source-line-no">2997</span><span id="line-2997"> break;</span>
<span class="source-line-no">2998</span><span id="line-2998"> case BACKUP_MASTERS:</span>
<span class="source-line-no">2999</span><span id="line-2999"> builder.setBackerMasterNames(getBackupMasters());</span>
<span class="source-line-no">3000</span><span id="line-3000"> break;</span>
<span class="source-line-no">3001</span><span id="line-3001"> case TASKS: {</span>
<span class="source-line-no">3002</span><span id="line-3002"> // Master tasks</span>
<span class="source-line-no">3003</span><span id="line-3003"> builder.setMasterTasks(TaskMonitor.get().getTasks().stream()</span>
<span class="source-line-no">3004</span><span id="line-3004"> .map(task -&gt; ServerTaskBuilder.newBuilder().setDescription(task.getDescription())</span>
<span class="source-line-no">3005</span><span id="line-3005"> .setStatus(task.getStatus())</span>
<span class="source-line-no">3006</span><span id="line-3006"> .setState(ServerTask.State.valueOf(task.getState().name()))</span>
<span class="source-line-no">3007</span><span id="line-3007"> .setStartTime(task.getStartTime()).setCompletionTime(task.getCompletionTimestamp())</span>
<span class="source-line-no">3008</span><span id="line-3008"> .build())</span>
<span class="source-line-no">3009</span><span id="line-3009"> .collect(Collectors.toList()));</span>
<span class="source-line-no">3010</span><span id="line-3010"> // TASKS is also synonymous with LIVE_SERVERS for now because task information for</span>
<span class="source-line-no">3011</span><span id="line-3011"> // regionservers is carried in ServerLoad.</span>
<span class="source-line-no">3012</span><span id="line-3012"> // Add entries to serverMetricsMap for all live servers, if we haven't already done so</span>
<span class="source-line-no">3013</span><span id="line-3013"> if (serverMetricsMap == null) {</span>
<span class="source-line-no">3014</span><span id="line-3014"> serverMetricsMap = getOnlineServers();</span>
<span class="source-line-no">3015</span><span id="line-3015"> }</span>
<span class="source-line-no">3016</span><span id="line-3016"> break;</span>
<span class="source-line-no">3017</span><span id="line-3017"> }</span>
<span class="source-line-no">3018</span><span id="line-3018"> case LIVE_SERVERS: {</span>
<span class="source-line-no">3019</span><span id="line-3019"> // Add entries to serverMetricsMap for all live servers, if we haven't already done so</span>
<span class="source-line-no">3020</span><span id="line-3020"> if (serverMetricsMap == null) {</span>
<span class="source-line-no">3021</span><span id="line-3021"> serverMetricsMap = getOnlineServers();</span>
<span class="source-line-no">3022</span><span id="line-3022"> }</span>
<span class="source-line-no">3023</span><span id="line-3023"> break;</span>
<span class="source-line-no">3024</span><span id="line-3024"> }</span>
<span class="source-line-no">3025</span><span id="line-3025"> case DEAD_SERVERS: {</span>
<span class="source-line-no">3026</span><span id="line-3026"> if (serverManager != null) {</span>
<span class="source-line-no">3027</span><span id="line-3027"> builder.setDeadServerNames(</span>
<span class="source-line-no">3028</span><span id="line-3028"> new ArrayList&lt;&gt;(serverManager.getDeadServers().copyServerNames()));</span>
<span class="source-line-no">3029</span><span id="line-3029"> }</span>
<span class="source-line-no">3030</span><span id="line-3030"> break;</span>
<span class="source-line-no">3031</span><span id="line-3031"> }</span>
<span class="source-line-no">3032</span><span id="line-3032"> case UNKNOWN_SERVERS: {</span>
<span class="source-line-no">3033</span><span id="line-3033"> if (serverManager != null) {</span>
<span class="source-line-no">3034</span><span id="line-3034"> builder.setUnknownServerNames(getUnknownServers());</span>
<span class="source-line-no">3035</span><span id="line-3035"> }</span>
<span class="source-line-no">3036</span><span id="line-3036"> break;</span>
<span class="source-line-no">3037</span><span id="line-3037"> }</span>
<span class="source-line-no">3038</span><span id="line-3038"> case MASTER_COPROCESSORS: {</span>
<span class="source-line-no">3039</span><span id="line-3039"> if (cpHost != null) {</span>
<span class="source-line-no">3040</span><span id="line-3040"> builder.setMasterCoprocessorNames(Arrays.asList(getMasterCoprocessors()));</span>
<span class="source-line-no">3041</span><span id="line-3041"> }</span>
<span class="source-line-no">3042</span><span id="line-3042"> break;</span>
<span class="source-line-no">3043</span><span id="line-3043"> }</span>
<span class="source-line-no">3044</span><span id="line-3044"> case REGIONS_IN_TRANSITION: {</span>
<span class="source-line-no">3045</span><span id="line-3045"> if (assignmentManager != null) {</span>
<span class="source-line-no">3046</span><span id="line-3046"> builder.setRegionsInTransition(</span>
<span class="source-line-no">3047</span><span id="line-3047"> assignmentManager.getRegionStates().getRegionsStateInTransition());</span>
<span class="source-line-no">3048</span><span id="line-3048"> }</span>
<span class="source-line-no">3049</span><span id="line-3049"> break;</span>
<span class="source-line-no">3050</span><span id="line-3050"> }</span>
<span class="source-line-no">3051</span><span id="line-3051"> case BALANCER_ON: {</span>
<span class="source-line-no">3052</span><span id="line-3052"> if (loadBalancerStateStore != null) {</span>
<span class="source-line-no">3053</span><span id="line-3053"> builder.setBalancerOn(loadBalancerStateStore.get());</span>
<span class="source-line-no">3054</span><span id="line-3054"> }</span>
<span class="source-line-no">3055</span><span id="line-3055"> break;</span>
<span class="source-line-no">3056</span><span id="line-3056"> }</span>
<span class="source-line-no">3057</span><span id="line-3057"> case MASTER_INFO_PORT: {</span>
<span class="source-line-no">3058</span><span id="line-3058"> if (infoServer != null) {</span>
<span class="source-line-no">3059</span><span id="line-3059"> builder.setMasterInfoPort(infoServer.getPort());</span>
<span class="source-line-no">3060</span><span id="line-3060"> }</span>
<span class="source-line-no">3061</span><span id="line-3061"> break;</span>
<span class="source-line-no">3062</span><span id="line-3062"> }</span>
<span class="source-line-no">3063</span><span id="line-3063"> case SERVERS_NAME: {</span>
<span class="source-line-no">3064</span><span id="line-3064"> if (serverManager != null) {</span>
<span class="source-line-no">3065</span><span id="line-3065"> builder.setServerNames(serverManager.getOnlineServersList());</span>
<span class="source-line-no">3066</span><span id="line-3066"> }</span>
<span class="source-line-no">3067</span><span id="line-3067"> break;</span>
<span class="source-line-no">3068</span><span id="line-3068"> }</span>
<span class="source-line-no">3069</span><span id="line-3069"> case TABLE_TO_REGIONS_COUNT: {</span>
<span class="source-line-no">3070</span><span id="line-3070"> if (isActiveMaster() &amp;&amp; isInitialized() &amp;&amp; assignmentManager != null) {</span>
<span class="source-line-no">3071</span><span id="line-3071"> try {</span>
<span class="source-line-no">3072</span><span id="line-3072"> Map&lt;TableName, RegionStatesCount&gt; tableRegionStatesCountMap = new HashMap&lt;&gt;();</span>
<span class="source-line-no">3073</span><span id="line-3073"> Map&lt;String, TableDescriptor&gt; tableDescriptorMap = getTableDescriptors().getAll();</span>
<span class="source-line-no">3074</span><span id="line-3074"> for (TableDescriptor tableDescriptor : tableDescriptorMap.values()) {</span>
<span class="source-line-no">3075</span><span id="line-3075"> TableName tableName = tableDescriptor.getTableName();</span>
<span class="source-line-no">3076</span><span id="line-3076"> RegionStatesCount regionStatesCount =</span>
<span class="source-line-no">3077</span><span id="line-3077"> assignmentManager.getRegionStatesCount(tableName);</span>
<span class="source-line-no">3078</span><span id="line-3078"> tableRegionStatesCountMap.put(tableName, regionStatesCount);</span>
<span class="source-line-no">3079</span><span id="line-3079"> }</span>
<span class="source-line-no">3080</span><span id="line-3080"> builder.setTableRegionStatesCount(tableRegionStatesCountMap);</span>
<span class="source-line-no">3081</span><span id="line-3081"> } catch (IOException e) {</span>
<span class="source-line-no">3082</span><span id="line-3082"> LOG.error("Error while populating TABLE_TO_REGIONS_COUNT for Cluster Metrics..", e);</span>
<span class="source-line-no">3083</span><span id="line-3083"> }</span>
<span class="source-line-no">3084</span><span id="line-3084"> }</span>
<span class="source-line-no">3085</span><span id="line-3085"> break;</span>
<span class="source-line-no">3086</span><span id="line-3086"> }</span>
<span class="source-line-no">3087</span><span id="line-3087"> case DECOMMISSIONED_SERVERS: {</span>
<span class="source-line-no">3088</span><span id="line-3088"> if (serverManager != null) {</span>
<span class="source-line-no">3089</span><span id="line-3089"> builder.setDecommissionedServerNames(serverManager.getDrainingServersList());</span>
<span class="source-line-no">3090</span><span id="line-3090"> }</span>
<span class="source-line-no">3091</span><span id="line-3091"> break;</span>
<span class="source-line-no">3092</span><span id="line-3092"> }</span>
<span class="source-line-no">3093</span><span id="line-3093"> }</span>
<span class="source-line-no">3094</span><span id="line-3094"> }</span>
<span class="source-line-no">3095</span><span id="line-3095"></span>
<span class="source-line-no">3096</span><span id="line-3096"> if (serverMetricsMap != null) {</span>
<span class="source-line-no">3097</span><span id="line-3097"> builder.setLiveServerMetrics(serverMetricsMap);</span>
<span class="source-line-no">3098</span><span id="line-3098"> }</span>
<span class="source-line-no">3099</span><span id="line-3099"></span>
<span class="source-line-no">3100</span><span id="line-3100"> return builder.build();</span>
<span class="source-line-no">3101</span><span id="line-3101"> }</span>
<span class="source-line-no">3102</span><span id="line-3102"></span>
<span class="source-line-no">3103</span><span id="line-3103"> private List&lt;ServerName&gt; getUnknownServers() {</span>
<span class="source-line-no">3104</span><span id="line-3104"> if (serverManager != null) {</span>
<span class="source-line-no">3105</span><span id="line-3105"> final Set&lt;ServerName&gt; serverNames = getAssignmentManager().getRegionStates().getRegionStates()</span>
<span class="source-line-no">3106</span><span id="line-3106"> .stream().map(RegionState::getServerName).collect(Collectors.toSet());</span>
<span class="source-line-no">3107</span><span id="line-3107"> final List&lt;ServerName&gt; unknownServerNames = serverNames.stream()</span>
<span class="source-line-no">3108</span><span id="line-3108"> .filter(sn -&gt; sn != null &amp;&amp; serverManager.isServerUnknown(sn)).collect(Collectors.toList());</span>
<span class="source-line-no">3109</span><span id="line-3109"> return unknownServerNames;</span>
<span class="source-line-no">3110</span><span id="line-3110"> }</span>
<span class="source-line-no">3111</span><span id="line-3111"> return null;</span>
<span class="source-line-no">3112</span><span id="line-3112"> }</span>
<span class="source-line-no">3113</span><span id="line-3113"></span>
<span class="source-line-no">3114</span><span id="line-3114"> private Map&lt;ServerName, ServerMetrics&gt; getOnlineServers() {</span>
<span class="source-line-no">3115</span><span id="line-3115"> if (serverManager != null) {</span>
<span class="source-line-no">3116</span><span id="line-3116"> final Map&lt;ServerName, ServerMetrics&gt; map = new HashMap&lt;&gt;();</span>
<span class="source-line-no">3117</span><span id="line-3117"> serverManager.getOnlineServers().entrySet().forEach(e -&gt; map.put(e.getKey(), e.getValue()));</span>
<span class="source-line-no">3118</span><span id="line-3118"> return map;</span>
<span class="source-line-no">3119</span><span id="line-3119"> }</span>
<span class="source-line-no">3120</span><span id="line-3120"> return null;</span>
<span class="source-line-no">3121</span><span id="line-3121"> }</span>
<span class="source-line-no">3122</span><span id="line-3122"></span>
<span class="source-line-no">3123</span><span id="line-3123"> /** Returns cluster status */</span>
<span class="source-line-no">3124</span><span id="line-3124"> public ClusterMetrics getClusterMetrics() throws IOException {</span>
<span class="source-line-no">3125</span><span id="line-3125"> return getClusterMetrics(EnumSet.allOf(Option.class));</span>
<span class="source-line-no">3126</span><span id="line-3126"> }</span>
<span class="source-line-no">3127</span><span id="line-3127"></span>
<span class="source-line-no">3128</span><span id="line-3128"> public ClusterMetrics getClusterMetrics(EnumSet&lt;Option&gt; options) throws IOException {</span>
<span class="source-line-no">3129</span><span id="line-3129"> if (cpHost != null) {</span>
<span class="source-line-no">3130</span><span id="line-3130"> cpHost.preGetClusterMetrics();</span>
<span class="source-line-no">3131</span><span id="line-3131"> }</span>
<span class="source-line-no">3132</span><span id="line-3132"> ClusterMetrics status = getClusterMetricsWithoutCoprocessor(options);</span>
<span class="source-line-no">3133</span><span id="line-3133"> if (cpHost != null) {</span>
<span class="source-line-no">3134</span><span id="line-3134"> cpHost.postGetClusterMetrics(status);</span>
<span class="source-line-no">3135</span><span id="line-3135"> }</span>
<span class="source-line-no">3136</span><span id="line-3136"> return status;</span>
<span class="source-line-no">3137</span><span id="line-3137"> }</span>
<span class="source-line-no">3138</span><span id="line-3138"></span>
<span class="source-line-no">3139</span><span id="line-3139"> /** Returns info port of active master or 0 if any exception occurs. */</span>
<span class="source-line-no">3140</span><span id="line-3140"> public int getActiveMasterInfoPort() {</span>
<span class="source-line-no">3141</span><span id="line-3141"> return activeMasterManager.getActiveMasterInfoPort();</span>
<span class="source-line-no">3142</span><span id="line-3142"> }</span>
<span class="source-line-no">3143</span><span id="line-3143"></span>
<span class="source-line-no">3144</span><span id="line-3144"> /**</span>
<span class="source-line-no">3145</span><span id="line-3145"> * @param sn is ServerName of the backup master</span>
<span class="source-line-no">3146</span><span id="line-3146"> * @return info port of backup master or 0 if any exception occurs.</span>
<span class="source-line-no">3147</span><span id="line-3147"> */</span>
<span class="source-line-no">3148</span><span id="line-3148"> public int getBackupMasterInfoPort(final ServerName sn) {</span>
<span class="source-line-no">3149</span><span id="line-3149"> return activeMasterManager.getBackupMasterInfoPort(sn);</span>
<span class="source-line-no">3150</span><span id="line-3150"> }</span>
<span class="source-line-no">3151</span><span id="line-3151"></span>
<span class="source-line-no">3152</span><span id="line-3152"> /**</span>
<span class="source-line-no">3153</span><span id="line-3153"> * The set of loaded coprocessors is stored in a static set. Since it's statically allocated, it</span>
<span class="source-line-no">3154</span><span id="line-3154"> * does not require that HMaster's cpHost be initialized prior to accessing it.</span>
<span class="source-line-no">3155</span><span id="line-3155"> * @return a String representation of the set of names of the loaded coprocessors.</span>
<span class="source-line-no">3156</span><span id="line-3156"> */</span>
<span class="source-line-no">3157</span><span id="line-3157"> public static String getLoadedCoprocessors() {</span>
<span class="source-line-no">3158</span><span id="line-3158"> return CoprocessorHost.getLoadedCoprocessors().toString();</span>
<span class="source-line-no">3159</span><span id="line-3159"> }</span>
<span class="source-line-no">3160</span><span id="line-3160"></span>
<span class="source-line-no">3161</span><span id="line-3161"> /** Returns timestamp in millis when HMaster was started. */</span>
<span class="source-line-no">3162</span><span id="line-3162"> public long getMasterStartTime() {</span>
<span class="source-line-no">3163</span><span id="line-3163"> return startcode;</span>
<span class="source-line-no">3164</span><span id="line-3164"> }</span>
<span class="source-line-no">3165</span><span id="line-3165"></span>
<span class="source-line-no">3166</span><span id="line-3166"> /** Returns timestamp in millis when HMaster became the active master. */</span>
<span class="source-line-no">3167</span><span id="line-3167"> @Override</span>
<span class="source-line-no">3168</span><span id="line-3168"> public long getMasterActiveTime() {</span>
<span class="source-line-no">3169</span><span id="line-3169"> return masterActiveTime;</span>
<span class="source-line-no">3170</span><span id="line-3170"> }</span>
<span class="source-line-no">3171</span><span id="line-3171"></span>
<span class="source-line-no">3172</span><span id="line-3172"> /** Returns timestamp in millis when HMaster finished becoming the active master */</span>
<span class="source-line-no">3173</span><span id="line-3173"> public long getMasterFinishedInitializationTime() {</span>
<span class="source-line-no">3174</span><span id="line-3174"> return masterFinishedInitializationTime;</span>
<span class="source-line-no">3175</span><span id="line-3175"> }</span>
<span class="source-line-no">3176</span><span id="line-3176"></span>
<span class="source-line-no">3177</span><span id="line-3177"> public int getNumWALFiles() {</span>
<span class="source-line-no">3178</span><span id="line-3178"> return 0;</span>
<span class="source-line-no">3179</span><span id="line-3179"> }</span>
<span class="source-line-no">3180</span><span id="line-3180"></span>
<span class="source-line-no">3181</span><span id="line-3181"> public ProcedureStore getProcedureStore() {</span>
<span class="source-line-no">3182</span><span id="line-3182"> return procedureStore;</span>
<span class="source-line-no">3183</span><span id="line-3183"> }</span>
<span class="source-line-no">3184</span><span id="line-3184"></span>
<span class="source-line-no">3185</span><span id="line-3185"> public int getRegionServerInfoPort(final ServerName sn) {</span>
<span class="source-line-no">3186</span><span id="line-3186"> int port = this.serverManager.getInfoPort(sn);</span>
<span class="source-line-no">3187</span><span id="line-3187"> return port == 0</span>
<span class="source-line-no">3188</span><span id="line-3188"> ? conf.getInt(HConstants.REGIONSERVER_INFO_PORT, HConstants.DEFAULT_REGIONSERVER_INFOPORT)</span>
<span class="source-line-no">3189</span><span id="line-3189"> : port;</span>
<span class="source-line-no">3190</span><span id="line-3190"> }</span>
<span class="source-line-no">3191</span><span id="line-3191"></span>
<span class="source-line-no">3192</span><span id="line-3192"> @Override</span>
<span class="source-line-no">3193</span><span id="line-3193"> public String getRegionServerVersion(ServerName sn) {</span>
<span class="source-line-no">3194</span><span id="line-3194"> // Will return "0.0.0" if the server is not online to prevent move system region to unknown</span>
<span class="source-line-no">3195</span><span id="line-3195"> // version RS.</span>
<span class="source-line-no">3196</span><span id="line-3196"> return this.serverManager.getVersion(sn);</span>
<span class="source-line-no">3197</span><span id="line-3197"> }</span>
<span class="source-line-no">3198</span><span id="line-3198"></span>
<span class="source-line-no">3199</span><span id="line-3199"> @Override</span>
<span class="source-line-no">3200</span><span id="line-3200"> public void checkIfShouldMoveSystemRegionAsync() {</span>
<span class="source-line-no">3201</span><span id="line-3201"> assignmentManager.checkIfShouldMoveSystemRegionAsync();</span>
<span class="source-line-no">3202</span><span id="line-3202"> }</span>
<span class="source-line-no">3203</span><span id="line-3203"></span>
<span class="source-line-no">3204</span><span id="line-3204"> /** Returns array of coprocessor SimpleNames. */</span>
<span class="source-line-no">3205</span><span id="line-3205"> public String[] getMasterCoprocessors() {</span>
<span class="source-line-no">3206</span><span id="line-3206"> Set&lt;String&gt; masterCoprocessors = getMasterCoprocessorHost().getCoprocessors();</span>
<span class="source-line-no">3207</span><span id="line-3207"> return masterCoprocessors.toArray(new String[masterCoprocessors.size()]);</span>
<span class="source-line-no">3208</span><span id="line-3208"> }</span>
<span class="source-line-no">3209</span><span id="line-3209"></span>
<span class="source-line-no">3210</span><span id="line-3210"> @Override</span>
<span class="source-line-no">3211</span><span id="line-3211"> public void abort(String reason, Throwable cause) {</span>
<span class="source-line-no">3212</span><span id="line-3212"> if (!setAbortRequested() || isStopped()) {</span>
<span class="source-line-no">3213</span><span id="line-3213"> LOG.debug("Abort called but aborted={}, stopped={}", isAborted(), isStopped());</span>
<span class="source-line-no">3214</span><span id="line-3214"> return;</span>
<span class="source-line-no">3215</span><span id="line-3215"> }</span>
<span class="source-line-no">3216</span><span id="line-3216"> if (cpHost != null) {</span>
<span class="source-line-no">3217</span><span id="line-3217"> // HBASE-4014: dump a list of loaded coprocessors.</span>
<span class="source-line-no">3218</span><span id="line-3218"> LOG.error(HBaseMarkers.FATAL,</span>
<span class="source-line-no">3219</span><span id="line-3219"> "Master server abort: loaded coprocessors are: " + getLoadedCoprocessors());</span>
<span class="source-line-no">3220</span><span id="line-3220"> }</span>
<span class="source-line-no">3221</span><span id="line-3221"> String msg = "***** ABORTING master " + this + ": " + reason + " *****";</span>
<span class="source-line-no">3222</span><span id="line-3222"> if (cause != null) {</span>
<span class="source-line-no">3223</span><span id="line-3223"> LOG.error(HBaseMarkers.FATAL, msg, cause);</span>
<span class="source-line-no">3224</span><span id="line-3224"> } else {</span>
<span class="source-line-no">3225</span><span id="line-3225"> LOG.error(HBaseMarkers.FATAL, msg);</span>
<span class="source-line-no">3226</span><span id="line-3226"> }</span>
<span class="source-line-no">3227</span><span id="line-3227"></span>
<span class="source-line-no">3228</span><span id="line-3228"> try {</span>
<span class="source-line-no">3229</span><span id="line-3229"> stopMaster();</span>
<span class="source-line-no">3230</span><span id="line-3230"> } catch (IOException e) {</span>
<span class="source-line-no">3231</span><span id="line-3231"> LOG.error("Exception occurred while stopping master", e);</span>
<span class="source-line-no">3232</span><span id="line-3232"> }</span>
<span class="source-line-no">3233</span><span id="line-3233"> }</span>
<span class="source-line-no">3234</span><span id="line-3234"></span>
<span class="source-line-no">3235</span><span id="line-3235"> @Override</span>
<span class="source-line-no">3236</span><span id="line-3236"> public MasterCoprocessorHost getMasterCoprocessorHost() {</span>
<span class="source-line-no">3237</span><span id="line-3237"> return cpHost;</span>
<span class="source-line-no">3238</span><span id="line-3238"> }</span>
<span class="source-line-no">3239</span><span id="line-3239"></span>
<span class="source-line-no">3240</span><span id="line-3240"> @Override</span>
<span class="source-line-no">3241</span><span id="line-3241"> public MasterQuotaManager getMasterQuotaManager() {</span>
<span class="source-line-no">3242</span><span id="line-3242"> return quotaManager;</span>
<span class="source-line-no">3243</span><span id="line-3243"> }</span>
<span class="source-line-no">3244</span><span id="line-3244"></span>
<span class="source-line-no">3245</span><span id="line-3245"> @Override</span>
<span class="source-line-no">3246</span><span id="line-3246"> public ProcedureExecutor&lt;MasterProcedureEnv&gt; getMasterProcedureExecutor() {</span>
<span class="source-line-no">3247</span><span id="line-3247"> return procedureExecutor;</span>
<span class="source-line-no">3248</span><span id="line-3248"> }</span>
<span class="source-line-no">3249</span><span id="line-3249"></span>
<span class="source-line-no">3250</span><span id="line-3250"> @Override</span>
<span class="source-line-no">3251</span><span id="line-3251"> public ServerName getServerName() {</span>
<span class="source-line-no">3252</span><span id="line-3252"> return this.serverName;</span>
<span class="source-line-no">3253</span><span id="line-3253"> }</span>
<span class="source-line-no">3254</span><span id="line-3254"></span>
<span class="source-line-no">3255</span><span id="line-3255"> @Override</span>
<span class="source-line-no">3256</span><span id="line-3256"> public AssignmentManager getAssignmentManager() {</span>
<span class="source-line-no">3257</span><span id="line-3257"> return this.assignmentManager;</span>
<span class="source-line-no">3258</span><span id="line-3258"> }</span>
<span class="source-line-no">3259</span><span id="line-3259"></span>
<span class="source-line-no">3260</span><span id="line-3260"> @Override</span>
<span class="source-line-no">3261</span><span id="line-3261"> public CatalogJanitor getCatalogJanitor() {</span>
<span class="source-line-no">3262</span><span id="line-3262"> return this.catalogJanitorChore;</span>
<span class="source-line-no">3263</span><span id="line-3263"> }</span>
<span class="source-line-no">3264</span><span id="line-3264"></span>
<span class="source-line-no">3265</span><span id="line-3265"> public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {</span>
<span class="source-line-no">3266</span><span id="line-3266"> return rsFatals;</span>
<span class="source-line-no">3267</span><span id="line-3267"> }</span>
<span class="source-line-no">3268</span><span id="line-3268"></span>
<span class="source-line-no">3269</span><span id="line-3269"> public TaskGroup getStartupProgress() {</span>
<span class="source-line-no">3270</span><span id="line-3270"> return startupTaskGroup;</span>
<span class="source-line-no">3271</span><span id="line-3271"> }</span>
<span class="source-line-no">3272</span><span id="line-3272"></span>
<span class="source-line-no">3273</span><span id="line-3273"> /**</span>
<span class="source-line-no">3274</span><span id="line-3274"> * Shutdown the cluster. Master runs a coordinated stop of all RegionServers and then itself.</span>
<span class="source-line-no">3275</span><span id="line-3275"> */</span>
<span class="source-line-no">3276</span><span id="line-3276"> public void shutdown() throws IOException {</span>
<span class="source-line-no">3277</span><span id="line-3277"> TraceUtil.trace(() -&gt; {</span>
<span class="source-line-no">3278</span><span id="line-3278"> if (cpHost != null) {</span>
<span class="source-line-no">3279</span><span id="line-3279"> cpHost.preShutdown();</span>
<span class="source-line-no">3280</span><span id="line-3280"> }</span>
<span class="source-line-no">3281</span><span id="line-3281"></span>
<span class="source-line-no">3282</span><span id="line-3282"> // Tell the servermanager cluster shutdown has been called. This makes it so when Master is</span>
<span class="source-line-no">3283</span><span id="line-3283"> // last running server, it'll stop itself. Next, we broadcast the cluster shutdown by setting</span>
<span class="source-line-no">3284</span><span id="line-3284"> // the cluster status as down. RegionServers will notice this change in state and will start</span>
<span class="source-line-no">3285</span><span id="line-3285"> // shutting themselves down. When last has exited, Master can go down.</span>
<span class="source-line-no">3286</span><span id="line-3286"> if (this.serverManager != null) {</span>
<span class="source-line-no">3287</span><span id="line-3287"> this.serverManager.shutdownCluster();</span>
<span class="source-line-no">3288</span><span id="line-3288"> }</span>
<span class="source-line-no">3289</span><span id="line-3289"> if (this.clusterStatusTracker != null) {</span>
<span class="source-line-no">3290</span><span id="line-3290"> try {</span>
<span class="source-line-no">3291</span><span id="line-3291"> this.clusterStatusTracker.setClusterDown();</span>
<span class="source-line-no">3292</span><span id="line-3292"> } catch (KeeperException e) {</span>
<span class="source-line-no">3293</span><span id="line-3293"> LOG.error("ZooKeeper exception trying to set cluster as down in ZK", e);</span>
<span class="source-line-no">3294</span><span id="line-3294"> }</span>
<span class="source-line-no">3295</span><span id="line-3295"> }</span>
<span class="source-line-no">3296</span><span id="line-3296"> // Stop the procedure executor. Will stop any ongoing assign, unassign, server crash etc.,</span>
<span class="source-line-no">3297</span><span id="line-3297"> // processing so we can go down.</span>
<span class="source-line-no">3298</span><span id="line-3298"> if (this.procedureExecutor != null) {</span>
<span class="source-line-no">3299</span><span id="line-3299"> this.procedureExecutor.stop();</span>
<span class="source-line-no">3300</span><span id="line-3300"> }</span>
<span class="source-line-no">3301</span><span id="line-3301"> // Shutdown our cluster connection. This will kill any hosted RPCs that might be going on;</span>
<span class="source-line-no">3302</span><span id="line-3302"> // this is what we want especially if the Master is in startup phase doing call outs to</span>
<span class="source-line-no">3303</span><span id="line-3303"> // hbase:meta, etc. when cluster is down. Without ths connection close, we'd have to wait on</span>
<span class="source-line-no">3304</span><span id="line-3304"> // the rpc to timeout.</span>
<span class="source-line-no">3305</span><span id="line-3305"> if (this.asyncClusterConnection != null) {</span>
<span class="source-line-no">3306</span><span id="line-3306"> this.asyncClusterConnection.close();</span>
<span class="source-line-no">3307</span><span id="line-3307"> }</span>
<span class="source-line-no">3308</span><span id="line-3308"> }, "HMaster.shutdown");</span>
<span class="source-line-no">3309</span><span id="line-3309"> }</span>
<span class="source-line-no">3310</span><span id="line-3310"></span>
<span class="source-line-no">3311</span><span id="line-3311"> public void stopMaster() throws IOException {</span>
<span class="source-line-no">3312</span><span id="line-3312"> if (cpHost != null) {</span>
<span class="source-line-no">3313</span><span id="line-3313"> cpHost.preStopMaster();</span>
<span class="source-line-no">3314</span><span id="line-3314"> }</span>
<span class="source-line-no">3315</span><span id="line-3315"> stop("Stopped by " + Thread.currentThread().getName());</span>
<span class="source-line-no">3316</span><span id="line-3316"> }</span>
<span class="source-line-no">3317</span><span id="line-3317"></span>
<span class="source-line-no">3318</span><span id="line-3318"> @Override</span>
<span class="source-line-no">3319</span><span id="line-3319"> public void stop(String msg) {</span>
<span class="source-line-no">3320</span><span id="line-3320"> if (!this.stopped) {</span>
<span class="source-line-no">3321</span><span id="line-3321"> LOG.info("***** STOPPING master '" + this + "' *****");</span>
<span class="source-line-no">3322</span><span id="line-3322"> this.stopped = true;</span>
<span class="source-line-no">3323</span><span id="line-3323"> LOG.info("STOPPED: " + msg);</span>
<span class="source-line-no">3324</span><span id="line-3324"> // Wakes run() if it is sleeping</span>
<span class="source-line-no">3325</span><span id="line-3325"> sleeper.skipSleepCycle();</span>
<span class="source-line-no">3326</span><span id="line-3326"> if (this.activeMasterManager != null) {</span>
<span class="source-line-no">3327</span><span id="line-3327"> this.activeMasterManager.stop();</span>
<span class="source-line-no">3328</span><span id="line-3328"> }</span>
<span class="source-line-no">3329</span><span id="line-3329"> }</span>
<span class="source-line-no">3330</span><span id="line-3330"> }</span>
<span class="source-line-no">3331</span><span id="line-3331"></span>
<span class="source-line-no">3332</span><span id="line-3332"> protected void checkServiceStarted() throws ServerNotRunningYetException {</span>
<span class="source-line-no">3333</span><span id="line-3333"> if (!serviceStarted) {</span>
<span class="source-line-no">3334</span><span id="line-3334"> throw new ServerNotRunningYetException("Server is not running yet");</span>
<span class="source-line-no">3335</span><span id="line-3335"> }</span>
<span class="source-line-no">3336</span><span id="line-3336"> }</span>
<span class="source-line-no">3337</span><span id="line-3337"></span>
<span class="source-line-no">3338</span><span id="line-3338"> void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException,</span>
<span class="source-line-no">3339</span><span id="line-3339"> MasterNotRunningException, MasterStoppedException {</span>
<span class="source-line-no">3340</span><span id="line-3340"> checkServiceStarted();</span>
<span class="source-line-no">3341</span><span id="line-3341"> if (!isInitialized()) {</span>
<span class="source-line-no">3342</span><span id="line-3342"> throw new PleaseHoldException("Master is initializing");</span>
<span class="source-line-no">3343</span><span id="line-3343"> }</span>
<span class="source-line-no">3344</span><span id="line-3344"> if (isStopped()) {</span>
<span class="source-line-no">3345</span><span id="line-3345"> throw new MasterStoppedException();</span>
<span class="source-line-no">3346</span><span id="line-3346"> }</span>
<span class="source-line-no">3347</span><span id="line-3347"> }</span>
<span class="source-line-no">3348</span><span id="line-3348"></span>
<span class="source-line-no">3349</span><span id="line-3349"> /**</span>
<span class="source-line-no">3350</span><span id="line-3350"> * Report whether this master is currently the active master or not. If not active master, we are</span>
<span class="source-line-no">3351</span><span id="line-3351"> * parked on ZK waiting to become active. This method is used for testing.</span>
<span class="source-line-no">3352</span><span id="line-3352"> * @return true if active master, false if not.</span>
<span class="source-line-no">3353</span><span id="line-3353"> */</span>
<span class="source-line-no">3354</span><span id="line-3354"> @Override</span>
<span class="source-line-no">3355</span><span id="line-3355"> public boolean isActiveMaster() {</span>
<span class="source-line-no">3356</span><span id="line-3356"> return activeMaster;</span>
<span class="source-line-no">3357</span><span id="line-3357"> }</span>
<span class="source-line-no">3358</span><span id="line-3358"></span>
<span class="source-line-no">3359</span><span id="line-3359"> /**</span>
<span class="source-line-no">3360</span><span id="line-3360"> * Report whether this master has completed with its initialization and is ready. If ready, the</span>
<span class="source-line-no">3361</span><span id="line-3361"> * master is also the active master. A standby master is never ready. This method is used for</span>
<span class="source-line-no">3362</span><span id="line-3362"> * testing.</span>
<span class="source-line-no">3363</span><span id="line-3363"> * @return true if master is ready to go, false if not.</span>
<span class="source-line-no">3364</span><span id="line-3364"> */</span>
<span class="source-line-no">3365</span><span id="line-3365"> @Override</span>
<span class="source-line-no">3366</span><span id="line-3366"> public boolean isInitialized() {</span>
<span class="source-line-no">3367</span><span id="line-3367"> return initialized.isReady();</span>
<span class="source-line-no">3368</span><span id="line-3368"> }</span>
<span class="source-line-no">3369</span><span id="line-3369"></span>
<span class="source-line-no">3370</span><span id="line-3370"> /**</span>
<span class="source-line-no">3371</span><span id="line-3371"> * Report whether this master is started This method is used for testing.</span>
<span class="source-line-no">3372</span><span id="line-3372"> * @return true if master is ready to go, false if not.</span>
<span class="source-line-no">3373</span><span id="line-3373"> */</span>
<span class="source-line-no">3374</span><span id="line-3374"> public boolean isOnline() {</span>
<span class="source-line-no">3375</span><span id="line-3375"> return serviceStarted;</span>
<span class="source-line-no">3376</span><span id="line-3376"> }</span>
<span class="source-line-no">3377</span><span id="line-3377"></span>
<span class="source-line-no">3378</span><span id="line-3378"> /**</span>
<span class="source-line-no">3379</span><span id="line-3379"> * Report whether this master is in maintenance mode.</span>
<span class="source-line-no">3380</span><span id="line-3380"> * @return true if master is in maintenanceMode</span>
<span class="source-line-no">3381</span><span id="line-3381"> */</span>
<span class="source-line-no">3382</span><span id="line-3382"> @Override</span>
<span class="source-line-no">3383</span><span id="line-3383"> public boolean isInMaintenanceMode() {</span>
<span class="source-line-no">3384</span><span id="line-3384"> return maintenanceMode;</span>
<span class="source-line-no">3385</span><span id="line-3385"> }</span>
<span class="source-line-no">3386</span><span id="line-3386"></span>
<span class="source-line-no">3387</span><span id="line-3387"> public void setInitialized(boolean isInitialized) {</span>
<span class="source-line-no">3388</span><span id="line-3388"> procedureExecutor.getEnvironment().setEventReady(initialized, isInitialized);</span>
<span class="source-line-no">3389</span><span id="line-3389"> }</span>
<span class="source-line-no">3390</span><span id="line-3390"></span>
<span class="source-line-no">3391</span><span id="line-3391"> /**</span>
<span class="source-line-no">3392</span><span id="line-3392"> * Mainly used in procedure related tests, where we will restart ProcedureExecutor and</span>
<span class="source-line-no">3393</span><span id="line-3393"> * AssignmentManager, but we do not want to restart master(to speed up the test), so we need to</span>
<span class="source-line-no">3394</span><span id="line-3394"> * disable rpc for a while otherwise some critical rpc requests such as</span>
<span class="source-line-no">3395</span><span id="line-3395"> * reportRegionStateTransition could fail and cause region server to abort.</span>
<span class="source-line-no">3396</span><span id="line-3396"> */</span>
<span class="source-line-no">3397</span><span id="line-3397"> @RestrictedApi(explanation = "Should only be called in tests", link = "",</span>
<span class="source-line-no">3398</span><span id="line-3398"> allowedOnPath = ".*/src/test/.*")</span>
<span class="source-line-no">3399</span><span id="line-3399"> public void setServiceStarted(boolean started) {</span>
<span class="source-line-no">3400</span><span id="line-3400"> this.serviceStarted = started;</span>
<span class="source-line-no">3401</span><span id="line-3401"> }</span>
<span class="source-line-no">3402</span><span id="line-3402"></span>
<span class="source-line-no">3403</span><span id="line-3403"> @Override</span>
<span class="source-line-no">3404</span><span id="line-3404"> public ProcedureEvent&lt;?&gt; getInitializedEvent() {</span>
<span class="source-line-no">3405</span><span id="line-3405"> return initialized;</span>
<span class="source-line-no">3406</span><span id="line-3406"> }</span>
<span class="source-line-no">3407</span><span id="line-3407"></span>
<span class="source-line-no">3408</span><span id="line-3408"> /**</span>
<span class="source-line-no">3409</span><span id="line-3409"> * Compute the average load across all region servers. Currently, this uses a very naive</span>
<span class="source-line-no">3410</span><span id="line-3410"> * computation - just uses the number of regions being served, ignoring stats about number of</span>
<span class="source-line-no">3411</span><span id="line-3411"> * requests.</span>
<span class="source-line-no">3412</span><span id="line-3412"> * @return the average load</span>
<span class="source-line-no">3413</span><span id="line-3413"> */</span>
<span class="source-line-no">3414</span><span id="line-3414"> public double getAverageLoad() {</span>
<span class="source-line-no">3415</span><span id="line-3415"> if (this.assignmentManager == null) {</span>
<span class="source-line-no">3416</span><span id="line-3416"> return 0;</span>
<span class="source-line-no">3417</span><span id="line-3417"> }</span>
<span class="source-line-no">3418</span><span id="line-3418"></span>
<span class="source-line-no">3419</span><span id="line-3419"> RegionStates regionStates = this.assignmentManager.getRegionStates();</span>
<span class="source-line-no">3420</span><span id="line-3420"> if (regionStates == null) {</span>
<span class="source-line-no">3421</span><span id="line-3421"> return 0;</span>
<span class="source-line-no">3422</span><span id="line-3422"> }</span>
<span class="source-line-no">3423</span><span id="line-3423"> return regionStates.getAverageLoad();</span>
<span class="source-line-no">3424</span><span id="line-3424"> }</span>
<span class="source-line-no">3425</span><span id="line-3425"></span>
<span class="source-line-no">3426</span><span id="line-3426"> @Override</span>
<span class="source-line-no">3427</span><span id="line-3427"> public boolean registerService(Service instance) {</span>
<span class="source-line-no">3428</span><span id="line-3428"> /*</span>
<span class="source-line-no">3429</span><span id="line-3429"> * No stacking of instances is allowed for a single service name</span>
<span class="source-line-no">3430</span><span id="line-3430"> */</span>
<span class="source-line-no">3431</span><span id="line-3431"> Descriptors.ServiceDescriptor serviceDesc = instance.getDescriptorForType();</span>
<span class="source-line-no">3432</span><span id="line-3432"> String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc);</span>
<span class="source-line-no">3433</span><span id="line-3433"> if (coprocessorServiceHandlers.containsKey(serviceName)) {</span>
<span class="source-line-no">3434</span><span id="line-3434"> LOG.error("Coprocessor service " + serviceName</span>
<span class="source-line-no">3435</span><span id="line-3435"> + " already registered, rejecting request from " + instance);</span>
<span class="source-line-no">3436</span><span id="line-3436"> return false;</span>
<span class="source-line-no">3437</span><span id="line-3437"> }</span>
<span class="source-line-no">3438</span><span id="line-3438"></span>
<span class="source-line-no">3439</span><span id="line-3439"> coprocessorServiceHandlers.put(serviceName, instance);</span>
<span class="source-line-no">3440</span><span id="line-3440"> if (LOG.isDebugEnabled()) {</span>
<span class="source-line-no">3441</span><span id="line-3441"> LOG.debug("Registered master coprocessor service: service=" + serviceName);</span>
<span class="source-line-no">3442</span><span id="line-3442"> }</span>
<span class="source-line-no">3443</span><span id="line-3443"> return true;</span>
<span class="source-line-no">3444</span><span id="line-3444"> }</span>
<span class="source-line-no">3445</span><span id="line-3445"></span>
<span class="source-line-no">3446</span><span id="line-3446"> /**</span>
<span class="source-line-no">3447</span><span id="line-3447"> * Utility for constructing an instance of the passed HMaster class.</span>
<span class="source-line-no">3448</span><span id="line-3448"> * @return HMaster instance.</span>
<span class="source-line-no">3449</span><span id="line-3449"> */</span>
<span class="source-line-no">3450</span><span id="line-3450"> public static HMaster constructMaster(Class&lt;? extends HMaster&gt; masterClass,</span>
<span class="source-line-no">3451</span><span id="line-3451"> final Configuration conf) {</span>
<span class="source-line-no">3452</span><span id="line-3452"> try {</span>
<span class="source-line-no">3453</span><span id="line-3453"> Constructor&lt;? extends HMaster&gt; c = masterClass.getConstructor(Configuration.class);</span>
<span class="source-line-no">3454</span><span id="line-3454"> return c.newInstance(conf);</span>
<span class="source-line-no">3455</span><span id="line-3455"> } catch (Exception e) {</span>
<span class="source-line-no">3456</span><span id="line-3456"> Throwable error = e;</span>
<span class="source-line-no">3457</span><span id="line-3457"> if (</span>
<span class="source-line-no">3458</span><span id="line-3458"> e instanceof InvocationTargetException</span>
<span class="source-line-no">3459</span><span id="line-3459"> &amp;&amp; ((InvocationTargetException) e).getTargetException() != null</span>
<span class="source-line-no">3460</span><span id="line-3460"> ) {</span>
<span class="source-line-no">3461</span><span id="line-3461"> error = ((InvocationTargetException) e).getTargetException();</span>
<span class="source-line-no">3462</span><span id="line-3462"> }</span>
<span class="source-line-no">3463</span><span id="line-3463"> throw new RuntimeException("Failed construction of Master: " + masterClass.toString() + ". ",</span>
<span class="source-line-no">3464</span><span id="line-3464"> error);</span>
<span class="source-line-no">3465</span><span id="line-3465"> }</span>
<span class="source-line-no">3466</span><span id="line-3466"> }</span>
<span class="source-line-no">3467</span><span id="line-3467"></span>
<span class="source-line-no">3468</span><span id="line-3468"> /**</span>
<span class="source-line-no">3469</span><span id="line-3469"> * @see org.apache.hadoop.hbase.master.HMasterCommandLine</span>
<span class="source-line-no">3470</span><span id="line-3470"> */</span>
<span class="source-line-no">3471</span><span id="line-3471"> public static void main(String[] args) {</span>
<span class="source-line-no">3472</span><span id="line-3472"> LOG.info("STARTING service " + HMaster.class.getSimpleName());</span>
<span class="source-line-no">3473</span><span id="line-3473"> VersionInfo.logVersion();</span>
<span class="source-line-no">3474</span><span id="line-3474"> new HMasterCommandLine(HMaster.class).doMain(args);</span>
<span class="source-line-no">3475</span><span id="line-3475"> }</span>
<span class="source-line-no">3476</span><span id="line-3476"></span>
<span class="source-line-no">3477</span><span id="line-3477"> public HFileCleaner getHFileCleaner() {</span>
<span class="source-line-no">3478</span><span id="line-3478"> return this.hfileCleaners.get(0);</span>
<span class="source-line-no">3479</span><span id="line-3479"> }</span>
<span class="source-line-no">3480</span><span id="line-3480"></span>
<span class="source-line-no">3481</span><span id="line-3481"> public List&lt;HFileCleaner&gt; getHFileCleaners() {</span>
<span class="source-line-no">3482</span><span id="line-3482"> return this.hfileCleaners;</span>
<span class="source-line-no">3483</span><span id="line-3483"> }</span>
<span class="source-line-no">3484</span><span id="line-3484"></span>
<span class="source-line-no">3485</span><span id="line-3485"> public LogCleaner getLogCleaner() {</span>
<span class="source-line-no">3486</span><span id="line-3486"> return this.logCleaner;</span>
<span class="source-line-no">3487</span><span id="line-3487"> }</span>
<span class="source-line-no">3488</span><span id="line-3488"></span>
<span class="source-line-no">3489</span><span id="line-3489"> /** Returns the underlying snapshot manager */</span>
<span class="source-line-no">3490</span><span id="line-3490"> @Override</span>
<span class="source-line-no">3491</span><span id="line-3491"> public SnapshotManager getSnapshotManager() {</span>
<span class="source-line-no">3492</span><span id="line-3492"> return this.snapshotManager;</span>
<span class="source-line-no">3493</span><span id="line-3493"> }</span>
<span class="source-line-no">3494</span><span id="line-3494"></span>
<span class="source-line-no">3495</span><span id="line-3495"> /** Returns the underlying MasterProcedureManagerHost */</span>
<span class="source-line-no">3496</span><span id="line-3496"> @Override</span>
<span class="source-line-no">3497</span><span id="line-3497"> public MasterProcedureManagerHost getMasterProcedureManagerHost() {</span>
<span class="source-line-no">3498</span><span id="line-3498"> return mpmHost;</span>
<span class="source-line-no">3499</span><span id="line-3499"> }</span>
<span class="source-line-no">3500</span><span id="line-3500"></span>
<span class="source-line-no">3501</span><span id="line-3501"> @Override</span>
<span class="source-line-no">3502</span><span id="line-3502"> public ClusterSchema getClusterSchema() {</span>
<span class="source-line-no">3503</span><span id="line-3503"> return this.clusterSchemaService;</span>
<span class="source-line-no">3504</span><span id="line-3504"> }</span>
<span class="source-line-no">3505</span><span id="line-3505"></span>
<span class="source-line-no">3506</span><span id="line-3506"> /**</span>
<span class="source-line-no">3507</span><span id="line-3507"> * Create a new Namespace.</span>
<span class="source-line-no">3508</span><span id="line-3508"> * @param namespaceDescriptor descriptor for new Namespace</span>
<span class="source-line-no">3509</span><span id="line-3509"> * @param nonceGroup Identifier for the source of the request, a client or process.</span>
<span class="source-line-no">3510</span><span id="line-3510"> * @param nonce A unique identifier for this operation from the client or process</span>
<span class="source-line-no">3511</span><span id="line-3511"> * identified by &lt;code&gt;nonceGroup&lt;/code&gt; (the source must ensure each</span>
<span class="source-line-no">3512</span><span id="line-3512"> * operation gets a unique id).</span>
<span class="source-line-no">3513</span><span id="line-3513"> * @return procedure id</span>
<span class="source-line-no">3514</span><span id="line-3514"> */</span>
<span class="source-line-no">3515</span><span id="line-3515"> long createNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,</span>
<span class="source-line-no">3516</span><span id="line-3516"> final long nonce) throws IOException {</span>
<span class="source-line-no">3517</span><span id="line-3517"> checkInitialized();</span>
<span class="source-line-no">3518</span><span id="line-3518"></span>
<span class="source-line-no">3519</span><span id="line-3519"> TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));</span>
<span class="source-line-no">3520</span><span id="line-3520"></span>
<span class="source-line-no">3521</span><span id="line-3521"> return MasterProcedureUtil</span>
<span class="source-line-no">3522</span><span id="line-3522"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">3523</span><span id="line-3523"> @Override</span>
<span class="source-line-no">3524</span><span id="line-3524"> protected void run() throws IOException {</span>
<span class="source-line-no">3525</span><span id="line-3525"> getMaster().getMasterCoprocessorHost().preCreateNamespace(namespaceDescriptor);</span>
<span class="source-line-no">3526</span><span id="line-3526"> // We need to wait for the procedure to potentially fail due to "prepare" sanity</span>
<span class="source-line-no">3527</span><span id="line-3527"> // checks. This will block only the beginning of the procedure. See HBASE-19953.</span>
<span class="source-line-no">3528</span><span id="line-3528"> ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();</span>
<span class="source-line-no">3529</span><span id="line-3529"> LOG.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor);</span>
<span class="source-line-no">3530</span><span id="line-3530"> // Execute the operation synchronously - wait for the operation to complete before</span>
<span class="source-line-no">3531</span><span id="line-3531"> // continuing.</span>
<span class="source-line-no">3532</span><span id="line-3532"> setProcId(getClusterSchema().createNamespace(namespaceDescriptor, getNonceKey(), latch));</span>
<span class="source-line-no">3533</span><span id="line-3533"> latch.await();</span>
<span class="source-line-no">3534</span><span id="line-3534"> getMaster().getMasterCoprocessorHost().postCreateNamespace(namespaceDescriptor);</span>
<span class="source-line-no">3535</span><span id="line-3535"> }</span>
<span class="source-line-no">3536</span><span id="line-3536"></span>
<span class="source-line-no">3537</span><span id="line-3537"> @Override</span>
<span class="source-line-no">3538</span><span id="line-3538"> protected String getDescription() {</span>
<span class="source-line-no">3539</span><span id="line-3539"> return "CreateNamespaceProcedure";</span>
<span class="source-line-no">3540</span><span id="line-3540"> }</span>
<span class="source-line-no">3541</span><span id="line-3541"> });</span>
<span class="source-line-no">3542</span><span id="line-3542"> }</span>
<span class="source-line-no">3543</span><span id="line-3543"></span>
<span class="source-line-no">3544</span><span id="line-3544"> /**</span>
<span class="source-line-no">3545</span><span id="line-3545"> * Modify an existing Namespace.</span>
<span class="source-line-no">3546</span><span id="line-3546"> * @param nonceGroup Identifier for the source of the request, a client or process.</span>
<span class="source-line-no">3547</span><span id="line-3547"> * @param nonce A unique identifier for this operation from the client or process identified</span>
<span class="source-line-no">3548</span><span id="line-3548"> * by &lt;code&gt;nonceGroup&lt;/code&gt; (the source must ensure each operation gets a</span>
<span class="source-line-no">3549</span><span id="line-3549"> * unique id).</span>
<span class="source-line-no">3550</span><span id="line-3550"> * @return procedure id</span>
<span class="source-line-no">3551</span><span id="line-3551"> */</span>
<span class="source-line-no">3552</span><span id="line-3552"> long modifyNamespace(final NamespaceDescriptor newNsDescriptor, final long nonceGroup,</span>
<span class="source-line-no">3553</span><span id="line-3553"> final long nonce) throws IOException {</span>
<span class="source-line-no">3554</span><span id="line-3554"> checkInitialized();</span>
<span class="source-line-no">3555</span><span id="line-3555"></span>
<span class="source-line-no">3556</span><span id="line-3556"> TableName.isLegalNamespaceName(Bytes.toBytes(newNsDescriptor.getName()));</span>
<span class="source-line-no">3557</span><span id="line-3557"></span>
<span class="source-line-no">3558</span><span id="line-3558"> return MasterProcedureUtil</span>
<span class="source-line-no">3559</span><span id="line-3559"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">3560</span><span id="line-3560"> @Override</span>
<span class="source-line-no">3561</span><span id="line-3561"> protected void run() throws IOException {</span>
<span class="source-line-no">3562</span><span id="line-3562"> NamespaceDescriptor oldNsDescriptor = getNamespace(newNsDescriptor.getName());</span>
<span class="source-line-no">3563</span><span id="line-3563"> getMaster().getMasterCoprocessorHost().preModifyNamespace(oldNsDescriptor,</span>
<span class="source-line-no">3564</span><span id="line-3564"> newNsDescriptor);</span>
<span class="source-line-no">3565</span><span id="line-3565"> // We need to wait for the procedure to potentially fail due to "prepare" sanity</span>
<span class="source-line-no">3566</span><span id="line-3566"> // checks. This will block only the beginning of the procedure. See HBASE-19953.</span>
<span class="source-line-no">3567</span><span id="line-3567"> ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();</span>
<span class="source-line-no">3568</span><span id="line-3568"> LOG.info(getClientIdAuditPrefix() + " modify " + newNsDescriptor);</span>
<span class="source-line-no">3569</span><span id="line-3569"> // Execute the operation synchronously - wait for the operation to complete before</span>
<span class="source-line-no">3570</span><span id="line-3570"> // continuing.</span>
<span class="source-line-no">3571</span><span id="line-3571"> setProcId(getClusterSchema().modifyNamespace(newNsDescriptor, getNonceKey(), latch));</span>
<span class="source-line-no">3572</span><span id="line-3572"> latch.await();</span>
<span class="source-line-no">3573</span><span id="line-3573"> getMaster().getMasterCoprocessorHost().postModifyNamespace(oldNsDescriptor,</span>
<span class="source-line-no">3574</span><span id="line-3574"> newNsDescriptor);</span>
<span class="source-line-no">3575</span><span id="line-3575"> }</span>
<span class="source-line-no">3576</span><span id="line-3576"></span>
<span class="source-line-no">3577</span><span id="line-3577"> @Override</span>
<span class="source-line-no">3578</span><span id="line-3578"> protected String getDescription() {</span>
<span class="source-line-no">3579</span><span id="line-3579"> return "ModifyNamespaceProcedure";</span>
<span class="source-line-no">3580</span><span id="line-3580"> }</span>
<span class="source-line-no">3581</span><span id="line-3581"> });</span>
<span class="source-line-no">3582</span><span id="line-3582"> }</span>
<span class="source-line-no">3583</span><span id="line-3583"></span>
<span class="source-line-no">3584</span><span id="line-3584"> /**</span>
<span class="source-line-no">3585</span><span id="line-3585"> * Delete an existing Namespace. Only empty Namespaces (no tables) can be removed.</span>
<span class="source-line-no">3586</span><span id="line-3586"> * @param nonceGroup Identifier for the source of the request, a client or process.</span>
<span class="source-line-no">3587</span><span id="line-3587"> * @param nonce A unique identifier for this operation from the client or process identified</span>
<span class="source-line-no">3588</span><span id="line-3588"> * by &lt;code&gt;nonceGroup&lt;/code&gt; (the source must ensure each operation gets a</span>
<span class="source-line-no">3589</span><span id="line-3589"> * unique id).</span>
<span class="source-line-no">3590</span><span id="line-3590"> * @return procedure id</span>
<span class="source-line-no">3591</span><span id="line-3591"> */</span>
<span class="source-line-no">3592</span><span id="line-3592"> long deleteNamespace(final String name, final long nonceGroup, final long nonce)</span>
<span class="source-line-no">3593</span><span id="line-3593"> throws IOException {</span>
<span class="source-line-no">3594</span><span id="line-3594"> checkInitialized();</span>
<span class="source-line-no">3595</span><span id="line-3595"></span>
<span class="source-line-no">3596</span><span id="line-3596"> return MasterProcedureUtil</span>
<span class="source-line-no">3597</span><span id="line-3597"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">3598</span><span id="line-3598"> @Override</span>
<span class="source-line-no">3599</span><span id="line-3599"> protected void run() throws IOException {</span>
<span class="source-line-no">3600</span><span id="line-3600"> getMaster().getMasterCoprocessorHost().preDeleteNamespace(name);</span>
<span class="source-line-no">3601</span><span id="line-3601"> LOG.info(getClientIdAuditPrefix() + " delete " + name);</span>
<span class="source-line-no">3602</span><span id="line-3602"> // Execute the operation synchronously - wait for the operation to complete before</span>
<span class="source-line-no">3603</span><span id="line-3603"> // continuing.</span>
<span class="source-line-no">3604</span><span id="line-3604"> //</span>
<span class="source-line-no">3605</span><span id="line-3605"> // We need to wait for the procedure to potentially fail due to "prepare" sanity</span>
<span class="source-line-no">3606</span><span id="line-3606"> // checks. This will block only the beginning of the procedure. See HBASE-19953.</span>
<span class="source-line-no">3607</span><span id="line-3607"> ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();</span>
<span class="source-line-no">3608</span><span id="line-3608"> setProcId(submitProcedure(</span>
<span class="source-line-no">3609</span><span id="line-3609"> new DeleteNamespaceProcedure(procedureExecutor.getEnvironment(), name, latch)));</span>
<span class="source-line-no">3610</span><span id="line-3610"> latch.await();</span>
<span class="source-line-no">3611</span><span id="line-3611"> // Will not be invoked in the face of Exception thrown by the Procedure's execution</span>
<span class="source-line-no">3612</span><span id="line-3612"> getMaster().getMasterCoprocessorHost().postDeleteNamespace(name);</span>
<span class="source-line-no">3613</span><span id="line-3613"> }</span>
<span class="source-line-no">3614</span><span id="line-3614"></span>
<span class="source-line-no">3615</span><span id="line-3615"> @Override</span>
<span class="source-line-no">3616</span><span id="line-3616"> protected String getDescription() {</span>
<span class="source-line-no">3617</span><span id="line-3617"> return "DeleteNamespaceProcedure";</span>
<span class="source-line-no">3618</span><span id="line-3618"> }</span>
<span class="source-line-no">3619</span><span id="line-3619"> });</span>
<span class="source-line-no">3620</span><span id="line-3620"> }</span>
<span class="source-line-no">3621</span><span id="line-3621"></span>
<span class="source-line-no">3622</span><span id="line-3622"> /**</span>
<span class="source-line-no">3623</span><span id="line-3623"> * Get a Namespace</span>
<span class="source-line-no">3624</span><span id="line-3624"> * @param name Name of the Namespace</span>
<span class="source-line-no">3625</span><span id="line-3625"> * @return Namespace descriptor for &lt;code&gt;name&lt;/code&gt;</span>
<span class="source-line-no">3626</span><span id="line-3626"> */</span>
<span class="source-line-no">3627</span><span id="line-3627"> NamespaceDescriptor getNamespace(String name) throws IOException {</span>
<span class="source-line-no">3628</span><span id="line-3628"> checkInitialized();</span>
<span class="source-line-no">3629</span><span id="line-3629"> if (this.cpHost != null) this.cpHost.preGetNamespaceDescriptor(name);</span>
<span class="source-line-no">3630</span><span id="line-3630"> NamespaceDescriptor nsd = this.clusterSchemaService.getNamespace(name);</span>
<span class="source-line-no">3631</span><span id="line-3631"> if (this.cpHost != null) this.cpHost.postGetNamespaceDescriptor(nsd);</span>
<span class="source-line-no">3632</span><span id="line-3632"> return nsd;</span>
<span class="source-line-no">3633</span><span id="line-3633"> }</span>
<span class="source-line-no">3634</span><span id="line-3634"></span>
<span class="source-line-no">3635</span><span id="line-3635"> /**</span>
<span class="source-line-no">3636</span><span id="line-3636"> * Get all Namespaces</span>
<span class="source-line-no">3637</span><span id="line-3637"> * @return All Namespace descriptors</span>
<span class="source-line-no">3638</span><span id="line-3638"> */</span>
<span class="source-line-no">3639</span><span id="line-3639"> List&lt;NamespaceDescriptor&gt; getNamespaces() throws IOException {</span>
<span class="source-line-no">3640</span><span id="line-3640"> checkInitialized();</span>
<span class="source-line-no">3641</span><span id="line-3641"> final List&lt;NamespaceDescriptor&gt; nsds = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">3642</span><span id="line-3642"> if (cpHost != null) {</span>
<span class="source-line-no">3643</span><span id="line-3643"> cpHost.preListNamespaceDescriptors(nsds);</span>
<span class="source-line-no">3644</span><span id="line-3644"> }</span>
<span class="source-line-no">3645</span><span id="line-3645"> nsds.addAll(this.clusterSchemaService.getNamespaces());</span>
<span class="source-line-no">3646</span><span id="line-3646"> if (this.cpHost != null) {</span>
<span class="source-line-no">3647</span><span id="line-3647"> this.cpHost.postListNamespaceDescriptors(nsds);</span>
<span class="source-line-no">3648</span><span id="line-3648"> }</span>
<span class="source-line-no">3649</span><span id="line-3649"> return nsds;</span>
<span class="source-line-no">3650</span><span id="line-3650"> }</span>
<span class="source-line-no">3651</span><span id="line-3651"></span>
<span class="source-line-no">3652</span><span id="line-3652"> /**</span>
<span class="source-line-no">3653</span><span id="line-3653"> * List namespace names</span>
<span class="source-line-no">3654</span><span id="line-3654"> * @return All namespace names</span>
<span class="source-line-no">3655</span><span id="line-3655"> */</span>
<span class="source-line-no">3656</span><span id="line-3656"> public List&lt;String&gt; listNamespaces() throws IOException {</span>
<span class="source-line-no">3657</span><span id="line-3657"> checkInitialized();</span>
<span class="source-line-no">3658</span><span id="line-3658"> List&lt;String&gt; namespaces = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">3659</span><span id="line-3659"> if (cpHost != null) {</span>
<span class="source-line-no">3660</span><span id="line-3660"> cpHost.preListNamespaces(namespaces);</span>
<span class="source-line-no">3661</span><span id="line-3661"> }</span>
<span class="source-line-no">3662</span><span id="line-3662"> for (NamespaceDescriptor namespace : clusterSchemaService.getNamespaces()) {</span>
<span class="source-line-no">3663</span><span id="line-3663"> namespaces.add(namespace.getName());</span>
<span class="source-line-no">3664</span><span id="line-3664"> }</span>
<span class="source-line-no">3665</span><span id="line-3665"> if (cpHost != null) {</span>
<span class="source-line-no">3666</span><span id="line-3666"> cpHost.postListNamespaces(namespaces);</span>
<span class="source-line-no">3667</span><span id="line-3667"> }</span>
<span class="source-line-no">3668</span><span id="line-3668"> return namespaces;</span>
<span class="source-line-no">3669</span><span id="line-3669"> }</span>
<span class="source-line-no">3670</span><span id="line-3670"></span>
<span class="source-line-no">3671</span><span id="line-3671"> @Override</span>
<span class="source-line-no">3672</span><span id="line-3672"> public List&lt;TableName&gt; listTableNamesByNamespace(String name) throws IOException {</span>
<span class="source-line-no">3673</span><span id="line-3673"> checkInitialized();</span>
<span class="source-line-no">3674</span><span id="line-3674"> return listTableNames(name, null, true);</span>
<span class="source-line-no">3675</span><span id="line-3675"> }</span>
<span class="source-line-no">3676</span><span id="line-3676"></span>
<span class="source-line-no">3677</span><span id="line-3677"> @Override</span>
<span class="source-line-no">3678</span><span id="line-3678"> public List&lt;TableDescriptor&gt; listTableDescriptorsByNamespace(String name) throws IOException {</span>
<span class="source-line-no">3679</span><span id="line-3679"> checkInitialized();</span>
<span class="source-line-no">3680</span><span id="line-3680"> return listTableDescriptors(name, null, null, true);</span>
<span class="source-line-no">3681</span><span id="line-3681"> }</span>
<span class="source-line-no">3682</span><span id="line-3682"></span>
<span class="source-line-no">3683</span><span id="line-3683"> @Override</span>
<span class="source-line-no">3684</span><span id="line-3684"> public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)</span>
<span class="source-line-no">3685</span><span id="line-3685"> throws IOException {</span>
<span class="source-line-no">3686</span><span id="line-3686"> if (cpHost != null) {</span>
<span class="source-line-no">3687</span><span id="line-3687"> cpHost.preAbortProcedure(this.procedureExecutor, procId);</span>
<span class="source-line-no">3688</span><span id="line-3688"> }</span>
<span class="source-line-no">3689</span><span id="line-3689"></span>
<span class="source-line-no">3690</span><span id="line-3690"> final boolean result = this.procedureExecutor.abort(procId, mayInterruptIfRunning);</span>
<span class="source-line-no">3691</span><span id="line-3691"></span>
<span class="source-line-no">3692</span><span id="line-3692"> if (cpHost != null) {</span>
<span class="source-line-no">3693</span><span id="line-3693"> cpHost.postAbortProcedure();</span>
<span class="source-line-no">3694</span><span id="line-3694"> }</span>
<span class="source-line-no">3695</span><span id="line-3695"></span>
<span class="source-line-no">3696</span><span id="line-3696"> return result;</span>
<span class="source-line-no">3697</span><span id="line-3697"> }</span>
<span class="source-line-no">3698</span><span id="line-3698"></span>
<span class="source-line-no">3699</span><span id="line-3699"> @Override</span>
<span class="source-line-no">3700</span><span id="line-3700"> public List&lt;Procedure&lt;?&gt;&gt; getProcedures() throws IOException {</span>
<span class="source-line-no">3701</span><span id="line-3701"> if (cpHost != null) {</span>
<span class="source-line-no">3702</span><span id="line-3702"> cpHost.preGetProcedures();</span>
<span class="source-line-no">3703</span><span id="line-3703"> }</span>
<span class="source-line-no">3704</span><span id="line-3704"></span>
<span class="source-line-no">3705</span><span id="line-3705"> @SuppressWarnings({ "unchecked", "rawtypes" })</span>
<span class="source-line-no">3706</span><span id="line-3706"> List&lt;Procedure&lt;?&gt;&gt; procList = (List) this.procedureExecutor.getProcedures();</span>
<span class="source-line-no">3707</span><span id="line-3707"></span>
<span class="source-line-no">3708</span><span id="line-3708"> if (cpHost != null) {</span>
<span class="source-line-no">3709</span><span id="line-3709"> cpHost.postGetProcedures(procList);</span>
<span class="source-line-no">3710</span><span id="line-3710"> }</span>
<span class="source-line-no">3711</span><span id="line-3711"></span>
<span class="source-line-no">3712</span><span id="line-3712"> return procList;</span>
<span class="source-line-no">3713</span><span id="line-3713"> }</span>
<span class="source-line-no">3714</span><span id="line-3714"></span>
<span class="source-line-no">3715</span><span id="line-3715"> @Override</span>
<span class="source-line-no">3716</span><span id="line-3716"> public List&lt;LockedResource&gt; getLocks() throws IOException {</span>
<span class="source-line-no">3717</span><span id="line-3717"> if (cpHost != null) {</span>
<span class="source-line-no">3718</span><span id="line-3718"> cpHost.preGetLocks();</span>
<span class="source-line-no">3719</span><span id="line-3719"> }</span>
<span class="source-line-no">3720</span><span id="line-3720"></span>
<span class="source-line-no">3721</span><span id="line-3721"> MasterProcedureScheduler procedureScheduler =</span>
<span class="source-line-no">3722</span><span id="line-3722"> procedureExecutor.getEnvironment().getProcedureScheduler();</span>
<span class="source-line-no">3723</span><span id="line-3723"></span>
<span class="source-line-no">3724</span><span id="line-3724"> final List&lt;LockedResource&gt; lockedResources = procedureScheduler.getLocks();</span>
<span class="source-line-no">3725</span><span id="line-3725"></span>
<span class="source-line-no">3726</span><span id="line-3726"> if (cpHost != null) {</span>
<span class="source-line-no">3727</span><span id="line-3727"> cpHost.postGetLocks(lockedResources);</span>
<span class="source-line-no">3728</span><span id="line-3728"> }</span>
<span class="source-line-no">3729</span><span id="line-3729"></span>
<span class="source-line-no">3730</span><span id="line-3730"> return lockedResources;</span>
<span class="source-line-no">3731</span><span id="line-3731"> }</span>
<span class="source-line-no">3732</span><span id="line-3732"></span>
<span class="source-line-no">3733</span><span id="line-3733"> /**</span>
<span class="source-line-no">3734</span><span id="line-3734"> * Returns the list of table descriptors that match the specified request</span>
<span class="source-line-no">3735</span><span id="line-3735"> * @param namespace the namespace to query, or null if querying for all</span>
<span class="source-line-no">3736</span><span id="line-3736"> * @param regex The regular expression to match against, or null if querying for all</span>
<span class="source-line-no">3737</span><span id="line-3737"> * @param tableNameList the list of table names, or null if querying for all</span>
<span class="source-line-no">3738</span><span id="line-3738"> * @param includeSysTables False to match only against userspace tables</span>
<span class="source-line-no">3739</span><span id="line-3739"> * @return the list of table descriptors</span>
<span class="source-line-no">3740</span><span id="line-3740"> */</span>
<span class="source-line-no">3741</span><span id="line-3741"> public List&lt;TableDescriptor&gt; listTableDescriptors(final String namespace, final String regex,</span>
<span class="source-line-no">3742</span><span id="line-3742"> final List&lt;TableName&gt; tableNameList, final boolean includeSysTables) throws IOException {</span>
<span class="source-line-no">3743</span><span id="line-3743"> List&lt;TableDescriptor&gt; htds = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">3744</span><span id="line-3744"> if (cpHost != null) {</span>
<span class="source-line-no">3745</span><span id="line-3745"> cpHost.preGetTableDescriptors(tableNameList, htds, regex);</span>
<span class="source-line-no">3746</span><span id="line-3746"> }</span>
<span class="source-line-no">3747</span><span id="line-3747"> htds = getTableDescriptors(htds, namespace, regex, tableNameList, includeSysTables);</span>
<span class="source-line-no">3748</span><span id="line-3748"> if (cpHost != null) {</span>
<span class="source-line-no">3749</span><span id="line-3749"> cpHost.postGetTableDescriptors(tableNameList, htds, regex);</span>
<span class="source-line-no">3750</span><span id="line-3750"> }</span>
<span class="source-line-no">3751</span><span id="line-3751"> return htds;</span>
<span class="source-line-no">3752</span><span id="line-3752"> }</span>
<span class="source-line-no">3753</span><span id="line-3753"></span>
<span class="source-line-no">3754</span><span id="line-3754"> /**</span>
<span class="source-line-no">3755</span><span id="line-3755"> * Returns the list of table names that match the specified request</span>
<span class="source-line-no">3756</span><span id="line-3756"> * @param regex The regular expression to match against, or null if querying for all</span>
<span class="source-line-no">3757</span><span id="line-3757"> * @param namespace the namespace to query, or null if querying for all</span>
<span class="source-line-no">3758</span><span id="line-3758"> * @param includeSysTables False to match only against userspace tables</span>
<span class="source-line-no">3759</span><span id="line-3759"> * @return the list of table names</span>
<span class="source-line-no">3760</span><span id="line-3760"> */</span>
<span class="source-line-no">3761</span><span id="line-3761"> public List&lt;TableName&gt; listTableNames(final String namespace, final String regex,</span>
<span class="source-line-no">3762</span><span id="line-3762"> final boolean includeSysTables) throws IOException {</span>
<span class="source-line-no">3763</span><span id="line-3763"> List&lt;TableDescriptor&gt; htds = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">3764</span><span id="line-3764"> if (cpHost != null) {</span>
<span class="source-line-no">3765</span><span id="line-3765"> cpHost.preGetTableNames(htds, regex);</span>
<span class="source-line-no">3766</span><span id="line-3766"> }</span>
<span class="source-line-no">3767</span><span id="line-3767"> htds = getTableDescriptors(htds, namespace, regex, null, includeSysTables);</span>
<span class="source-line-no">3768</span><span id="line-3768"> if (cpHost != null) {</span>
<span class="source-line-no">3769</span><span id="line-3769"> cpHost.postGetTableNames(htds, regex);</span>
<span class="source-line-no">3770</span><span id="line-3770"> }</span>
<span class="source-line-no">3771</span><span id="line-3771"> List&lt;TableName&gt; result = new ArrayList&lt;&gt;(htds.size());</span>
<span class="source-line-no">3772</span><span id="line-3772"> for (TableDescriptor htd : htds)</span>
<span class="source-line-no">3773</span><span id="line-3773"> result.add(htd.getTableName());</span>
<span class="source-line-no">3774</span><span id="line-3774"> return result;</span>
<span class="source-line-no">3775</span><span id="line-3775"> }</span>
<span class="source-line-no">3776</span><span id="line-3776"></span>
<span class="source-line-no">3777</span><span id="line-3777"> /**</span>
<span class="source-line-no">3778</span><span id="line-3778"> * Return a list of table table descriptors after applying any provided filter parameters. Note</span>
<span class="source-line-no">3779</span><span id="line-3779"> * that the user-facing description of this filter logic is presented on the class-level javadoc</span>
<span class="source-line-no">3780</span><span id="line-3780"> * of {@link NormalizeTableFilterParams}.</span>
<span class="source-line-no">3781</span><span id="line-3781"> */</span>
<span class="source-line-no">3782</span><span id="line-3782"> private List&lt;TableDescriptor&gt; getTableDescriptors(final List&lt;TableDescriptor&gt; htds,</span>
<span class="source-line-no">3783</span><span id="line-3783"> final String namespace, final String regex, final List&lt;TableName&gt; tableNameList,</span>
<span class="source-line-no">3784</span><span id="line-3784"> final boolean includeSysTables) throws IOException {</span>
<span class="source-line-no">3785</span><span id="line-3785"> if (tableNameList == null || tableNameList.isEmpty()) {</span>
<span class="source-line-no">3786</span><span id="line-3786"> // request for all TableDescriptors</span>
<span class="source-line-no">3787</span><span id="line-3787"> Collection&lt;TableDescriptor&gt; allHtds;</span>
<span class="source-line-no">3788</span><span id="line-3788"> if (namespace != null &amp;&amp; namespace.length() &gt; 0) {</span>
<span class="source-line-no">3789</span><span id="line-3789"> // Do a check on the namespace existence. Will fail if does not exist.</span>
<span class="source-line-no">3790</span><span id="line-3790"> this.clusterSchemaService.getNamespace(namespace);</span>
<span class="source-line-no">3791</span><span id="line-3791"> allHtds = tableDescriptors.getByNamespace(namespace).values();</span>
<span class="source-line-no">3792</span><span id="line-3792"> } else {</span>
<span class="source-line-no">3793</span><span id="line-3793"> allHtds = tableDescriptors.getAll().values();</span>
<span class="source-line-no">3794</span><span id="line-3794"> }</span>
<span class="source-line-no">3795</span><span id="line-3795"> for (TableDescriptor desc : allHtds) {</span>
<span class="source-line-no">3796</span><span id="line-3796"> if (</span>
<span class="source-line-no">3797</span><span id="line-3797"> tableStateManager.isTablePresent(desc.getTableName())</span>
<span class="source-line-no">3798</span><span id="line-3798"> &amp;&amp; (includeSysTables || !desc.getTableName().isSystemTable())</span>
<span class="source-line-no">3799</span><span id="line-3799"> ) {</span>
<span class="source-line-no">3800</span><span id="line-3800"> htds.add(desc);</span>
<span class="source-line-no">3801</span><span id="line-3801"> }</span>
<span class="source-line-no">3802</span><span id="line-3802"> }</span>
<span class="source-line-no">3803</span><span id="line-3803"> } else {</span>
<span class="source-line-no">3804</span><span id="line-3804"> for (TableName s : tableNameList) {</span>
<span class="source-line-no">3805</span><span id="line-3805"> if (tableStateManager.isTablePresent(s)) {</span>
<span class="source-line-no">3806</span><span id="line-3806"> TableDescriptor desc = tableDescriptors.get(s);</span>
<span class="source-line-no">3807</span><span id="line-3807"> if (desc != null) {</span>
<span class="source-line-no">3808</span><span id="line-3808"> htds.add(desc);</span>
<span class="source-line-no">3809</span><span id="line-3809"> }</span>
<span class="source-line-no">3810</span><span id="line-3810"> }</span>
<span class="source-line-no">3811</span><span id="line-3811"> }</span>
<span class="source-line-no">3812</span><span id="line-3812"> }</span>
<span class="source-line-no">3813</span><span id="line-3813"></span>
<span class="source-line-no">3814</span><span id="line-3814"> // Retains only those matched by regular expression.</span>
<span class="source-line-no">3815</span><span id="line-3815"> if (regex != null) filterTablesByRegex(htds, Pattern.compile(regex));</span>
<span class="source-line-no">3816</span><span id="line-3816"> return htds;</span>
<span class="source-line-no">3817</span><span id="line-3817"> }</span>
<span class="source-line-no">3818</span><span id="line-3818"></span>
<span class="source-line-no">3819</span><span id="line-3819"> /**</span>
<span class="source-line-no">3820</span><span id="line-3820"> * Removes the table descriptors that don't match the pattern.</span>
<span class="source-line-no">3821</span><span id="line-3821"> * @param descriptors list of table descriptors to filter</span>
<span class="source-line-no">3822</span><span id="line-3822"> * @param pattern the regex to use</span>
<span class="source-line-no">3823</span><span id="line-3823"> */</span>
<span class="source-line-no">3824</span><span id="line-3824"> private static void filterTablesByRegex(final Collection&lt;TableDescriptor&gt; descriptors,</span>
<span class="source-line-no">3825</span><span id="line-3825"> final Pattern pattern) {</span>
<span class="source-line-no">3826</span><span id="line-3826"> final String defaultNS = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR;</span>
<span class="source-line-no">3827</span><span id="line-3827"> Iterator&lt;TableDescriptor&gt; itr = descriptors.iterator();</span>
<span class="source-line-no">3828</span><span id="line-3828"> while (itr.hasNext()) {</span>
<span class="source-line-no">3829</span><span id="line-3829"> TableDescriptor htd = itr.next();</span>
<span class="source-line-no">3830</span><span id="line-3830"> String tableName = htd.getTableName().getNameAsString();</span>
<span class="source-line-no">3831</span><span id="line-3831"> boolean matched = pattern.matcher(tableName).matches();</span>
<span class="source-line-no">3832</span><span id="line-3832"> if (!matched &amp;&amp; htd.getTableName().getNamespaceAsString().equals(defaultNS)) {</span>
<span class="source-line-no">3833</span><span id="line-3833"> matched = pattern.matcher(defaultNS + TableName.NAMESPACE_DELIM + tableName).matches();</span>
<span class="source-line-no">3834</span><span id="line-3834"> }</span>
<span class="source-line-no">3835</span><span id="line-3835"> if (!matched) {</span>
<span class="source-line-no">3836</span><span id="line-3836"> itr.remove();</span>
<span class="source-line-no">3837</span><span id="line-3837"> }</span>
<span class="source-line-no">3838</span><span id="line-3838"> }</span>
<span class="source-line-no">3839</span><span id="line-3839"> }</span>
<span class="source-line-no">3840</span><span id="line-3840"></span>
<span class="source-line-no">3841</span><span id="line-3841"> @Override</span>
<span class="source-line-no">3842</span><span id="line-3842"> public long getLastMajorCompactionTimestamp(TableName table) throws IOException {</span>
<span class="source-line-no">3843</span><span id="line-3843"> return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))</span>
<span class="source-line-no">3844</span><span id="line-3844"> .getLastMajorCompactionTimestamp(table);</span>
<span class="source-line-no">3845</span><span id="line-3845"> }</span>
<span class="source-line-no">3846</span><span id="line-3846"></span>
<span class="source-line-no">3847</span><span id="line-3847"> @Override</span>
<span class="source-line-no">3848</span><span id="line-3848"> public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {</span>
<span class="source-line-no">3849</span><span id="line-3849"> return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))</span>
<span class="source-line-no">3850</span><span id="line-3850"> .getLastMajorCompactionTimestamp(regionName);</span>
<span class="source-line-no">3851</span><span id="line-3851"> }</span>
<span class="source-line-no">3852</span><span id="line-3852"></span>
<span class="source-line-no">3853</span><span id="line-3853"> /**</span>
<span class="source-line-no">3854</span><span id="line-3854"> * Gets the mob file compaction state for a specific table. Whether all the mob files are selected</span>
<span class="source-line-no">3855</span><span id="line-3855"> * is known during the compaction execution, but the statistic is done just before compaction</span>
<span class="source-line-no">3856</span><span id="line-3856"> * starts, it is hard to know the compaction type at that time, so the rough statistics are chosen</span>
<span class="source-line-no">3857</span><span id="line-3857"> * for the mob file compaction. Only two compaction states are available,</span>
<span class="source-line-no">3858</span><span id="line-3858"> * CompactionState.MAJOR_AND_MINOR and CompactionState.NONE.</span>
<span class="source-line-no">3859</span><span id="line-3859"> * @param tableName The current table name.</span>
<span class="source-line-no">3860</span><span id="line-3860"> * @return If a given table is in mob file compaction now.</span>
<span class="source-line-no">3861</span><span id="line-3861"> */</span>
<span class="source-line-no">3862</span><span id="line-3862"> public GetRegionInfoResponse.CompactionState getMobCompactionState(TableName tableName) {</span>
<span class="source-line-no">3863</span><span id="line-3863"> AtomicInteger compactionsCount = mobCompactionStates.get(tableName);</span>
<span class="source-line-no">3864</span><span id="line-3864"> if (compactionsCount != null &amp;&amp; compactionsCount.get() != 0) {</span>
<span class="source-line-no">3865</span><span id="line-3865"> return GetRegionInfoResponse.CompactionState.MAJOR_AND_MINOR;</span>
<span class="source-line-no">3866</span><span id="line-3866"> }</span>
<span class="source-line-no">3867</span><span id="line-3867"> return GetRegionInfoResponse.CompactionState.NONE;</span>
<span class="source-line-no">3868</span><span id="line-3868"> }</span>
<span class="source-line-no">3869</span><span id="line-3869"></span>
<span class="source-line-no">3870</span><span id="line-3870"> public void reportMobCompactionStart(TableName tableName) throws IOException {</span>
<span class="source-line-no">3871</span><span id="line-3871"> IdLock.Entry lockEntry = null;</span>
<span class="source-line-no">3872</span><span id="line-3872"> try {</span>
<span class="source-line-no">3873</span><span id="line-3873"> lockEntry = mobCompactionLock.getLockEntry(tableName.hashCode());</span>
<span class="source-line-no">3874</span><span id="line-3874"> AtomicInteger compactionsCount = mobCompactionStates.get(tableName);</span>
<span class="source-line-no">3875</span><span id="line-3875"> if (compactionsCount == null) {</span>
<span class="source-line-no">3876</span><span id="line-3876"> compactionsCount = new AtomicInteger(0);</span>
<span class="source-line-no">3877</span><span id="line-3877"> mobCompactionStates.put(tableName, compactionsCount);</span>
<span class="source-line-no">3878</span><span id="line-3878"> }</span>
<span class="source-line-no">3879</span><span id="line-3879"> compactionsCount.incrementAndGet();</span>
<span class="source-line-no">3880</span><span id="line-3880"> } finally {</span>
<span class="source-line-no">3881</span><span id="line-3881"> if (lockEntry != null) {</span>
<span class="source-line-no">3882</span><span id="line-3882"> mobCompactionLock.releaseLockEntry(lockEntry);</span>
<span class="source-line-no">3883</span><span id="line-3883"> }</span>
<span class="source-line-no">3884</span><span id="line-3884"> }</span>
<span class="source-line-no">3885</span><span id="line-3885"> }</span>
<span class="source-line-no">3886</span><span id="line-3886"></span>
<span class="source-line-no">3887</span><span id="line-3887"> public void reportMobCompactionEnd(TableName tableName) throws IOException {</span>
<span class="source-line-no">3888</span><span id="line-3888"> IdLock.Entry lockEntry = null;</span>
<span class="source-line-no">3889</span><span id="line-3889"> try {</span>
<span class="source-line-no">3890</span><span id="line-3890"> lockEntry = mobCompactionLock.getLockEntry(tableName.hashCode());</span>
<span class="source-line-no">3891</span><span id="line-3891"> AtomicInteger compactionsCount = mobCompactionStates.get(tableName);</span>
<span class="source-line-no">3892</span><span id="line-3892"> if (compactionsCount != null) {</span>
<span class="source-line-no">3893</span><span id="line-3893"> int count = compactionsCount.decrementAndGet();</span>
<span class="source-line-no">3894</span><span id="line-3894"> // remove the entry if the count is 0.</span>
<span class="source-line-no">3895</span><span id="line-3895"> if (count == 0) {</span>
<span class="source-line-no">3896</span><span id="line-3896"> mobCompactionStates.remove(tableName);</span>
<span class="source-line-no">3897</span><span id="line-3897"> }</span>
<span class="source-line-no">3898</span><span id="line-3898"> }</span>
<span class="source-line-no">3899</span><span id="line-3899"> } finally {</span>
<span class="source-line-no">3900</span><span id="line-3900"> if (lockEntry != null) {</span>
<span class="source-line-no">3901</span><span id="line-3901"> mobCompactionLock.releaseLockEntry(lockEntry);</span>
<span class="source-line-no">3902</span><span id="line-3902"> }</span>
<span class="source-line-no">3903</span><span id="line-3903"> }</span>
<span class="source-line-no">3904</span><span id="line-3904"> }</span>
<span class="source-line-no">3905</span><span id="line-3905"></span>
<span class="source-line-no">3906</span><span id="line-3906"> /**</span>
<span class="source-line-no">3907</span><span id="line-3907"> * Queries the state of the {@link LoadBalancerStateStore}. If the balancer is not initialized,</span>
<span class="source-line-no">3908</span><span id="line-3908"> * false is returned.</span>
<span class="source-line-no">3909</span><span id="line-3909"> * @return The state of the load balancer, or false if the load balancer isn't defined.</span>
<span class="source-line-no">3910</span><span id="line-3910"> */</span>
<span class="source-line-no">3911</span><span id="line-3911"> public boolean isBalancerOn() {</span>
<span class="source-line-no">3912</span><span id="line-3912"> return !isInMaintenanceMode() &amp;&amp; loadBalancerStateStore != null &amp;&amp; loadBalancerStateStore.get();</span>
<span class="source-line-no">3913</span><span id="line-3913"> }</span>
<span class="source-line-no">3914</span><span id="line-3914"></span>
<span class="source-line-no">3915</span><span id="line-3915"> /**</span>
<span class="source-line-no">3916</span><span id="line-3916"> * Queries the state of the {@link RegionNormalizerStateStore}. If it's not initialized, false is</span>
<span class="source-line-no">3917</span><span id="line-3917"> * returned.</span>
<span class="source-line-no">3918</span><span id="line-3918"> */</span>
<span class="source-line-no">3919</span><span id="line-3919"> public boolean isNormalizerOn() {</span>
<span class="source-line-no">3920</span><span id="line-3920"> return !isInMaintenanceMode() &amp;&amp; getRegionNormalizerManager().isNormalizerOn();</span>
<span class="source-line-no">3921</span><span id="line-3921"> }</span>
<span class="source-line-no">3922</span><span id="line-3922"></span>
<span class="source-line-no">3923</span><span id="line-3923"> /**</span>
<span class="source-line-no">3924</span><span id="line-3924"> * Queries the state of the {@link SplitOrMergeStateStore}. If it is not initialized, false is</span>
<span class="source-line-no">3925</span><span id="line-3925"> * returned. If switchType is illegal, false will return.</span>
<span class="source-line-no">3926</span><span id="line-3926"> * @param switchType see {@link org.apache.hadoop.hbase.client.MasterSwitchType}</span>
<span class="source-line-no">3927</span><span id="line-3927"> * @return The state of the switch</span>
<span class="source-line-no">3928</span><span id="line-3928"> */</span>
<span class="source-line-no">3929</span><span id="line-3929"> @Override</span>
<span class="source-line-no">3930</span><span id="line-3930"> public boolean isSplitOrMergeEnabled(MasterSwitchType switchType) {</span>
<span class="source-line-no">3931</span><span id="line-3931"> return !isInMaintenanceMode() &amp;&amp; splitOrMergeStateStore != null</span>
<span class="source-line-no">3932</span><span id="line-3932"> &amp;&amp; splitOrMergeStateStore.isSplitOrMergeEnabled(switchType);</span>
<span class="source-line-no">3933</span><span id="line-3933"> }</span>
<span class="source-line-no">3934</span><span id="line-3934"></span>
<span class="source-line-no">3935</span><span id="line-3935"> /**</span>
<span class="source-line-no">3936</span><span id="line-3936"> * Fetch the configured {@link LoadBalancer} class name. If none is set, a default is returned.</span>
<span class="source-line-no">3937</span><span id="line-3937"> * &lt;p/&gt;</span>
<span class="source-line-no">3938</span><span id="line-3938"> * Notice that, the base load balancer will always be {@link RSGroupBasedLoadBalancer} now, so</span>
<span class="source-line-no">3939</span><span id="line-3939"> * this method will return the balancer used inside each rs group.</span>
<span class="source-line-no">3940</span><span id="line-3940"> * @return The name of the {@link LoadBalancer} in use.</span>
<span class="source-line-no">3941</span><span id="line-3941"> */</span>
<span class="source-line-no">3942</span><span id="line-3942"> public String getLoadBalancerClassName() {</span>
<span class="source-line-no">3943</span><span id="line-3943"> return conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,</span>
<span class="source-line-no">3944</span><span id="line-3944"> LoadBalancerFactory.getDefaultLoadBalancerClass().getName());</span>
<span class="source-line-no">3945</span><span id="line-3945"> }</span>
<span class="source-line-no">3946</span><span id="line-3946"></span>
<span class="source-line-no">3947</span><span id="line-3947"> public SplitOrMergeStateStore getSplitOrMergeStateStore() {</span>
<span class="source-line-no">3948</span><span id="line-3948"> return splitOrMergeStateStore;</span>
<span class="source-line-no">3949</span><span id="line-3949"> }</span>
<span class="source-line-no">3950</span><span id="line-3950"></span>
<span class="source-line-no">3951</span><span id="line-3951"> @Override</span>
<span class="source-line-no">3952</span><span id="line-3952"> public RSGroupBasedLoadBalancer getLoadBalancer() {</span>
<span class="source-line-no">3953</span><span id="line-3953"> return balancer;</span>
<span class="source-line-no">3954</span><span id="line-3954"> }</span>
<span class="source-line-no">3955</span><span id="line-3955"></span>
<span class="source-line-no">3956</span><span id="line-3956"> @Override</span>
<span class="source-line-no">3957</span><span id="line-3957"> public FavoredNodesManager getFavoredNodesManager() {</span>
<span class="source-line-no">3958</span><span id="line-3958"> return balancer.getFavoredNodesManager();</span>
<span class="source-line-no">3959</span><span id="line-3959"> }</span>
<span class="source-line-no">3960</span><span id="line-3960"></span>
<span class="source-line-no">3961</span><span id="line-3961"> private long executePeerProcedure(AbstractPeerProcedure&lt;?&gt; procedure) throws IOException {</span>
<span class="source-line-no">3962</span><span id="line-3962"> if (!isReplicationPeerModificationEnabled()) {</span>
<span class="source-line-no">3963</span><span id="line-3963"> throw new IOException("Replication peer modification disabled");</span>
<span class="source-line-no">3964</span><span id="line-3964"> }</span>
<span class="source-line-no">3965</span><span id="line-3965"> long procId = procedureExecutor.submitProcedure(procedure);</span>
<span class="source-line-no">3966</span><span id="line-3966"> procedure.getLatch().await();</span>
<span class="source-line-no">3967</span><span id="line-3967"> return procId;</span>
<span class="source-line-no">3968</span><span id="line-3968"> }</span>
<span class="source-line-no">3969</span><span id="line-3969"></span>
<span class="source-line-no">3970</span><span id="line-3970"> @Override</span>
<span class="source-line-no">3971</span><span id="line-3971"> public long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)</span>
<span class="source-line-no">3972</span><span id="line-3972"> throws ReplicationException, IOException {</span>
<span class="source-line-no">3973</span><span id="line-3973"> LOG.info(getClientIdAuditPrefix() + " creating replication peer, id=" + peerId + ", config="</span>
<span class="source-line-no">3974</span><span id="line-3974"> + peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"));</span>
<span class="source-line-no">3975</span><span id="line-3975"> return executePeerProcedure(new AddPeerProcedure(peerId, peerConfig, enabled));</span>
<span class="source-line-no">3976</span><span id="line-3976"> }</span>
<span class="source-line-no">3977</span><span id="line-3977"></span>
<span class="source-line-no">3978</span><span id="line-3978"> @Override</span>
<span class="source-line-no">3979</span><span id="line-3979"> public long removeReplicationPeer(String peerId) throws ReplicationException, IOException {</span>
<span class="source-line-no">3980</span><span id="line-3980"> LOG.info(getClientIdAuditPrefix() + " removing replication peer, id=" + peerId);</span>
<span class="source-line-no">3981</span><span id="line-3981"> return executePeerProcedure(new RemovePeerProcedure(peerId));</span>
<span class="source-line-no">3982</span><span id="line-3982"> }</span>
<span class="source-line-no">3983</span><span id="line-3983"></span>
<span class="source-line-no">3984</span><span id="line-3984"> @Override</span>
<span class="source-line-no">3985</span><span id="line-3985"> public long enableReplicationPeer(String peerId) throws ReplicationException, IOException {</span>
<span class="source-line-no">3986</span><span id="line-3986"> LOG.info(getClientIdAuditPrefix() + " enable replication peer, id=" + peerId);</span>
<span class="source-line-no">3987</span><span id="line-3987"> return executePeerProcedure(new EnablePeerProcedure(peerId));</span>
<span class="source-line-no">3988</span><span id="line-3988"> }</span>
<span class="source-line-no">3989</span><span id="line-3989"></span>
<span class="source-line-no">3990</span><span id="line-3990"> @Override</span>
<span class="source-line-no">3991</span><span id="line-3991"> public long disableReplicationPeer(String peerId) throws ReplicationException, IOException {</span>
<span class="source-line-no">3992</span><span id="line-3992"> LOG.info(getClientIdAuditPrefix() + " disable replication peer, id=" + peerId);</span>
<span class="source-line-no">3993</span><span id="line-3993"> return executePeerProcedure(new DisablePeerProcedure(peerId));</span>
<span class="source-line-no">3994</span><span id="line-3994"> }</span>
<span class="source-line-no">3995</span><span id="line-3995"></span>
<span class="source-line-no">3996</span><span id="line-3996"> @Override</span>
<span class="source-line-no">3997</span><span id="line-3997"> public ReplicationPeerConfig getReplicationPeerConfig(String peerId)</span>
<span class="source-line-no">3998</span><span id="line-3998"> throws ReplicationException, IOException {</span>
<span class="source-line-no">3999</span><span id="line-3999"> if (cpHost != null) {</span>
<span class="source-line-no">4000</span><span id="line-4000"> cpHost.preGetReplicationPeerConfig(peerId);</span>
<span class="source-line-no">4001</span><span id="line-4001"> }</span>
<span class="source-line-no">4002</span><span id="line-4002"> LOG.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId);</span>
<span class="source-line-no">4003</span><span id="line-4003"> ReplicationPeerConfig peerConfig = this.replicationPeerManager.getPeerConfig(peerId)</span>
<span class="source-line-no">4004</span><span id="line-4004"> .orElseThrow(() -&gt; new ReplicationPeerNotFoundException(peerId));</span>
<span class="source-line-no">4005</span><span id="line-4005"> if (cpHost != null) {</span>
<span class="source-line-no">4006</span><span id="line-4006"> cpHost.postGetReplicationPeerConfig(peerId);</span>
<span class="source-line-no">4007</span><span id="line-4007"> }</span>
<span class="source-line-no">4008</span><span id="line-4008"> return peerConfig;</span>
<span class="source-line-no">4009</span><span id="line-4009"> }</span>
<span class="source-line-no">4010</span><span id="line-4010"></span>
<span class="source-line-no">4011</span><span id="line-4011"> @Override</span>
<span class="source-line-no">4012</span><span id="line-4012"> public long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)</span>
<span class="source-line-no">4013</span><span id="line-4013"> throws ReplicationException, IOException {</span>
<span class="source-line-no">4014</span><span id="line-4014"> LOG.info(getClientIdAuditPrefix() + " update replication peer config, id=" + peerId</span>
<span class="source-line-no">4015</span><span id="line-4015"> + ", config=" + peerConfig);</span>
<span class="source-line-no">4016</span><span id="line-4016"> return executePeerProcedure(new UpdatePeerConfigProcedure(peerId, peerConfig));</span>
<span class="source-line-no">4017</span><span id="line-4017"> }</span>
<span class="source-line-no">4018</span><span id="line-4018"></span>
<span class="source-line-no">4019</span><span id="line-4019"> @Override</span>
<span class="source-line-no">4020</span><span id="line-4020"> public List&lt;ReplicationPeerDescription&gt; listReplicationPeers(String regex)</span>
<span class="source-line-no">4021</span><span id="line-4021"> throws ReplicationException, IOException {</span>
<span class="source-line-no">4022</span><span id="line-4022"> if (cpHost != null) {</span>
<span class="source-line-no">4023</span><span id="line-4023"> cpHost.preListReplicationPeers(regex);</span>
<span class="source-line-no">4024</span><span id="line-4024"> }</span>
<span class="source-line-no">4025</span><span id="line-4025"> LOG.debug("{} list replication peers, regex={}", getClientIdAuditPrefix(), regex);</span>
<span class="source-line-no">4026</span><span id="line-4026"> Pattern pattern = regex == null ? null : Pattern.compile(regex);</span>
<span class="source-line-no">4027</span><span id="line-4027"> List&lt;ReplicationPeerDescription&gt; peers = this.replicationPeerManager.listPeers(pattern);</span>
<span class="source-line-no">4028</span><span id="line-4028"> if (cpHost != null) {</span>
<span class="source-line-no">4029</span><span id="line-4029"> cpHost.postListReplicationPeers(regex);</span>
<span class="source-line-no">4030</span><span id="line-4030"> }</span>
<span class="source-line-no">4031</span><span id="line-4031"> return peers;</span>
<span class="source-line-no">4032</span><span id="line-4032"> }</span>
<span class="source-line-no">4033</span><span id="line-4033"></span>
<span class="source-line-no">4034</span><span id="line-4034"> @Override</span>
<span class="source-line-no">4035</span><span id="line-4035"> public long transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state)</span>
<span class="source-line-no">4036</span><span id="line-4036"> throws ReplicationException, IOException {</span>
<span class="source-line-no">4037</span><span id="line-4037"> LOG.info(</span>
<span class="source-line-no">4038</span><span id="line-4038"> getClientIdAuditPrefix()</span>
<span class="source-line-no">4039</span><span id="line-4039"> + " transit current cluster state to {} in a synchronous replication peer id={}",</span>
<span class="source-line-no">4040</span><span id="line-4040"> state, peerId);</span>
<span class="source-line-no">4041</span><span id="line-4041"> return executePeerProcedure(new TransitPeerSyncReplicationStateProcedure(peerId, state));</span>
<span class="source-line-no">4042</span><span id="line-4042"> }</span>
<span class="source-line-no">4043</span><span id="line-4043"></span>
<span class="source-line-no">4044</span><span id="line-4044"> @Override</span>
<span class="source-line-no">4045</span><span id="line-4045"> public boolean replicationPeerModificationSwitch(boolean on) throws IOException {</span>
<span class="source-line-no">4046</span><span id="line-4046"> return replicationPeerModificationStateStore.set(on);</span>
<span class="source-line-no">4047</span><span id="line-4047"> }</span>
<span class="source-line-no">4048</span><span id="line-4048"></span>
<span class="source-line-no">4049</span><span id="line-4049"> @Override</span>
<span class="source-line-no">4050</span><span id="line-4050"> public boolean isReplicationPeerModificationEnabled() {</span>
<span class="source-line-no">4051</span><span id="line-4051"> return replicationPeerModificationStateStore.get();</span>
<span class="source-line-no">4052</span><span id="line-4052"> }</span>
<span class="source-line-no">4053</span><span id="line-4053"></span>
<span class="source-line-no">4054</span><span id="line-4054"> /**</span>
<span class="source-line-no">4055</span><span id="line-4055"> * Mark region server(s) as decommissioned (previously called 'draining') to prevent additional</span>
<span class="source-line-no">4056</span><span id="line-4056"> * regions from getting assigned to them. Also unload the regions on the servers asynchronously.0</span>
<span class="source-line-no">4057</span><span id="line-4057"> * @param servers Region servers to decommission.</span>
<span class="source-line-no">4058</span><span id="line-4058"> */</span>
<span class="source-line-no">4059</span><span id="line-4059"> public void decommissionRegionServers(final List&lt;ServerName&gt; servers, final boolean offload)</span>
<span class="source-line-no">4060</span><span id="line-4060"> throws IOException {</span>
<span class="source-line-no">4061</span><span id="line-4061"> List&lt;ServerName&gt; serversAdded = new ArrayList&lt;&gt;(servers.size());</span>
<span class="source-line-no">4062</span><span id="line-4062"> // Place the decommission marker first.</span>
<span class="source-line-no">4063</span><span id="line-4063"> String parentZnode = getZooKeeper().getZNodePaths().drainingZNode;</span>
<span class="source-line-no">4064</span><span id="line-4064"> for (ServerName server : servers) {</span>
<span class="source-line-no">4065</span><span id="line-4065"> try {</span>
<span class="source-line-no">4066</span><span id="line-4066"> String node = ZNodePaths.joinZNode(parentZnode, server.getServerName());</span>
<span class="source-line-no">4067</span><span id="line-4067"> ZKUtil.createAndFailSilent(getZooKeeper(), node);</span>
<span class="source-line-no">4068</span><span id="line-4068"> } catch (KeeperException ke) {</span>
<span class="source-line-no">4069</span><span id="line-4069"> throw new HBaseIOException(</span>
<span class="source-line-no">4070</span><span id="line-4070"> this.zooKeeper.prefix("Unable to decommission '" + server.getServerName() + "'."), ke);</span>
<span class="source-line-no">4071</span><span id="line-4071"> }</span>
<span class="source-line-no">4072</span><span id="line-4072"> if (this.serverManager.addServerToDrainList(server)) {</span>
<span class="source-line-no">4073</span><span id="line-4073"> serversAdded.add(server);</span>
<span class="source-line-no">4074</span><span id="line-4074"> }</span>
<span class="source-line-no">4075</span><span id="line-4075"> }</span>
<span class="source-line-no">4076</span><span id="line-4076"> // Move the regions off the decommissioned servers.</span>
<span class="source-line-no">4077</span><span id="line-4077"> if (offload) {</span>
<span class="source-line-no">4078</span><span id="line-4078"> final List&lt;ServerName&gt; destServers = this.serverManager.createDestinationServersList();</span>
<span class="source-line-no">4079</span><span id="line-4079"> for (ServerName server : serversAdded) {</span>
<span class="source-line-no">4080</span><span id="line-4080"> final List&lt;RegionInfo&gt; regionsOnServer = this.assignmentManager.getRegionsOnServer(server);</span>
<span class="source-line-no">4081</span><span id="line-4081"> for (RegionInfo hri : regionsOnServer) {</span>
<span class="source-line-no">4082</span><span id="line-4082"> ServerName dest = balancer.randomAssignment(hri, destServers);</span>
<span class="source-line-no">4083</span><span id="line-4083"> if (dest == null) {</span>
<span class="source-line-no">4084</span><span id="line-4084"> throw new HBaseIOException("Unable to determine a plan to move " + hri);</span>
<span class="source-line-no">4085</span><span id="line-4085"> }</span>
<span class="source-line-no">4086</span><span id="line-4086"> RegionPlan rp = new RegionPlan(hri, server, dest);</span>
<span class="source-line-no">4087</span><span id="line-4087"> this.assignmentManager.moveAsync(rp);</span>
<span class="source-line-no">4088</span><span id="line-4088"> }</span>
<span class="source-line-no">4089</span><span id="line-4089"> }</span>
<span class="source-line-no">4090</span><span id="line-4090"> }</span>
<span class="source-line-no">4091</span><span id="line-4091"> }</span>
<span class="source-line-no">4092</span><span id="line-4092"></span>
<span class="source-line-no">4093</span><span id="line-4093"> /**</span>
<span class="source-line-no">4094</span><span id="line-4094"> * List region servers marked as decommissioned (previously called 'draining') to not get regions</span>
<span class="source-line-no">4095</span><span id="line-4095"> * assigned to them.</span>
<span class="source-line-no">4096</span><span id="line-4096"> * @return List of decommissioned servers.</span>
<span class="source-line-no">4097</span><span id="line-4097"> */</span>
<span class="source-line-no">4098</span><span id="line-4098"> public List&lt;ServerName&gt; listDecommissionedRegionServers() {</span>
<span class="source-line-no">4099</span><span id="line-4099"> return this.serverManager.getDrainingServersList();</span>
<span class="source-line-no">4100</span><span id="line-4100"> }</span>
<span class="source-line-no">4101</span><span id="line-4101"></span>
<span class="source-line-no">4102</span><span id="line-4102"> /**</span>
<span class="source-line-no">4103</span><span id="line-4103"> * Remove decommission marker (previously called 'draining') from a region server to allow regions</span>
<span class="source-line-no">4104</span><span id="line-4104"> * assignments. Load regions onto the server asynchronously if a list of regions is given</span>
<span class="source-line-no">4105</span><span id="line-4105"> * @param server Region server to remove decommission marker from.</span>
<span class="source-line-no">4106</span><span id="line-4106"> */</span>
<span class="source-line-no">4107</span><span id="line-4107"> public void recommissionRegionServer(final ServerName server,</span>
<span class="source-line-no">4108</span><span id="line-4108"> final List&lt;byte[]&gt; encodedRegionNames) throws IOException {</span>
<span class="source-line-no">4109</span><span id="line-4109"> // Remove the server from decommissioned (draining) server list.</span>
<span class="source-line-no">4110</span><span id="line-4110"> String parentZnode = getZooKeeper().getZNodePaths().drainingZNode;</span>
<span class="source-line-no">4111</span><span id="line-4111"> String node = ZNodePaths.joinZNode(parentZnode, server.getServerName());</span>
<span class="source-line-no">4112</span><span id="line-4112"> try {</span>
<span class="source-line-no">4113</span><span id="line-4113"> ZKUtil.deleteNodeFailSilent(getZooKeeper(), node);</span>
<span class="source-line-no">4114</span><span id="line-4114"> } catch (KeeperException ke) {</span>
<span class="source-line-no">4115</span><span id="line-4115"> throw new HBaseIOException(</span>
<span class="source-line-no">4116</span><span id="line-4116"> this.zooKeeper.prefix("Unable to recommission '" + server.getServerName() + "'."), ke);</span>
<span class="source-line-no">4117</span><span id="line-4117"> }</span>
<span class="source-line-no">4118</span><span id="line-4118"> this.serverManager.removeServerFromDrainList(server);</span>
<span class="source-line-no">4119</span><span id="line-4119"></span>
<span class="source-line-no">4120</span><span id="line-4120"> // Load the regions onto the server if we are given a list of regions.</span>
<span class="source-line-no">4121</span><span id="line-4121"> if (encodedRegionNames == null || encodedRegionNames.isEmpty()) {</span>
<span class="source-line-no">4122</span><span id="line-4122"> return;</span>
<span class="source-line-no">4123</span><span id="line-4123"> }</span>
<span class="source-line-no">4124</span><span id="line-4124"> if (!this.serverManager.isServerOnline(server)) {</span>
<span class="source-line-no">4125</span><span id="line-4125"> return;</span>
<span class="source-line-no">4126</span><span id="line-4126"> }</span>
<span class="source-line-no">4127</span><span id="line-4127"> for (byte[] encodedRegionName : encodedRegionNames) {</span>
<span class="source-line-no">4128</span><span id="line-4128"> RegionState regionState =</span>
<span class="source-line-no">4129</span><span id="line-4129"> assignmentManager.getRegionStates().getRegionState(Bytes.toString(encodedRegionName));</span>
<span class="source-line-no">4130</span><span id="line-4130"> if (regionState == null) {</span>
<span class="source-line-no">4131</span><span id="line-4131"> LOG.warn("Unknown region " + Bytes.toStringBinary(encodedRegionName));</span>
<span class="source-line-no">4132</span><span id="line-4132"> continue;</span>
<span class="source-line-no">4133</span><span id="line-4133"> }</span>
<span class="source-line-no">4134</span><span id="line-4134"> RegionInfo hri = regionState.getRegion();</span>
<span class="source-line-no">4135</span><span id="line-4135"> if (server.equals(regionState.getServerName())) {</span>
<span class="source-line-no">4136</span><span id="line-4136"> LOG.info("Skipping move of region " + hri.getRegionNameAsString()</span>
<span class="source-line-no">4137</span><span id="line-4137"> + " because region already assigned to the same server " + server + ".");</span>
<span class="source-line-no">4138</span><span id="line-4138"> continue;</span>
<span class="source-line-no">4139</span><span id="line-4139"> }</span>
<span class="source-line-no">4140</span><span id="line-4140"> RegionPlan rp = new RegionPlan(hri, regionState.getServerName(), server);</span>
<span class="source-line-no">4141</span><span id="line-4141"> this.assignmentManager.moveAsync(rp);</span>
<span class="source-line-no">4142</span><span id="line-4142"> }</span>
<span class="source-line-no">4143</span><span id="line-4143"> }</span>
<span class="source-line-no">4144</span><span id="line-4144"></span>
<span class="source-line-no">4145</span><span id="line-4145"> @Override</span>
<span class="source-line-no">4146</span><span id="line-4146"> public LockManager getLockManager() {</span>
<span class="source-line-no">4147</span><span id="line-4147"> return lockManager;</span>
<span class="source-line-no">4148</span><span id="line-4148"> }</span>
<span class="source-line-no">4149</span><span id="line-4149"></span>
<span class="source-line-no">4150</span><span id="line-4150"> public QuotaObserverChore getQuotaObserverChore() {</span>
<span class="source-line-no">4151</span><span id="line-4151"> return this.quotaObserverChore;</span>
<span class="source-line-no">4152</span><span id="line-4152"> }</span>
<span class="source-line-no">4153</span><span id="line-4153"></span>
<span class="source-line-no">4154</span><span id="line-4154"> public SpaceQuotaSnapshotNotifier getSpaceQuotaSnapshotNotifier() {</span>
<span class="source-line-no">4155</span><span id="line-4155"> return this.spaceQuotaSnapshotNotifier;</span>
<span class="source-line-no">4156</span><span id="line-4156"> }</span>
<span class="source-line-no">4157</span><span id="line-4157"></span>
<span class="source-line-no">4158</span><span id="line-4158"> @SuppressWarnings("unchecked")</span>
<span class="source-line-no">4159</span><span id="line-4159"> private RemoteProcedure&lt;MasterProcedureEnv, ?&gt; getRemoteProcedure(long procId) {</span>
<span class="source-line-no">4160</span><span id="line-4160"> Procedure&lt;?&gt; procedure = procedureExecutor.getProcedure(procId);</span>
<span class="source-line-no">4161</span><span id="line-4161"> if (procedure == null) {</span>
<span class="source-line-no">4162</span><span id="line-4162"> return null;</span>
<span class="source-line-no">4163</span><span id="line-4163"> }</span>
<span class="source-line-no">4164</span><span id="line-4164"> assert procedure instanceof RemoteProcedure;</span>
<span class="source-line-no">4165</span><span id="line-4165"> return (RemoteProcedure&lt;MasterProcedureEnv, ?&gt;) procedure;</span>
<span class="source-line-no">4166</span><span id="line-4166"> }</span>
<span class="source-line-no">4167</span><span id="line-4167"></span>
<span class="source-line-no">4168</span><span id="line-4168"> public void remoteProcedureCompleted(long procId) {</span>
<span class="source-line-no">4169</span><span id="line-4169"> LOG.debug("Remote procedure done, pid={}", procId);</span>
<span class="source-line-no">4170</span><span id="line-4170"> RemoteProcedure&lt;MasterProcedureEnv, ?&gt; procedure = getRemoteProcedure(procId);</span>
<span class="source-line-no">4171</span><span id="line-4171"> if (procedure != null) {</span>
<span class="source-line-no">4172</span><span id="line-4172"> procedure.remoteOperationCompleted(procedureExecutor.getEnvironment());</span>
<span class="source-line-no">4173</span><span id="line-4173"> }</span>
<span class="source-line-no">4174</span><span id="line-4174"> }</span>
<span class="source-line-no">4175</span><span id="line-4175"></span>
<span class="source-line-no">4176</span><span id="line-4176"> public void remoteProcedureFailed(long procId, RemoteProcedureException error) {</span>
<span class="source-line-no">4177</span><span id="line-4177"> LOG.debug("Remote procedure failed, pid={}", procId, error);</span>
<span class="source-line-no">4178</span><span id="line-4178"> RemoteProcedure&lt;MasterProcedureEnv, ?&gt; procedure = getRemoteProcedure(procId);</span>
<span class="source-line-no">4179</span><span id="line-4179"> if (procedure != null) {</span>
<span class="source-line-no">4180</span><span id="line-4180"> procedure.remoteOperationFailed(procedureExecutor.getEnvironment(), error);</span>
<span class="source-line-no">4181</span><span id="line-4181"> }</span>
<span class="source-line-no">4182</span><span id="line-4182"> }</span>
<span class="source-line-no">4183</span><span id="line-4183"></span>
<span class="source-line-no">4184</span><span id="line-4184"> /**</span>
<span class="source-line-no">4185</span><span id="line-4185"> * Reopen regions provided in the argument</span>
<span class="source-line-no">4186</span><span id="line-4186"> * @param tableName The current table name</span>
<span class="source-line-no">4187</span><span id="line-4187"> * @param regionNames The region names of the regions to reopen</span>
<span class="source-line-no">4188</span><span id="line-4188"> * @param nonceGroup Identifier for the source of the request, a client or process</span>
<span class="source-line-no">4189</span><span id="line-4189"> * @param nonce A unique identifier for this operation from the client or process identified</span>
<span class="source-line-no">4190</span><span id="line-4190"> * by &lt;code&gt;nonceGroup&lt;/code&gt; (the source must ensure each operation gets a</span>
<span class="source-line-no">4191</span><span id="line-4191"> * unique id).</span>
<span class="source-line-no">4192</span><span id="line-4192"> * @return procedure Id</span>
<span class="source-line-no">4193</span><span id="line-4193"> * @throws IOException if reopening region fails while running procedure</span>
<span class="source-line-no">4194</span><span id="line-4194"> */</span>
<span class="source-line-no">4195</span><span id="line-4195"> long reopenRegions(final TableName tableName, final List&lt;byte[]&gt; regionNames,</span>
<span class="source-line-no">4196</span><span id="line-4196"> final long nonceGroup, final long nonce) throws IOException {</span>
<span class="source-line-no">4197</span><span id="line-4197"></span>
<span class="source-line-no">4198</span><span id="line-4198"> return MasterProcedureUtil</span>
<span class="source-line-no">4199</span><span id="line-4199"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">4200</span><span id="line-4200"></span>
<span class="source-line-no">4201</span><span id="line-4201"> @Override</span>
<span class="source-line-no">4202</span><span id="line-4202"> protected void run() throws IOException {</span>
<span class="source-line-no">4203</span><span id="line-4203"> submitProcedure(new ReopenTableRegionsProcedure(tableName, regionNames));</span>
<span class="source-line-no">4204</span><span id="line-4204"> }</span>
<span class="source-line-no">4205</span><span id="line-4205"></span>
<span class="source-line-no">4206</span><span id="line-4206"> @Override</span>
<span class="source-line-no">4207</span><span id="line-4207"> protected String getDescription() {</span>
<span class="source-line-no">4208</span><span id="line-4208"> return "ReopenTableRegionsProcedure";</span>
<span class="source-line-no">4209</span><span id="line-4209"> }</span>
<span class="source-line-no">4210</span><span id="line-4210"></span>
<span class="source-line-no">4211</span><span id="line-4211"> });</span>
<span class="source-line-no">4212</span><span id="line-4212"></span>
<span class="source-line-no">4213</span><span id="line-4213"> }</span>
<span class="source-line-no">4214</span><span id="line-4214"></span>
<span class="source-line-no">4215</span><span id="line-4215"> @Override</span>
<span class="source-line-no">4216</span><span id="line-4216"> public ReplicationPeerManager getReplicationPeerManager() {</span>
<span class="source-line-no">4217</span><span id="line-4217"> return replicationPeerManager;</span>
<span class="source-line-no">4218</span><span id="line-4218"> }</span>
<span class="source-line-no">4219</span><span id="line-4219"></span>
<span class="source-line-no">4220</span><span id="line-4220"> @Override</span>
<span class="source-line-no">4221</span><span id="line-4221"> public ReplicationLogCleanerBarrier getReplicationLogCleanerBarrier() {</span>
<span class="source-line-no">4222</span><span id="line-4222"> return replicationLogCleanerBarrier;</span>
<span class="source-line-no">4223</span><span id="line-4223"> }</span>
<span class="source-line-no">4224</span><span id="line-4224"></span>
<span class="source-line-no">4225</span><span id="line-4225"> @Override</span>
<span class="source-line-no">4226</span><span id="line-4226"> public Semaphore getSyncReplicationPeerLock() {</span>
<span class="source-line-no">4227</span><span id="line-4227"> return syncReplicationPeerLock;</span>
<span class="source-line-no">4228</span><span id="line-4228"> }</span>
<span class="source-line-no">4229</span><span id="line-4229"></span>
<span class="source-line-no">4230</span><span id="line-4230"> public HashMap&lt;String, List&lt;Pair&lt;ServerName, ReplicationLoadSource&gt;&gt;&gt;</span>
<span class="source-line-no">4231</span><span id="line-4231"> getReplicationLoad(ServerName[] serverNames) {</span>
<span class="source-line-no">4232</span><span id="line-4232"> List&lt;ReplicationPeerDescription&gt; peerList = this.getReplicationPeerManager().listPeers(null);</span>
<span class="source-line-no">4233</span><span id="line-4233"> if (peerList == null) {</span>
<span class="source-line-no">4234</span><span id="line-4234"> return null;</span>
<span class="source-line-no">4235</span><span id="line-4235"> }</span>
<span class="source-line-no">4236</span><span id="line-4236"> HashMap&lt;String, List&lt;Pair&lt;ServerName, ReplicationLoadSource&gt;&gt;&gt; replicationLoadSourceMap =</span>
<span class="source-line-no">4237</span><span id="line-4237"> new HashMap&lt;&gt;(peerList.size());</span>
<span class="source-line-no">4238</span><span id="line-4238"> peerList.stream()</span>
<span class="source-line-no">4239</span><span id="line-4239"> .forEach(peer -&gt; replicationLoadSourceMap.put(peer.getPeerId(), new ArrayList&lt;&gt;()));</span>
<span class="source-line-no">4240</span><span id="line-4240"> for (ServerName serverName : serverNames) {</span>
<span class="source-line-no">4241</span><span id="line-4241"> List&lt;ReplicationLoadSource&gt; replicationLoadSources =</span>
<span class="source-line-no">4242</span><span id="line-4242"> getServerManager().getLoad(serverName).getReplicationLoadSourceList();</span>
<span class="source-line-no">4243</span><span id="line-4243"> for (ReplicationLoadSource replicationLoadSource : replicationLoadSources) {</span>
<span class="source-line-no">4244</span><span id="line-4244"> List&lt;Pair&lt;ServerName, ReplicationLoadSource&gt;&gt; replicationLoadSourceList =</span>
<span class="source-line-no">4245</span><span id="line-4245"> replicationLoadSourceMap.get(replicationLoadSource.getPeerID());</span>
<span class="source-line-no">4246</span><span id="line-4246"> if (replicationLoadSourceList == null) {</span>
<span class="source-line-no">4247</span><span id="line-4247"> LOG.debug("{} does not exist, but it exists "</span>
<span class="source-line-no">4248</span><span id="line-4248"> + "in znode(/hbase/replication/rs). when the rs restarts, peerId is deleted, so "</span>
<span class="source-line-no">4249</span><span id="line-4249"> + "we just need to ignore it", replicationLoadSource.getPeerID());</span>
<span class="source-line-no">4250</span><span id="line-4250"> continue;</span>
<span class="source-line-no">4251</span><span id="line-4251"> }</span>
<span class="source-line-no">4252</span><span id="line-4252"> replicationLoadSourceList.add(new Pair&lt;&gt;(serverName, replicationLoadSource));</span>
<span class="source-line-no">4253</span><span id="line-4253"> }</span>
<span class="source-line-no">4254</span><span id="line-4254"> }</span>
<span class="source-line-no">4255</span><span id="line-4255"> for (List&lt;Pair&lt;ServerName, ReplicationLoadSource&gt;&gt; loads : replicationLoadSourceMap.values()) {</span>
<span class="source-line-no">4256</span><span id="line-4256"> if (loads.size() &gt; 0) {</span>
<span class="source-line-no">4257</span><span id="line-4257"> loads.sort(Comparator.comparingLong(load -&gt; (-1) * load.getSecond().getReplicationLag()));</span>
<span class="source-line-no">4258</span><span id="line-4258"> }</span>
<span class="source-line-no">4259</span><span id="line-4259"> }</span>
<span class="source-line-no">4260</span><span id="line-4260"> return replicationLoadSourceMap;</span>
<span class="source-line-no">4261</span><span id="line-4261"> }</span>
<span class="source-line-no">4262</span><span id="line-4262"></span>
<span class="source-line-no">4263</span><span id="line-4263"> /**</span>
<span class="source-line-no">4264</span><span id="line-4264"> * This method modifies the master's configuration in order to inject replication-related features</span>
<span class="source-line-no">4265</span><span id="line-4265"> */</span>
<span class="source-line-no">4266</span><span id="line-4266"> @InterfaceAudience.Private</span>
<span class="source-line-no">4267</span><span id="line-4267"> public static void decorateMasterConfiguration(Configuration conf) {</span>
<span class="source-line-no">4268</span><span id="line-4268"> String plugins = conf.get(HBASE_MASTER_LOGCLEANER_PLUGINS);</span>
<span class="source-line-no">4269</span><span id="line-4269"> String cleanerClass = ReplicationLogCleaner.class.getCanonicalName();</span>
<span class="source-line-no">4270</span><span id="line-4270"> if (plugins == null || !plugins.contains(cleanerClass)) {</span>
<span class="source-line-no">4271</span><span id="line-4271"> conf.set(HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);</span>
<span class="source-line-no">4272</span><span id="line-4272"> }</span>
<span class="source-line-no">4273</span><span id="line-4273"> if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) {</span>
<span class="source-line-no">4274</span><span id="line-4274"> plugins = conf.get(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS);</span>
<span class="source-line-no">4275</span><span id="line-4275"> cleanerClass = ReplicationHFileCleaner.class.getCanonicalName();</span>
<span class="source-line-no">4276</span><span id="line-4276"> if (!plugins.contains(cleanerClass)) {</span>
<span class="source-line-no">4277</span><span id="line-4277"> conf.set(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, plugins + "," + cleanerClass);</span>
<span class="source-line-no">4278</span><span id="line-4278"> }</span>
<span class="source-line-no">4279</span><span id="line-4279"> }</span>
<span class="source-line-no">4280</span><span id="line-4280"> }</span>
<span class="source-line-no">4281</span><span id="line-4281"></span>
<span class="source-line-no">4282</span><span id="line-4282"> public SnapshotQuotaObserverChore getSnapshotQuotaObserverChore() {</span>
<span class="source-line-no">4283</span><span id="line-4283"> return this.snapshotQuotaChore;</span>
<span class="source-line-no">4284</span><span id="line-4284"> }</span>
<span class="source-line-no">4285</span><span id="line-4285"></span>
<span class="source-line-no">4286</span><span id="line-4286"> public ActiveMasterManager getActiveMasterManager() {</span>
<span class="source-line-no">4287</span><span id="line-4287"> return activeMasterManager;</span>
<span class="source-line-no">4288</span><span id="line-4288"> }</span>
<span class="source-line-no">4289</span><span id="line-4289"></span>
<span class="source-line-no">4290</span><span id="line-4290"> @Override</span>
<span class="source-line-no">4291</span><span id="line-4291"> public SyncReplicationReplayWALManager getSyncReplicationReplayWALManager() {</span>
<span class="source-line-no">4292</span><span id="line-4292"> return this.syncReplicationReplayWALManager;</span>
<span class="source-line-no">4293</span><span id="line-4293"> }</span>
<span class="source-line-no">4294</span><span id="line-4294"></span>
<span class="source-line-no">4295</span><span id="line-4295"> @Override</span>
<span class="source-line-no">4296</span><span id="line-4296"> public HbckChore getHbckChore() {</span>
<span class="source-line-no">4297</span><span id="line-4297"> return this.hbckChore;</span>
<span class="source-line-no">4298</span><span id="line-4298"> }</span>
<span class="source-line-no">4299</span><span id="line-4299"></span>
<span class="source-line-no">4300</span><span id="line-4300"> @Override</span>
<span class="source-line-no">4301</span><span id="line-4301"> public void runReplicationBarrierCleaner() {</span>
<span class="source-line-no">4302</span><span id="line-4302"> ReplicationBarrierCleaner rbc = this.replicationBarrierCleaner;</span>
<span class="source-line-no">4303</span><span id="line-4303"> if (rbc != null) {</span>
<span class="source-line-no">4304</span><span id="line-4304"> rbc.chore();</span>
<span class="source-line-no">4305</span><span id="line-4305"> }</span>
<span class="source-line-no">4306</span><span id="line-4306"> }</span>
<span class="source-line-no">4307</span><span id="line-4307"></span>
<span class="source-line-no">4308</span><span id="line-4308"> @Override</span>
<span class="source-line-no">4309</span><span id="line-4309"> public RSGroupInfoManager getRSGroupInfoManager() {</span>
<span class="source-line-no">4310</span><span id="line-4310"> return rsGroupInfoManager;</span>
<span class="source-line-no">4311</span><span id="line-4311"> }</span>
<span class="source-line-no">4312</span><span id="line-4312"></span>
<span class="source-line-no">4313</span><span id="line-4313"> /**</span>
<span class="source-line-no">4314</span><span id="line-4314"> * Get the compaction state of the table</span>
<span class="source-line-no">4315</span><span id="line-4315"> * @param tableName The table name</span>
<span class="source-line-no">4316</span><span id="line-4316"> * @return CompactionState Compaction state of the table</span>
<span class="source-line-no">4317</span><span id="line-4317"> */</span>
<span class="source-line-no">4318</span><span id="line-4318"> public CompactionState getCompactionState(final TableName tableName) {</span>
<span class="source-line-no">4319</span><span id="line-4319"> CompactionState compactionState = CompactionState.NONE;</span>
<span class="source-line-no">4320</span><span id="line-4320"> try {</span>
<span class="source-line-no">4321</span><span id="line-4321"> List&lt;RegionInfo&gt; regions = assignmentManager.getRegionStates().getRegionsOfTable(tableName);</span>
<span class="source-line-no">4322</span><span id="line-4322"> for (RegionInfo regionInfo : regions) {</span>
<span class="source-line-no">4323</span><span id="line-4323"> ServerName serverName =</span>
<span class="source-line-no">4324</span><span id="line-4324"> assignmentManager.getRegionStates().getRegionServerOfRegion(regionInfo);</span>
<span class="source-line-no">4325</span><span id="line-4325"> if (serverName == null) {</span>
<span class="source-line-no">4326</span><span id="line-4326"> continue;</span>
<span class="source-line-no">4327</span><span id="line-4327"> }</span>
<span class="source-line-no">4328</span><span id="line-4328"> ServerMetrics sl = serverManager.getLoad(serverName);</span>
<span class="source-line-no">4329</span><span id="line-4329"> if (sl == null) {</span>
<span class="source-line-no">4330</span><span id="line-4330"> continue;</span>
<span class="source-line-no">4331</span><span id="line-4331"> }</span>
<span class="source-line-no">4332</span><span id="line-4332"> RegionMetrics regionMetrics = sl.getRegionMetrics().get(regionInfo.getRegionName());</span>
<span class="source-line-no">4333</span><span id="line-4333"> if (regionMetrics == null) {</span>
<span class="source-line-no">4334</span><span id="line-4334"> LOG.warn("Can not get compaction details for the region: {} , it may be not online.",</span>
<span class="source-line-no">4335</span><span id="line-4335"> regionInfo.getRegionNameAsString());</span>
<span class="source-line-no">4336</span><span id="line-4336"> continue;</span>
<span class="source-line-no">4337</span><span id="line-4337"> }</span>
<span class="source-line-no">4338</span><span id="line-4338"> if (regionMetrics.getCompactionState() == CompactionState.MAJOR) {</span>
<span class="source-line-no">4339</span><span id="line-4339"> if (compactionState == CompactionState.MINOR) {</span>
<span class="source-line-no">4340</span><span id="line-4340"> compactionState = CompactionState.MAJOR_AND_MINOR;</span>
<span class="source-line-no">4341</span><span id="line-4341"> } else {</span>
<span class="source-line-no">4342</span><span id="line-4342"> compactionState = CompactionState.MAJOR;</span>
<span class="source-line-no">4343</span><span id="line-4343"> }</span>
<span class="source-line-no">4344</span><span id="line-4344"> } else if (regionMetrics.getCompactionState() == CompactionState.MINOR) {</span>
<span class="source-line-no">4345</span><span id="line-4345"> if (compactionState == CompactionState.MAJOR) {</span>
<span class="source-line-no">4346</span><span id="line-4346"> compactionState = CompactionState.MAJOR_AND_MINOR;</span>
<span class="source-line-no">4347</span><span id="line-4347"> } else {</span>
<span class="source-line-no">4348</span><span id="line-4348"> compactionState = CompactionState.MINOR;</span>
<span class="source-line-no">4349</span><span id="line-4349"> }</span>
<span class="source-line-no">4350</span><span id="line-4350"> }</span>
<span class="source-line-no">4351</span><span id="line-4351"> }</span>
<span class="source-line-no">4352</span><span id="line-4352"> } catch (Exception e) {</span>
<span class="source-line-no">4353</span><span id="line-4353"> compactionState = null;</span>
<span class="source-line-no">4354</span><span id="line-4354"> LOG.error("Exception when get compaction state for " + tableName.getNameAsString(), e);</span>
<span class="source-line-no">4355</span><span id="line-4355"> }</span>
<span class="source-line-no">4356</span><span id="line-4356"> return compactionState;</span>
<span class="source-line-no">4357</span><span id="line-4357"> }</span>
<span class="source-line-no">4358</span><span id="line-4358"></span>
<span class="source-line-no">4359</span><span id="line-4359"> @Override</span>
<span class="source-line-no">4360</span><span id="line-4360"> public MetaLocationSyncer getMetaLocationSyncer() {</span>
<span class="source-line-no">4361</span><span id="line-4361"> return metaLocationSyncer;</span>
<span class="source-line-no">4362</span><span id="line-4362"> }</span>
<span class="source-line-no">4363</span><span id="line-4363"></span>
<span class="source-line-no">4364</span><span id="line-4364"> @RestrictedApi(explanation = "Should only be called in tests", link = "",</span>
<span class="source-line-no">4365</span><span id="line-4365"> allowedOnPath = ".*/src/test/.*")</span>
<span class="source-line-no">4366</span><span id="line-4366"> public MasterRegion getMasterRegion() {</span>
<span class="source-line-no">4367</span><span id="line-4367"> return masterRegion;</span>
<span class="source-line-no">4368</span><span id="line-4368"> }</span>
<span class="source-line-no">4369</span><span id="line-4369"></span>
<span class="source-line-no">4370</span><span id="line-4370"> @Override</span>
<span class="source-line-no">4371</span><span id="line-4371"> public void onConfigurationChange(Configuration newConf) {</span>
<span class="source-line-no">4372</span><span id="line-4372"> try {</span>
<span class="source-line-no">4373</span><span id="line-4373"> Superusers.initialize(newConf);</span>
<span class="source-line-no">4374</span><span id="line-4374"> } catch (IOException e) {</span>
<span class="source-line-no">4375</span><span id="line-4375"> LOG.warn("Failed to initialize SuperUsers on reloading of the configuration");</span>
<span class="source-line-no">4376</span><span id="line-4376"> }</span>
<span class="source-line-no">4377</span><span id="line-4377"> // append the quotas observer back to the master coprocessor key</span>
<span class="source-line-no">4378</span><span id="line-4378"> setQuotasObserver(newConf);</span>
<span class="source-line-no">4379</span><span id="line-4379"> // update region server coprocessor if the configuration has changed.</span>
<span class="source-line-no">4380</span><span id="line-4380"> if (</span>
<span class="source-line-no">4381</span><span id="line-4381"> CoprocessorConfigurationUtil.checkConfigurationChange(getConfiguration(), newConf,</span>
<span class="source-line-no">4382</span><span id="line-4382"> CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY) &amp;&amp; !maintenanceMode</span>
<span class="source-line-no">4383</span><span id="line-4383"> ) {</span>
<span class="source-line-no">4384</span><span id="line-4384"> LOG.info("Update the master coprocessor(s) because the configuration has changed");</span>
<span class="source-line-no">4385</span><span id="line-4385"> initializeCoprocessorHost(newConf);</span>
<span class="source-line-no">4386</span><span id="line-4386"> }</span>
<span class="source-line-no">4387</span><span id="line-4387"> }</span>
<span class="source-line-no">4388</span><span id="line-4388"></span>
<span class="source-line-no">4389</span><span id="line-4389"> @Override</span>
<span class="source-line-no">4390</span><span id="line-4390"> protected NamedQueueRecorder createNamedQueueRecord() {</span>
<span class="source-line-no">4391</span><span id="line-4391"> final boolean isBalancerDecisionRecording =</span>
<span class="source-line-no">4392</span><span id="line-4392"> conf.getBoolean(BaseLoadBalancer.BALANCER_DECISION_BUFFER_ENABLED,</span>
<span class="source-line-no">4393</span><span id="line-4393"> BaseLoadBalancer.DEFAULT_BALANCER_DECISION_BUFFER_ENABLED);</span>
<span class="source-line-no">4394</span><span id="line-4394"> final boolean isBalancerRejectionRecording =</span>
<span class="source-line-no">4395</span><span id="line-4395"> conf.getBoolean(BaseLoadBalancer.BALANCER_REJECTION_BUFFER_ENABLED,</span>
<span class="source-line-no">4396</span><span id="line-4396"> BaseLoadBalancer.DEFAULT_BALANCER_REJECTION_BUFFER_ENABLED);</span>
<span class="source-line-no">4397</span><span id="line-4397"> if (isBalancerDecisionRecording || isBalancerRejectionRecording) {</span>
<span class="source-line-no">4398</span><span id="line-4398"> return NamedQueueRecorder.getInstance(conf);</span>
<span class="source-line-no">4399</span><span id="line-4399"> } else {</span>
<span class="source-line-no">4400</span><span id="line-4400"> return null;</span>
<span class="source-line-no">4401</span><span id="line-4401"> }</span>
<span class="source-line-no">4402</span><span id="line-4402"> }</span>
<span class="source-line-no">4403</span><span id="line-4403"></span>
<span class="source-line-no">4404</span><span id="line-4404"> @Override</span>
<span class="source-line-no">4405</span><span id="line-4405"> protected boolean clusterMode() {</span>
<span class="source-line-no">4406</span><span id="line-4406"> return true;</span>
<span class="source-line-no">4407</span><span id="line-4407"> }</span>
<span class="source-line-no">4408</span><span id="line-4408"></span>
<span class="source-line-no">4409</span><span id="line-4409"> public String getClusterId() {</span>
<span class="source-line-no">4410</span><span id="line-4410"> if (activeMaster) {</span>
<span class="source-line-no">4411</span><span id="line-4411"> return clusterId;</span>
<span class="source-line-no">4412</span><span id="line-4412"> }</span>
<span class="source-line-no">4413</span><span id="line-4413"> return cachedClusterId.getFromCacheOrFetch();</span>
<span class="source-line-no">4414</span><span id="line-4414"> }</span>
<span class="source-line-no">4415</span><span id="line-4415"></span>
<span class="source-line-no">4416</span><span id="line-4416"> public Optional&lt;ServerName&gt; getActiveMaster() {</span>
<span class="source-line-no">4417</span><span id="line-4417"> return activeMasterManager.getActiveMasterServerName();</span>
<span class="source-line-no">4418</span><span id="line-4418"> }</span>
<span class="source-line-no">4419</span><span id="line-4419"></span>
<span class="source-line-no">4420</span><span id="line-4420"> public List&lt;ServerName&gt; getBackupMasters() {</span>
<span class="source-line-no">4421</span><span id="line-4421"> return activeMasterManager.getBackupMasters();</span>
<span class="source-line-no">4422</span><span id="line-4422"> }</span>
<span class="source-line-no">4423</span><span id="line-4423"></span>
<span class="source-line-no">4424</span><span id="line-4424"> @Override</span>
<span class="source-line-no">4425</span><span id="line-4425"> public Iterator&lt;ServerName&gt; getBootstrapNodes() {</span>
<span class="source-line-no">4426</span><span id="line-4426"> return regionServerTracker.getRegionServers().iterator();</span>
<span class="source-line-no">4427</span><span id="line-4427"> }</span>
<span class="source-line-no">4428</span><span id="line-4428"></span>
<span class="source-line-no">4429</span><span id="line-4429"> @Override</span>
<span class="source-line-no">4430</span><span id="line-4430"> public List&lt;HRegionLocation&gt; getMetaLocations() {</span>
<span class="source-line-no">4431</span><span id="line-4431"> return metaRegionLocationCache.getMetaRegionLocations();</span>
<span class="source-line-no">4432</span><span id="line-4432"> }</span>
<span class="source-line-no">4433</span><span id="line-4433"></span>
<span class="source-line-no">4434</span><span id="line-4434"> @Override</span>
<span class="source-line-no">4435</span><span id="line-4435"> public void flushMasterStore() throws IOException {</span>
<span class="source-line-no">4436</span><span id="line-4436"> LOG.info("Force flush master local region.");</span>
<span class="source-line-no">4437</span><span id="line-4437"> if (this.cpHost != null) {</span>
<span class="source-line-no">4438</span><span id="line-4438"> try {</span>
<span class="source-line-no">4439</span><span id="line-4439"> cpHost.preMasterStoreFlush();</span>
<span class="source-line-no">4440</span><span id="line-4440"> } catch (IOException ioe) {</span>
<span class="source-line-no">4441</span><span id="line-4441"> LOG.error("Error invoking master coprocessor preMasterStoreFlush()", ioe);</span>
<span class="source-line-no">4442</span><span id="line-4442"> }</span>
<span class="source-line-no">4443</span><span id="line-4443"> }</span>
<span class="source-line-no">4444</span><span id="line-4444"> masterRegion.flush(true);</span>
<span class="source-line-no">4445</span><span id="line-4445"> if (this.cpHost != null) {</span>
<span class="source-line-no">4446</span><span id="line-4446"> try {</span>
<span class="source-line-no">4447</span><span id="line-4447"> cpHost.postMasterStoreFlush();</span>
<span class="source-line-no">4448</span><span id="line-4448"> } catch (IOException ioe) {</span>
<span class="source-line-no">4449</span><span id="line-4449"> LOG.error("Error invoking master coprocessor postMasterStoreFlush()", ioe);</span>
<span class="source-line-no">4450</span><span id="line-4450"> }</span>
<span class="source-line-no">4451</span><span id="line-4451"> }</span>
<span class="source-line-no">4452</span><span id="line-4452"> }</span>
<span class="source-line-no">4453</span><span id="line-4453"></span>
<span class="source-line-no">4454</span><span id="line-4454"> public Collection&lt;ServerName&gt; getLiveRegionServers() {</span>
<span class="source-line-no">4455</span><span id="line-4455"> return regionServerTracker.getRegionServers();</span>
<span class="source-line-no">4456</span><span id="line-4456"> }</span>
<span class="source-line-no">4457</span><span id="line-4457"></span>
<span class="source-line-no">4458</span><span id="line-4458"> @RestrictedApi(explanation = "Should only be called in tests", link = "",</span>
<span class="source-line-no">4459</span><span id="line-4459"> allowedOnPath = ".*/src/test/.*")</span>
<span class="source-line-no">4460</span><span id="line-4460"> void setLoadBalancer(RSGroupBasedLoadBalancer loadBalancer) {</span>
<span class="source-line-no">4461</span><span id="line-4461"> this.balancer = loadBalancer;</span>
<span class="source-line-no">4462</span><span id="line-4462"> }</span>
<span class="source-line-no">4463</span><span id="line-4463"></span>
<span class="source-line-no">4464</span><span id="line-4464"> @RestrictedApi(explanation = "Should only be called in tests", link = "",</span>
<span class="source-line-no">4465</span><span id="line-4465"> allowedOnPath = ".*/src/test/.*")</span>
<span class="source-line-no">4466</span><span id="line-4466"> void setAssignmentManager(AssignmentManager assignmentManager) {</span>
<span class="source-line-no">4467</span><span id="line-4467"> this.assignmentManager = assignmentManager;</span>
<span class="source-line-no">4468</span><span id="line-4468"> }</span>
<span class="source-line-no">4469</span><span id="line-4469"></span>
<span class="source-line-no">4470</span><span id="line-4470"> @RestrictedApi(explanation = "Should only be called in tests", link = "",</span>
<span class="source-line-no">4471</span><span id="line-4471"> allowedOnPath = ".*/src/test/.*")</span>
<span class="source-line-no">4472</span><span id="line-4472"> static void setDisableBalancerChoreForTest(boolean disable) {</span>
<span class="source-line-no">4473</span><span id="line-4473"> disableBalancerChoreForTest = disable;</span>
<span class="source-line-no">4474</span><span id="line-4474"> }</span>
<span class="source-line-no">4475</span><span id="line-4475"></span>
<span class="source-line-no">4476</span><span id="line-4476"> private void setQuotasObserver(Configuration conf) {</span>
<span class="source-line-no">4477</span><span id="line-4477"> // Add the Observer to delete quotas on table deletion before starting all CPs by</span>
<span class="source-line-no">4478</span><span id="line-4478"> // default with quota support, avoiding if user specifically asks to not load this Observer.</span>
<span class="source-line-no">4479</span><span id="line-4479"> if (QuotaUtil.isQuotaEnabled(conf)) {</span>
<span class="source-line-no">4480</span><span id="line-4480"> updateConfigurationForQuotasObserver(conf);</span>
<span class="source-line-no">4481</span><span id="line-4481"> }</span>
<span class="source-line-no">4482</span><span id="line-4482"> }</span>
<span class="source-line-no">4483</span><span id="line-4483"></span>
<span class="source-line-no">4484</span><span id="line-4484"> private void initializeCoprocessorHost(Configuration conf) {</span>
<span class="source-line-no">4485</span><span id="line-4485"> // initialize master side coprocessors before we start handling requests</span>
<span class="source-line-no">4486</span><span id="line-4486"> this.cpHost = new MasterCoprocessorHost(this, conf);</span>
<span class="source-line-no">4487</span><span id="line-4487"> }</span>
<span class="source-line-no">4488</span><span id="line-4488"></span>
<span class="source-line-no">4489</span><span id="line-4489"> @Override</span>
<span class="source-line-no">4490</span><span id="line-4490"> public long flushTable(TableName tableName, List&lt;byte[]&gt; columnFamilies, long nonceGroup,</span>
<span class="source-line-no">4491</span><span id="line-4491"> long nonce) throws IOException {</span>
<span class="source-line-no">4492</span><span id="line-4492"> checkInitialized();</span>
<span class="source-line-no">4493</span><span id="line-4493"></span>
<span class="source-line-no">4494</span><span id="line-4494"> if (</span>
<span class="source-line-no">4495</span><span id="line-4495"> !getConfiguration().getBoolean(MasterFlushTableProcedureManager.FLUSH_PROCEDURE_ENABLED,</span>
<span class="source-line-no">4496</span><span id="line-4496"> MasterFlushTableProcedureManager.FLUSH_PROCEDURE_ENABLED_DEFAULT)</span>
<span class="source-line-no">4497</span><span id="line-4497"> ) {</span>
<span class="source-line-no">4498</span><span id="line-4498"> throw new DoNotRetryIOException("FlushTableProcedureV2 is DISABLED");</span>
<span class="source-line-no">4499</span><span id="line-4499"> }</span>
<span class="source-line-no">4500</span><span id="line-4500"></span>
<span class="source-line-no">4501</span><span id="line-4501"> return MasterProcedureUtil</span>
<span class="source-line-no">4502</span><span id="line-4502"> .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {</span>
<span class="source-line-no">4503</span><span id="line-4503"> @Override</span>
<span class="source-line-no">4504</span><span id="line-4504"> protected void run() throws IOException {</span>
<span class="source-line-no">4505</span><span id="line-4505"> getMaster().getMasterCoprocessorHost().preTableFlush(tableName);</span>
<span class="source-line-no">4506</span><span id="line-4506"> LOG.info(getClientIdAuditPrefix() + " flush " + tableName);</span>
<span class="source-line-no">4507</span><span id="line-4507"> submitProcedure(</span>
<span class="source-line-no">4508</span><span id="line-4508"> new FlushTableProcedure(procedureExecutor.getEnvironment(), tableName, columnFamilies));</span>
<span class="source-line-no">4509</span><span id="line-4509"> getMaster().getMasterCoprocessorHost().postTableFlush(tableName);</span>
<span class="source-line-no">4510</span><span id="line-4510"> }</span>
<span class="source-line-no">4511</span><span id="line-4511"></span>
<span class="source-line-no">4512</span><span id="line-4512"> @Override</span>
<span class="source-line-no">4513</span><span id="line-4513"> protected String getDescription() {</span>
<span class="source-line-no">4514</span><span id="line-4514"> return "FlushTableProcedure";</span>
<span class="source-line-no">4515</span><span id="line-4515"> }</span>
<span class="source-line-no">4516</span><span id="line-4516"> });</span>
<span class="source-line-no">4517</span><span id="line-4517"> }</span>
<span class="source-line-no">4518</span><span id="line-4518">}</span>
</pre>
</div>
</main>
</body>
</html>