blob: db2609fd45a7a9749a3faf09e37bd181e4d2d087 [file] [log] [blame]
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
<html lang="en">
<head>
<title>Source code</title>
<link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style">
</head>
<body>
<div class="sourceContainer">
<pre><span class="sourceLineNo">001</span>/*<a name="line.1"></a>
<span class="sourceLineNo">002</span> * Licensed to the Apache Software Foundation (ASF) under one<a name="line.2"></a>
<span class="sourceLineNo">003</span> * or more contributor license agreements. See the NOTICE file<a name="line.3"></a>
<span class="sourceLineNo">004</span> * distributed with this work for additional information<a name="line.4"></a>
<span class="sourceLineNo">005</span> * regarding copyright ownership. The ASF licenses this file<a name="line.5"></a>
<span class="sourceLineNo">006</span> * to you under the Apache License, Version 2.0 (the<a name="line.6"></a>
<span class="sourceLineNo">007</span> * "License"); you may not use this file except in compliance<a name="line.7"></a>
<span class="sourceLineNo">008</span> * with the License. You may obtain a copy of the License at<a name="line.8"></a>
<span class="sourceLineNo">009</span> *<a name="line.9"></a>
<span class="sourceLineNo">010</span> * http://www.apache.org/licenses/LICENSE-2.0<a name="line.10"></a>
<span class="sourceLineNo">011</span> *<a name="line.11"></a>
<span class="sourceLineNo">012</span> * Unless required by applicable law or agreed to in writing, software<a name="line.12"></a>
<span class="sourceLineNo">013</span> * distributed under the License is distributed on an "AS IS" BASIS,<a name="line.13"></a>
<span class="sourceLineNo">014</span> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.<a name="line.14"></a>
<span class="sourceLineNo">015</span> * See the License for the specific language governing permissions and<a name="line.15"></a>
<span class="sourceLineNo">016</span> * limitations under the License.<a name="line.16"></a>
<span class="sourceLineNo">017</span> */<a name="line.17"></a>
<span class="sourceLineNo">018</span>package org.apache.hadoop.hbase.master;<a name="line.18"></a>
<span class="sourceLineNo">019</span>import static org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective.CHECK;<a name="line.19"></a>
<span class="sourceLineNo">020</span>import static org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective.FORCE;<a name="line.20"></a>
<span class="sourceLineNo">021</span>import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.DELETED;<a name="line.21"></a>
<span class="sourceLineNo">022</span>import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.FAILURE;<a name="line.22"></a>
<span class="sourceLineNo">023</span>import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.IN_PROGRESS;<a name="line.23"></a>
<span class="sourceLineNo">024</span>import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.SUCCESS;<a name="line.24"></a>
<span class="sourceLineNo">025</span>import java.io.IOException;<a name="line.25"></a>
<span class="sourceLineNo">026</span>import java.util.ArrayList;<a name="line.26"></a>
<span class="sourceLineNo">027</span>import java.util.Arrays;<a name="line.27"></a>
<span class="sourceLineNo">028</span>import java.util.Collections;<a name="line.28"></a>
<span class="sourceLineNo">029</span>import java.util.HashSet;<a name="line.29"></a>
<span class="sourceLineNo">030</span>import java.util.List;<a name="line.30"></a>
<span class="sourceLineNo">031</span>import java.util.Map;<a name="line.31"></a>
<span class="sourceLineNo">032</span>import java.util.Set;<a name="line.32"></a>
<span class="sourceLineNo">033</span>import java.util.concurrent.ConcurrentHashMap;<a name="line.33"></a>
<span class="sourceLineNo">034</span>import java.util.concurrent.ConcurrentMap;<a name="line.34"></a>
<span class="sourceLineNo">035</span>import java.util.concurrent.atomic.AtomicInteger;<a name="line.35"></a>
<span class="sourceLineNo">036</span>import org.apache.hadoop.conf.Configuration;<a name="line.36"></a>
<span class="sourceLineNo">037</span>import org.apache.hadoop.fs.FileStatus;<a name="line.37"></a>
<span class="sourceLineNo">038</span>import org.apache.hadoop.fs.FileSystem;<a name="line.38"></a>
<span class="sourceLineNo">039</span>import org.apache.hadoop.fs.Path;<a name="line.39"></a>
<span class="sourceLineNo">040</span>import org.apache.hadoop.fs.PathFilter;<a name="line.40"></a>
<span class="sourceLineNo">041</span>import org.apache.hadoop.hbase.ChoreService;<a name="line.41"></a>
<span class="sourceLineNo">042</span>import org.apache.hadoop.hbase.ScheduledChore;<a name="line.42"></a>
<span class="sourceLineNo">043</span>import org.apache.hadoop.hbase.ServerName;<a name="line.43"></a>
<span class="sourceLineNo">044</span>import org.apache.hadoop.hbase.SplitLogCounters;<a name="line.44"></a>
<span class="sourceLineNo">045</span>import org.apache.hadoop.hbase.Stoppable;<a name="line.45"></a>
<span class="sourceLineNo">046</span>import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination;<a name="line.46"></a>
<span class="sourceLineNo">047</span>import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLogManagerDetails;<a name="line.47"></a>
<span class="sourceLineNo">048</span>import org.apache.hadoop.hbase.log.HBaseMarkers;<a name="line.48"></a>
<span class="sourceLineNo">049</span>import org.apache.hadoop.hbase.monitoring.MonitoredTask;<a name="line.49"></a>
<span class="sourceLineNo">050</span>import org.apache.hadoop.hbase.monitoring.TaskMonitor;<a name="line.50"></a>
<span class="sourceLineNo">051</span>import org.apache.hadoop.hbase.procedure2.util.StringUtils;<a name="line.51"></a>
<span class="sourceLineNo">052</span>import org.apache.hadoop.hbase.util.CommonFSUtils;<a name="line.52"></a>
<span class="sourceLineNo">053</span>import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;<a name="line.53"></a>
<span class="sourceLineNo">054</span>import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;<a name="line.54"></a>
<span class="sourceLineNo">055</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.55"></a>
<span class="sourceLineNo">056</span>import org.slf4j.Logger;<a name="line.56"></a>
<span class="sourceLineNo">057</span>import org.slf4j.LoggerFactory;<a name="line.57"></a>
<span class="sourceLineNo">058</span><a name="line.58"></a>
<span class="sourceLineNo">059</span>/**<a name="line.59"></a>
<span class="sourceLineNo">060</span> * Distributes the task of log splitting to the available region servers.<a name="line.60"></a>
<span class="sourceLineNo">061</span> * Coordination happens via coordination engine. For every log file that has to be split a<a name="line.61"></a>
<span class="sourceLineNo">062</span> * task is created. SplitLogWorkers race to grab a task.<a name="line.62"></a>
<span class="sourceLineNo">063</span> *<a name="line.63"></a>
<span class="sourceLineNo">064</span> * &lt;p&gt;SplitLogManager monitors the tasks that it creates using the<a name="line.64"></a>
<span class="sourceLineNo">065</span> * timeoutMonitor thread. If a task's progress is slow then<a name="line.65"></a>
<span class="sourceLineNo">066</span> * {@link SplitLogManagerCoordination#checkTasks} will take away the<a name="line.66"></a>
<span class="sourceLineNo">067</span> * task from the owner {@link org.apache.hadoop.hbase.regionserver.SplitLogWorker}<a name="line.67"></a>
<span class="sourceLineNo">068</span> * and the task will be up for grabs again. When the task is done then it is<a name="line.68"></a>
<span class="sourceLineNo">069</span> * deleted by SplitLogManager.<a name="line.69"></a>
<span class="sourceLineNo">070</span> *<a name="line.70"></a>
<span class="sourceLineNo">071</span> * &lt;p&gt;Clients call {@link #splitLogDistributed(Path)} to split a region server's<a name="line.71"></a>
<span class="sourceLineNo">072</span> * log files. The caller thread waits in this method until all the log files<a name="line.72"></a>
<span class="sourceLineNo">073</span> * have been split.<a name="line.73"></a>
<span class="sourceLineNo">074</span> *<a name="line.74"></a>
<span class="sourceLineNo">075</span> * &lt;p&gt;All the coordination calls made by this class are asynchronous. This is mainly<a name="line.75"></a>
<span class="sourceLineNo">076</span> * to help reduce response time seen by the callers.<a name="line.76"></a>
<span class="sourceLineNo">077</span> *<a name="line.77"></a>
<span class="sourceLineNo">078</span> * &lt;p&gt;There is race in this design between the SplitLogManager and the<a name="line.78"></a>
<span class="sourceLineNo">079</span> * SplitLogWorker. SplitLogManager might re-queue a task that has in reality<a name="line.79"></a>
<span class="sourceLineNo">080</span> * already been completed by a SplitLogWorker. We rely on the idempotency of<a name="line.80"></a>
<span class="sourceLineNo">081</span> * the log splitting task for correctness.<a name="line.81"></a>
<span class="sourceLineNo">082</span> *<a name="line.82"></a>
<span class="sourceLineNo">083</span> * &lt;p&gt;It is also assumed that every log splitting task is unique and once<a name="line.83"></a>
<span class="sourceLineNo">084</span> * completed (either with success or with error) it will be not be submitted<a name="line.84"></a>
<span class="sourceLineNo">085</span> * again. If a task is resubmitted then there is a risk that old "delete task"<a name="line.85"></a>
<span class="sourceLineNo">086</span> * can delete the re-submission.<a name="line.86"></a>
<span class="sourceLineNo">087</span> * @see SplitWALManager for an alternate implementation based on Procedures.<a name="line.87"></a>
<span class="sourceLineNo">088</span> * @deprecated since 2.4.0 and in 3.0.0, to be removed in 4.0.0, replaced by procedure-based<a name="line.88"></a>
<span class="sourceLineNo">089</span> * distributed WAL splitter, see SplitWALManager.<a name="line.89"></a>
<span class="sourceLineNo">090</span> */<a name="line.90"></a>
<span class="sourceLineNo">091</span>@Deprecated<a name="line.91"></a>
<span class="sourceLineNo">092</span>@InterfaceAudience.Private<a name="line.92"></a>
<span class="sourceLineNo">093</span>public class SplitLogManager {<a name="line.93"></a>
<span class="sourceLineNo">094</span> private static final Logger LOG = LoggerFactory.getLogger(SplitLogManager.class);<a name="line.94"></a>
<span class="sourceLineNo">095</span><a name="line.95"></a>
<span class="sourceLineNo">096</span> private final MasterServices server;<a name="line.96"></a>
<span class="sourceLineNo">097</span><a name="line.97"></a>
<span class="sourceLineNo">098</span> private final Configuration conf;<a name="line.98"></a>
<span class="sourceLineNo">099</span> private final ChoreService choreService;<a name="line.99"></a>
<span class="sourceLineNo">100</span><a name="line.100"></a>
<span class="sourceLineNo">101</span> public static final int DEFAULT_UNASSIGNED_TIMEOUT = (3 * 60 * 1000); // 3 min<a name="line.101"></a>
<span class="sourceLineNo">102</span><a name="line.102"></a>
<span class="sourceLineNo">103</span> private long unassignedTimeout;<a name="line.103"></a>
<span class="sourceLineNo">104</span> private long lastTaskCreateTime = Long.MAX_VALUE;<a name="line.104"></a>
<span class="sourceLineNo">105</span><a name="line.105"></a>
<span class="sourceLineNo">106</span> final ConcurrentMap&lt;String, Task&gt; tasks = new ConcurrentHashMap&lt;&gt;();<a name="line.106"></a>
<span class="sourceLineNo">107</span> private TimeoutMonitor timeoutMonitor;<a name="line.107"></a>
<span class="sourceLineNo">108</span><a name="line.108"></a>
<span class="sourceLineNo">109</span> private volatile Set&lt;ServerName&gt; deadWorkers = null;<a name="line.109"></a>
<span class="sourceLineNo">110</span> private final Object deadWorkersLock = new Object();<a name="line.110"></a>
<span class="sourceLineNo">111</span><a name="line.111"></a>
<span class="sourceLineNo">112</span> /**<a name="line.112"></a>
<span class="sourceLineNo">113</span> * Its OK to construct this object even when region-servers are not online. It does lookup the<a name="line.113"></a>
<span class="sourceLineNo">114</span> * orphan tasks in coordination engine but it doesn't block waiting for them to be done.<a name="line.114"></a>
<span class="sourceLineNo">115</span> * @param master the master services<a name="line.115"></a>
<span class="sourceLineNo">116</span> * @param conf the HBase configuration<a name="line.116"></a>
<span class="sourceLineNo">117</span> * @throws IOException<a name="line.117"></a>
<span class="sourceLineNo">118</span> */<a name="line.118"></a>
<span class="sourceLineNo">119</span> public SplitLogManager(MasterServices master, Configuration conf)<a name="line.119"></a>
<span class="sourceLineNo">120</span> throws IOException {<a name="line.120"></a>
<span class="sourceLineNo">121</span> this.server = master;<a name="line.121"></a>
<span class="sourceLineNo">122</span> this.conf = conf;<a name="line.122"></a>
<span class="sourceLineNo">123</span> // If no CoordinatedStateManager, skip registering as a chore service (The<a name="line.123"></a>
<span class="sourceLineNo">124</span> // CoordinatedStateManager is non-null if we are running the ZK-based distributed WAL<a name="line.124"></a>
<span class="sourceLineNo">125</span> // splitting. It is null if we are configured to use procedure-based distributed WAL<a name="line.125"></a>
<span class="sourceLineNo">126</span> // splitting.<a name="line.126"></a>
<span class="sourceLineNo">127</span> if (server.getCoordinatedStateManager() != null) {<a name="line.127"></a>
<span class="sourceLineNo">128</span> this.choreService =<a name="line.128"></a>
<span class="sourceLineNo">129</span> new ChoreService(master.getServerName().toShortString() + ".splitLogManager.");<a name="line.129"></a>
<span class="sourceLineNo">130</span> SplitLogManagerCoordination coordination = getSplitLogManagerCoordination();<a name="line.130"></a>
<span class="sourceLineNo">131</span> Set&lt;String&gt; failedDeletions = Collections.synchronizedSet(new HashSet&lt;String&gt;());<a name="line.131"></a>
<span class="sourceLineNo">132</span> SplitLogManagerDetails details = new SplitLogManagerDetails(tasks, master, failedDeletions);<a name="line.132"></a>
<span class="sourceLineNo">133</span> coordination.setDetails(details);<a name="line.133"></a>
<span class="sourceLineNo">134</span> coordination.init();<a name="line.134"></a>
<span class="sourceLineNo">135</span> this.unassignedTimeout =<a name="line.135"></a>
<span class="sourceLineNo">136</span> conf.getInt("hbase.splitlog.manager.unassigned.timeout", DEFAULT_UNASSIGNED_TIMEOUT);<a name="line.136"></a>
<span class="sourceLineNo">137</span> this.timeoutMonitor =<a name="line.137"></a>
<span class="sourceLineNo">138</span> new TimeoutMonitor(conf.getInt("hbase.splitlog.manager.timeoutmonitor.period", 1000),<a name="line.138"></a>
<span class="sourceLineNo">139</span> master);<a name="line.139"></a>
<span class="sourceLineNo">140</span> this.choreService.scheduleChore(timeoutMonitor);<a name="line.140"></a>
<span class="sourceLineNo">141</span> } else {<a name="line.141"></a>
<span class="sourceLineNo">142</span> this.choreService = null;<a name="line.142"></a>
<span class="sourceLineNo">143</span> this.timeoutMonitor = null;<a name="line.143"></a>
<span class="sourceLineNo">144</span> }<a name="line.144"></a>
<span class="sourceLineNo">145</span> }<a name="line.145"></a>
<span class="sourceLineNo">146</span><a name="line.146"></a>
<span class="sourceLineNo">147</span> private SplitLogManagerCoordination getSplitLogManagerCoordination() {<a name="line.147"></a>
<span class="sourceLineNo">148</span> return server.getCoordinatedStateManager().getSplitLogManagerCoordination();<a name="line.148"></a>
<span class="sourceLineNo">149</span> }<a name="line.149"></a>
<span class="sourceLineNo">150</span><a name="line.150"></a>
<span class="sourceLineNo">151</span> private List&lt;FileStatus&gt; getFileList(List&lt;Path&gt; logDirs, PathFilter filter) throws IOException {<a name="line.151"></a>
<span class="sourceLineNo">152</span> return getFileList(conf, logDirs, filter);<a name="line.152"></a>
<span class="sourceLineNo">153</span> }<a name="line.153"></a>
<span class="sourceLineNo">154</span><a name="line.154"></a>
<span class="sourceLineNo">155</span> /**<a name="line.155"></a>
<span class="sourceLineNo">156</span> * Get a list of paths that need to be split given a set of server-specific directories and<a name="line.156"></a>
<span class="sourceLineNo">157</span> * optionally a filter.<a name="line.157"></a>
<span class="sourceLineNo">158</span> *<a name="line.158"></a>
<span class="sourceLineNo">159</span> * See {@link AbstractFSWALProvider#getServerNameFromWALDirectoryName} for more info on directory<a name="line.159"></a>
<span class="sourceLineNo">160</span> * layout.<a name="line.160"></a>
<span class="sourceLineNo">161</span> *<a name="line.161"></a>
<span class="sourceLineNo">162</span> * Should be package-private, but is needed by<a name="line.162"></a>
<span class="sourceLineNo">163</span> * {@link org.apache.hadoop.hbase.wal.WALSplitter#split(Path, Path, Path, FileSystem,<a name="line.163"></a>
<span class="sourceLineNo">164</span> * Configuration, org.apache.hadoop.hbase.wal.WALFactory)} for tests.<a name="line.164"></a>
<span class="sourceLineNo">165</span> */<a name="line.165"></a>
<span class="sourceLineNo">166</span> public static List&lt;FileStatus&gt; getFileList(final Configuration conf, final List&lt;Path&gt; logDirs,<a name="line.166"></a>
<span class="sourceLineNo">167</span> final PathFilter filter)<a name="line.167"></a>
<span class="sourceLineNo">168</span> throws IOException {<a name="line.168"></a>
<span class="sourceLineNo">169</span> List&lt;FileStatus&gt; fileStatus = new ArrayList&lt;&gt;();<a name="line.169"></a>
<span class="sourceLineNo">170</span> for (Path logDir : logDirs) {<a name="line.170"></a>
<span class="sourceLineNo">171</span> final FileSystem fs = logDir.getFileSystem(conf);<a name="line.171"></a>
<span class="sourceLineNo">172</span> if (!fs.exists(logDir)) {<a name="line.172"></a>
<span class="sourceLineNo">173</span> LOG.warn(logDir + " doesn't exist. Nothing to do!");<a name="line.173"></a>
<span class="sourceLineNo">174</span> continue;<a name="line.174"></a>
<span class="sourceLineNo">175</span> }<a name="line.175"></a>
<span class="sourceLineNo">176</span> FileStatus[] logfiles = CommonFSUtils.listStatus(fs, logDir, filter);<a name="line.176"></a>
<span class="sourceLineNo">177</span> if (logfiles == null || logfiles.length == 0) {<a name="line.177"></a>
<span class="sourceLineNo">178</span> LOG.info("{} dir is empty, no logs to split.", logDir);<a name="line.178"></a>
<span class="sourceLineNo">179</span> } else {<a name="line.179"></a>
<span class="sourceLineNo">180</span> Collections.addAll(fileStatus, logfiles);<a name="line.180"></a>
<span class="sourceLineNo">181</span> }<a name="line.181"></a>
<span class="sourceLineNo">182</span> }<a name="line.182"></a>
<span class="sourceLineNo">183</span><a name="line.183"></a>
<span class="sourceLineNo">184</span> return fileStatus;<a name="line.184"></a>
<span class="sourceLineNo">185</span> }<a name="line.185"></a>
<span class="sourceLineNo">186</span><a name="line.186"></a>
<span class="sourceLineNo">187</span> /**<a name="line.187"></a>
<span class="sourceLineNo">188</span> * @param logDir one region sever wal dir path in .logs<a name="line.188"></a>
<span class="sourceLineNo">189</span> * @throws IOException if there was an error while splitting any log file<a name="line.189"></a>
<span class="sourceLineNo">190</span> * @return cumulative size of the logfiles split<a name="line.190"></a>
<span class="sourceLineNo">191</span> * @throws IOException<a name="line.191"></a>
<span class="sourceLineNo">192</span> */<a name="line.192"></a>
<span class="sourceLineNo">193</span> public long splitLogDistributed(final Path logDir) throws IOException {<a name="line.193"></a>
<span class="sourceLineNo">194</span> List&lt;Path&gt; logDirs = new ArrayList&lt;&gt;();<a name="line.194"></a>
<span class="sourceLineNo">195</span> logDirs.add(logDir);<a name="line.195"></a>
<span class="sourceLineNo">196</span> return splitLogDistributed(logDirs);<a name="line.196"></a>
<span class="sourceLineNo">197</span> }<a name="line.197"></a>
<span class="sourceLineNo">198</span><a name="line.198"></a>
<span class="sourceLineNo">199</span> /**<a name="line.199"></a>
<span class="sourceLineNo">200</span> * The caller will block until all the log files of the given region server have been processed -<a name="line.200"></a>
<span class="sourceLineNo">201</span> * successfully split or an error is encountered - by an available worker region server. This<a name="line.201"></a>
<span class="sourceLineNo">202</span> * method must only be called after the region servers have been brought online.<a name="line.202"></a>
<span class="sourceLineNo">203</span> * @param logDirs List of log dirs to split<a name="line.203"></a>
<span class="sourceLineNo">204</span> * @throws IOException If there was an error while splitting any log file<a name="line.204"></a>
<span class="sourceLineNo">205</span> * @return cumulative size of the logfiles split<a name="line.205"></a>
<span class="sourceLineNo">206</span> */<a name="line.206"></a>
<span class="sourceLineNo">207</span> public long splitLogDistributed(final List&lt;Path&gt; logDirs) throws IOException {<a name="line.207"></a>
<span class="sourceLineNo">208</span> if (logDirs.isEmpty()) {<a name="line.208"></a>
<span class="sourceLineNo">209</span> return 0;<a name="line.209"></a>
<span class="sourceLineNo">210</span> }<a name="line.210"></a>
<span class="sourceLineNo">211</span> Set&lt;ServerName&gt; serverNames = new HashSet&lt;&gt;();<a name="line.211"></a>
<span class="sourceLineNo">212</span> for (Path logDir : logDirs) {<a name="line.212"></a>
<span class="sourceLineNo">213</span> try {<a name="line.213"></a>
<span class="sourceLineNo">214</span> ServerName serverName = AbstractFSWALProvider.getServerNameFromWALDirectoryName(logDir);<a name="line.214"></a>
<span class="sourceLineNo">215</span> if (serverName != null) {<a name="line.215"></a>
<span class="sourceLineNo">216</span> serverNames.add(serverName);<a name="line.216"></a>
<span class="sourceLineNo">217</span> }<a name="line.217"></a>
<span class="sourceLineNo">218</span> } catch (IllegalArgumentException e) {<a name="line.218"></a>
<span class="sourceLineNo">219</span> // ignore invalid format error.<a name="line.219"></a>
<span class="sourceLineNo">220</span> LOG.warn("Cannot parse server name from " + logDir);<a name="line.220"></a>
<span class="sourceLineNo">221</span> }<a name="line.221"></a>
<span class="sourceLineNo">222</span> }<a name="line.222"></a>
<span class="sourceLineNo">223</span> return splitLogDistributed(serverNames, logDirs, null);<a name="line.223"></a>
<span class="sourceLineNo">224</span> }<a name="line.224"></a>
<span class="sourceLineNo">225</span><a name="line.225"></a>
<span class="sourceLineNo">226</span> /**<a name="line.226"></a>
<span class="sourceLineNo">227</span> * The caller will block until all the hbase:meta log files of the given region server have been<a name="line.227"></a>
<span class="sourceLineNo">228</span> * processed - successfully split or an error is encountered - by an available worker region<a name="line.228"></a>
<span class="sourceLineNo">229</span> * server. This method must only be called after the region servers have been brought online.<a name="line.229"></a>
<span class="sourceLineNo">230</span> * @param logDirs List of log dirs to split<a name="line.230"></a>
<span class="sourceLineNo">231</span> * @param filter the Path filter to select specific files for considering<a name="line.231"></a>
<span class="sourceLineNo">232</span> * @throws IOException If there was an error while splitting any log file<a name="line.232"></a>
<span class="sourceLineNo">233</span> * @return cumulative size of the logfiles split<a name="line.233"></a>
<span class="sourceLineNo">234</span> */<a name="line.234"></a>
<span class="sourceLineNo">235</span> public long splitLogDistributed(final Set&lt;ServerName&gt; serverNames, final List&lt;Path&gt; logDirs,<a name="line.235"></a>
<span class="sourceLineNo">236</span> PathFilter filter) throws IOException {<a name="line.236"></a>
<span class="sourceLineNo">237</span> MonitoredTask status = TaskMonitor.get().createStatus("Doing distributed log split in " +<a name="line.237"></a>
<span class="sourceLineNo">238</span> logDirs + " for serverName=" + serverNames);<a name="line.238"></a>
<span class="sourceLineNo">239</span> long totalSize = 0;<a name="line.239"></a>
<span class="sourceLineNo">240</span> TaskBatch batch = null;<a name="line.240"></a>
<span class="sourceLineNo">241</span> long startTime = 0;<a name="line.241"></a>
<span class="sourceLineNo">242</span> List&lt;FileStatus&gt; logfiles = getFileList(logDirs, filter);<a name="line.242"></a>
<span class="sourceLineNo">243</span> if (!logfiles.isEmpty()) {<a name="line.243"></a>
<span class="sourceLineNo">244</span> status.setStatus("Checking directory contents...");<a name="line.244"></a>
<span class="sourceLineNo">245</span> SplitLogCounters.tot_mgr_log_split_batch_start.increment();<a name="line.245"></a>
<span class="sourceLineNo">246</span> LOG.info("Started splitting " + logfiles.size() + " logs in " + logDirs +<a name="line.246"></a>
<span class="sourceLineNo">247</span> " for " + serverNames);<a name="line.247"></a>
<span class="sourceLineNo">248</span> startTime = EnvironmentEdgeManager.currentTime();<a name="line.248"></a>
<span class="sourceLineNo">249</span> batch = new TaskBatch();<a name="line.249"></a>
<span class="sourceLineNo">250</span> for (FileStatus lf : logfiles) {<a name="line.250"></a>
<span class="sourceLineNo">251</span> // TODO If the log file is still being written to - which is most likely<a name="line.251"></a>
<span class="sourceLineNo">252</span> // the case for the last log file - then its length will show up here<a name="line.252"></a>
<span class="sourceLineNo">253</span> // as zero. The size of such a file can only be retrieved after<a name="line.253"></a>
<span class="sourceLineNo">254</span> // recover-lease is done. totalSize will be under in most cases and the<a name="line.254"></a>
<span class="sourceLineNo">255</span> // metrics that it drives will also be under-reported.<a name="line.255"></a>
<span class="sourceLineNo">256</span> totalSize += lf.getLen();<a name="line.256"></a>
<span class="sourceLineNo">257</span> String pathToLog = CommonFSUtils.removeWALRootPath(lf.getPath(), conf);<a name="line.257"></a>
<span class="sourceLineNo">258</span> if (!enqueueSplitTask(pathToLog, batch)) {<a name="line.258"></a>
<span class="sourceLineNo">259</span> throw new IOException("duplicate log split scheduled for " + lf.getPath());<a name="line.259"></a>
<span class="sourceLineNo">260</span> }<a name="line.260"></a>
<span class="sourceLineNo">261</span> }<a name="line.261"></a>
<span class="sourceLineNo">262</span> waitForSplittingCompletion(batch, status);<a name="line.262"></a>
<span class="sourceLineNo">263</span> }<a name="line.263"></a>
<span class="sourceLineNo">264</span><a name="line.264"></a>
<span class="sourceLineNo">265</span> if (batch != null &amp;&amp; batch.done != batch.installed) {<a name="line.265"></a>
<span class="sourceLineNo">266</span> batch.isDead = true;<a name="line.266"></a>
<span class="sourceLineNo">267</span> SplitLogCounters.tot_mgr_log_split_batch_err.increment();<a name="line.267"></a>
<span class="sourceLineNo">268</span> LOG.warn("error while splitting logs in " + logDirs + " installed = " + batch.installed<a name="line.268"></a>
<span class="sourceLineNo">269</span> + " but only " + batch.done + " done");<a name="line.269"></a>
<span class="sourceLineNo">270</span> String msg = "error or interrupted while splitting logs in " + logDirs + " Task = " + batch;<a name="line.270"></a>
<span class="sourceLineNo">271</span> status.abort(msg);<a name="line.271"></a>
<span class="sourceLineNo">272</span> throw new IOException(msg);<a name="line.272"></a>
<span class="sourceLineNo">273</span> }<a name="line.273"></a>
<span class="sourceLineNo">274</span> for (Path logDir : logDirs) {<a name="line.274"></a>
<span class="sourceLineNo">275</span> status.setStatus("Cleaning up log directory...");<a name="line.275"></a>
<span class="sourceLineNo">276</span> final FileSystem fs = logDir.getFileSystem(conf);<a name="line.276"></a>
<span class="sourceLineNo">277</span> try {<a name="line.277"></a>
<span class="sourceLineNo">278</span> if (fs.exists(logDir) &amp;&amp; !fs.delete(logDir, false)) {<a name="line.278"></a>
<span class="sourceLineNo">279</span> LOG.warn("Unable to delete log src dir. Ignoring. " + logDir);<a name="line.279"></a>
<span class="sourceLineNo">280</span> }<a name="line.280"></a>
<span class="sourceLineNo">281</span> } catch (IOException ioe) {<a name="line.281"></a>
<span class="sourceLineNo">282</span> FileStatus[] files = fs.listStatus(logDir);<a name="line.282"></a>
<span class="sourceLineNo">283</span> if (files != null &amp;&amp; files.length &gt; 0) {<a name="line.283"></a>
<span class="sourceLineNo">284</span> LOG.warn("Returning success without actually splitting and "<a name="line.284"></a>
<span class="sourceLineNo">285</span> + "deleting all the log files in path " + logDir + ": "<a name="line.285"></a>
<span class="sourceLineNo">286</span> + Arrays.toString(files), ioe);<a name="line.286"></a>
<span class="sourceLineNo">287</span> } else {<a name="line.287"></a>
<span class="sourceLineNo">288</span> LOG.warn("Unable to delete log src dir. Ignoring. " + logDir, ioe);<a name="line.288"></a>
<span class="sourceLineNo">289</span> }<a name="line.289"></a>
<span class="sourceLineNo">290</span> }<a name="line.290"></a>
<span class="sourceLineNo">291</span> SplitLogCounters.tot_mgr_log_split_batch_success.increment();<a name="line.291"></a>
<span class="sourceLineNo">292</span> }<a name="line.292"></a>
<span class="sourceLineNo">293</span> String msg =<a name="line.293"></a>
<span class="sourceLineNo">294</span> "Finished splitting (more than or equal to) " + StringUtils.humanSize(totalSize) + " (" +<a name="line.294"></a>
<span class="sourceLineNo">295</span> totalSize + " bytes) in " + ((batch == null) ? 0 : batch.installed) + " log files in " +<a name="line.295"></a>
<span class="sourceLineNo">296</span> logDirs + " in " +<a name="line.296"></a>
<span class="sourceLineNo">297</span> ((startTime == 0) ? startTime : (EnvironmentEdgeManager.currentTime() - startTime)) +<a name="line.297"></a>
<span class="sourceLineNo">298</span> "ms";<a name="line.298"></a>
<span class="sourceLineNo">299</span> status.markComplete(msg);<a name="line.299"></a>
<span class="sourceLineNo">300</span> LOG.info(msg);<a name="line.300"></a>
<span class="sourceLineNo">301</span> return totalSize;<a name="line.301"></a>
<span class="sourceLineNo">302</span> }<a name="line.302"></a>
<span class="sourceLineNo">303</span><a name="line.303"></a>
<span class="sourceLineNo">304</span> /**<a name="line.304"></a>
<span class="sourceLineNo">305</span> * Add a task entry to coordination if it is not already there.<a name="line.305"></a>
<span class="sourceLineNo">306</span> * @param taskname the path of the log to be split<a name="line.306"></a>
<span class="sourceLineNo">307</span> * @param batch the batch this task belongs to<a name="line.307"></a>
<span class="sourceLineNo">308</span> * @return true if a new entry is created, false if it is already there.<a name="line.308"></a>
<span class="sourceLineNo">309</span> */<a name="line.309"></a>
<span class="sourceLineNo">310</span> boolean enqueueSplitTask(String taskname, TaskBatch batch) {<a name="line.310"></a>
<span class="sourceLineNo">311</span> lastTaskCreateTime = EnvironmentEdgeManager.currentTime();<a name="line.311"></a>
<span class="sourceLineNo">312</span> String task = getSplitLogManagerCoordination().prepareTask(taskname);<a name="line.312"></a>
<span class="sourceLineNo">313</span> Task oldtask = createTaskIfAbsent(task, batch);<a name="line.313"></a>
<span class="sourceLineNo">314</span> if (oldtask == null) {<a name="line.314"></a>
<span class="sourceLineNo">315</span> // publish the task in the coordination engine<a name="line.315"></a>
<span class="sourceLineNo">316</span> getSplitLogManagerCoordination().submitTask(task);<a name="line.316"></a>
<span class="sourceLineNo">317</span> return true;<a name="line.317"></a>
<span class="sourceLineNo">318</span> }<a name="line.318"></a>
<span class="sourceLineNo">319</span> return false;<a name="line.319"></a>
<span class="sourceLineNo">320</span> }<a name="line.320"></a>
<span class="sourceLineNo">321</span><a name="line.321"></a>
<span class="sourceLineNo">322</span> /**<a name="line.322"></a>
<span class="sourceLineNo">323</span> * Get the amount of time in milliseconds to wait till next check.<a name="line.323"></a>
<span class="sourceLineNo">324</span> * Check less frequently if a bunch of work to do still. At a max, check every minute.<a name="line.324"></a>
<span class="sourceLineNo">325</span> * At a minimum, check every 100ms. This is to alleviate case where perhaps there are a bunch of<a name="line.325"></a>
<span class="sourceLineNo">326</span> * threads waiting on a completion. For example, if the zk-based implementation, we will scan the<a name="line.326"></a>
<span class="sourceLineNo">327</span> * '/hbase/splitWAL' dir every time through this loop. If there are lots of WALs to<a name="line.327"></a>
<span class="sourceLineNo">328</span> * split -- could be tens of thousands if big cluster -- then it will take a while. If<a name="line.328"></a>
<span class="sourceLineNo">329</span> * the Master has many SCPs waiting on wal splitting -- could be up to 10 x the configured<a name="line.329"></a>
<span class="sourceLineNo">330</span> * PE thread count (default would be 160) -- then the Master will be putting up a bunch of<a name="line.330"></a>
<span class="sourceLineNo">331</span> * load on zk.<a name="line.331"></a>
<span class="sourceLineNo">332</span> */<a name="line.332"></a>
<span class="sourceLineNo">333</span> static int getBatchWaitTimeMillis(int remainingTasks) {<a name="line.333"></a>
<span class="sourceLineNo">334</span> return remainingTasks &lt; 10? 100: remainingTasks &lt; 100? 1000: 60_000;<a name="line.334"></a>
<span class="sourceLineNo">335</span> }<a name="line.335"></a>
<span class="sourceLineNo">336</span><a name="line.336"></a>
<span class="sourceLineNo">337</span> private void waitForSplittingCompletion(TaskBatch batch, MonitoredTask status) {<a name="line.337"></a>
<span class="sourceLineNo">338</span> synchronized (batch) {<a name="line.338"></a>
<span class="sourceLineNo">339</span> while ((batch.done + batch.error) != batch.installed) {<a name="line.339"></a>
<span class="sourceLineNo">340</span> try {<a name="line.340"></a>
<span class="sourceLineNo">341</span> status.setStatus("Waiting for distributed tasks to finish. " + " scheduled="<a name="line.341"></a>
<span class="sourceLineNo">342</span> + batch.installed + " done=" + batch.done + " error=" + batch.error);<a name="line.342"></a>
<span class="sourceLineNo">343</span> int remaining = batch.installed - (batch.done + batch.error);<a name="line.343"></a>
<span class="sourceLineNo">344</span> int actual = activeTasks(batch);<a name="line.344"></a>
<span class="sourceLineNo">345</span> if (remaining != actual) {<a name="line.345"></a>
<span class="sourceLineNo">346</span> LOG.warn("Expected " + remaining + " active tasks, but actually there are " + actual);<a name="line.346"></a>
<span class="sourceLineNo">347</span> }<a name="line.347"></a>
<span class="sourceLineNo">348</span> int remainingTasks = getSplitLogManagerCoordination().remainingTasksInCoordination();<a name="line.348"></a>
<span class="sourceLineNo">349</span> if (remainingTasks &gt;= 0 &amp;&amp; actual &gt; remainingTasks) {<a name="line.349"></a>
<span class="sourceLineNo">350</span> LOG.warn("Expected at least" + actual + " tasks remaining, but actually there are "<a name="line.350"></a>
<span class="sourceLineNo">351</span> + remainingTasks);<a name="line.351"></a>
<span class="sourceLineNo">352</span> }<a name="line.352"></a>
<span class="sourceLineNo">353</span> if (remainingTasks == 0 || actual == 0) {<a name="line.353"></a>
<span class="sourceLineNo">354</span> LOG.warn("No more task remaining, splitting "<a name="line.354"></a>
<span class="sourceLineNo">355</span> + "should have completed. Remaining tasks is " + remainingTasks<a name="line.355"></a>
<span class="sourceLineNo">356</span> + ", active tasks in map " + actual);<a name="line.356"></a>
<span class="sourceLineNo">357</span> if (remainingTasks == 0 &amp;&amp; actual == 0) {<a name="line.357"></a>
<span class="sourceLineNo">358</span> return;<a name="line.358"></a>
<span class="sourceLineNo">359</span> }<a name="line.359"></a>
<span class="sourceLineNo">360</span> }<a name="line.360"></a>
<span class="sourceLineNo">361</span> batch.wait(getBatchWaitTimeMillis(remainingTasks));<a name="line.361"></a>
<span class="sourceLineNo">362</span> if (server.isStopped()) {<a name="line.362"></a>
<span class="sourceLineNo">363</span> LOG.warn("Stopped while waiting for log splits to be completed");<a name="line.363"></a>
<span class="sourceLineNo">364</span> return;<a name="line.364"></a>
<span class="sourceLineNo">365</span> }<a name="line.365"></a>
<span class="sourceLineNo">366</span> } catch (InterruptedException e) {<a name="line.366"></a>
<span class="sourceLineNo">367</span> LOG.warn("Interrupted while waiting for log splits to be completed");<a name="line.367"></a>
<span class="sourceLineNo">368</span> Thread.currentThread().interrupt();<a name="line.368"></a>
<span class="sourceLineNo">369</span> return;<a name="line.369"></a>
<span class="sourceLineNo">370</span> }<a name="line.370"></a>
<span class="sourceLineNo">371</span> }<a name="line.371"></a>
<span class="sourceLineNo">372</span> }<a name="line.372"></a>
<span class="sourceLineNo">373</span> }<a name="line.373"></a>
<span class="sourceLineNo">374</span><a name="line.374"></a>
<span class="sourceLineNo">375</span> ConcurrentMap&lt;String, Task&gt; getTasks() {<a name="line.375"></a>
<span class="sourceLineNo">376</span> return tasks;<a name="line.376"></a>
<span class="sourceLineNo">377</span> }<a name="line.377"></a>
<span class="sourceLineNo">378</span><a name="line.378"></a>
<span class="sourceLineNo">379</span> private int activeTasks(final TaskBatch batch) {<a name="line.379"></a>
<span class="sourceLineNo">380</span> int count = 0;<a name="line.380"></a>
<span class="sourceLineNo">381</span> for (Task t : tasks.values()) {<a name="line.381"></a>
<span class="sourceLineNo">382</span> if (t.batch == batch &amp;&amp; t.status == TerminationStatus.IN_PROGRESS) {<a name="line.382"></a>
<span class="sourceLineNo">383</span> count++;<a name="line.383"></a>
<span class="sourceLineNo">384</span> }<a name="line.384"></a>
<span class="sourceLineNo">385</span> }<a name="line.385"></a>
<span class="sourceLineNo">386</span> return count;<a name="line.386"></a>
<span class="sourceLineNo">387</span><a name="line.387"></a>
<span class="sourceLineNo">388</span> }<a name="line.388"></a>
<span class="sourceLineNo">389</span><a name="line.389"></a>
<span class="sourceLineNo">390</span> /**<a name="line.390"></a>
<span class="sourceLineNo">391</span> * @param path<a name="line.391"></a>
<span class="sourceLineNo">392</span> * @param batch<a name="line.392"></a>
<span class="sourceLineNo">393</span> * @return null on success, existing task on error<a name="line.393"></a>
<span class="sourceLineNo">394</span> */<a name="line.394"></a>
<span class="sourceLineNo">395</span> private Task createTaskIfAbsent(String path, TaskBatch batch) {<a name="line.395"></a>
<span class="sourceLineNo">396</span> Task oldtask;<a name="line.396"></a>
<span class="sourceLineNo">397</span> // batch.installed is only changed via this function and<a name="line.397"></a>
<span class="sourceLineNo">398</span> // a single thread touches batch.installed.<a name="line.398"></a>
<span class="sourceLineNo">399</span> Task newtask = new Task();<a name="line.399"></a>
<span class="sourceLineNo">400</span> newtask.batch = batch;<a name="line.400"></a>
<span class="sourceLineNo">401</span> oldtask = tasks.putIfAbsent(path, newtask);<a name="line.401"></a>
<span class="sourceLineNo">402</span> if (oldtask == null) {<a name="line.402"></a>
<span class="sourceLineNo">403</span> batch.installed++;<a name="line.403"></a>
<span class="sourceLineNo">404</span> return null;<a name="line.404"></a>
<span class="sourceLineNo">405</span> }<a name="line.405"></a>
<span class="sourceLineNo">406</span> // new task was not used.<a name="line.406"></a>
<span class="sourceLineNo">407</span> synchronized (oldtask) {<a name="line.407"></a>
<span class="sourceLineNo">408</span> if (oldtask.isOrphan()) {<a name="line.408"></a>
<span class="sourceLineNo">409</span> if (oldtask.status == SUCCESS) {<a name="line.409"></a>
<span class="sourceLineNo">410</span> // The task is already done. Do not install the batch for this<a name="line.410"></a>
<span class="sourceLineNo">411</span> // task because it might be too late for setDone() to update<a name="line.411"></a>
<span class="sourceLineNo">412</span> // batch.done. There is no need for the batch creator to wait for<a name="line.412"></a>
<span class="sourceLineNo">413</span> // this task to complete.<a name="line.413"></a>
<span class="sourceLineNo">414</span> return (null);<a name="line.414"></a>
<span class="sourceLineNo">415</span> }<a name="line.415"></a>
<span class="sourceLineNo">416</span> if (oldtask.status == IN_PROGRESS) {<a name="line.416"></a>
<span class="sourceLineNo">417</span> oldtask.batch = batch;<a name="line.417"></a>
<span class="sourceLineNo">418</span> batch.installed++;<a name="line.418"></a>
<span class="sourceLineNo">419</span> LOG.debug("Previously orphan task " + path + " is now being waited upon");<a name="line.419"></a>
<span class="sourceLineNo">420</span> return null;<a name="line.420"></a>
<span class="sourceLineNo">421</span> }<a name="line.421"></a>
<span class="sourceLineNo">422</span> while (oldtask.status == FAILURE) {<a name="line.422"></a>
<span class="sourceLineNo">423</span> LOG.debug("wait for status of task " + path + " to change to DELETED");<a name="line.423"></a>
<span class="sourceLineNo">424</span> SplitLogCounters.tot_mgr_wait_for_zk_delete.increment();<a name="line.424"></a>
<span class="sourceLineNo">425</span> try {<a name="line.425"></a>
<span class="sourceLineNo">426</span> oldtask.wait();<a name="line.426"></a>
<span class="sourceLineNo">427</span> } catch (InterruptedException e) {<a name="line.427"></a>
<span class="sourceLineNo">428</span> Thread.currentThread().interrupt();<a name="line.428"></a>
<span class="sourceLineNo">429</span> LOG.warn("Interrupted when waiting for znode delete callback");<a name="line.429"></a>
<span class="sourceLineNo">430</span> // fall through to return failure<a name="line.430"></a>
<span class="sourceLineNo">431</span> break;<a name="line.431"></a>
<span class="sourceLineNo">432</span> }<a name="line.432"></a>
<span class="sourceLineNo">433</span> }<a name="line.433"></a>
<span class="sourceLineNo">434</span> if (oldtask.status != DELETED) {<a name="line.434"></a>
<span class="sourceLineNo">435</span> LOG.warn("Failure because previously failed task"<a name="line.435"></a>
<span class="sourceLineNo">436</span> + " state still present. Waiting for znode delete callback" + " path=" + path);<a name="line.436"></a>
<span class="sourceLineNo">437</span> return oldtask;<a name="line.437"></a>
<span class="sourceLineNo">438</span> }<a name="line.438"></a>
<span class="sourceLineNo">439</span> // reinsert the newTask and it must succeed this time<a name="line.439"></a>
<span class="sourceLineNo">440</span> Task t = tasks.putIfAbsent(path, newtask);<a name="line.440"></a>
<span class="sourceLineNo">441</span> if (t == null) {<a name="line.441"></a>
<span class="sourceLineNo">442</span> batch.installed++;<a name="line.442"></a>
<span class="sourceLineNo">443</span> return null;<a name="line.443"></a>
<span class="sourceLineNo">444</span> }<a name="line.444"></a>
<span class="sourceLineNo">445</span> LOG.error(HBaseMarkers.FATAL, "Logic error. Deleted task still present in tasks map");<a name="line.445"></a>
<span class="sourceLineNo">446</span> assert false : "Deleted task still present in tasks map";<a name="line.446"></a>
<span class="sourceLineNo">447</span> return t;<a name="line.447"></a>
<span class="sourceLineNo">448</span> }<a name="line.448"></a>
<span class="sourceLineNo">449</span> LOG.warn("Failure because two threads can't wait for the same task; path=" + path);<a name="line.449"></a>
<span class="sourceLineNo">450</span> return oldtask;<a name="line.450"></a>
<span class="sourceLineNo">451</span> }<a name="line.451"></a>
<span class="sourceLineNo">452</span> }<a name="line.452"></a>
<span class="sourceLineNo">453</span><a name="line.453"></a>
<span class="sourceLineNo">454</span> public void stop() {<a name="line.454"></a>
<span class="sourceLineNo">455</span> if (choreService != null) {<a name="line.455"></a>
<span class="sourceLineNo">456</span> choreService.shutdown();<a name="line.456"></a>
<span class="sourceLineNo">457</span> }<a name="line.457"></a>
<span class="sourceLineNo">458</span> if (timeoutMonitor != null) {<a name="line.458"></a>
<span class="sourceLineNo">459</span> timeoutMonitor.shutdown(true);<a name="line.459"></a>
<span class="sourceLineNo">460</span> }<a name="line.460"></a>
<span class="sourceLineNo">461</span> }<a name="line.461"></a>
<span class="sourceLineNo">462</span><a name="line.462"></a>
<span class="sourceLineNo">463</span> void handleDeadWorker(ServerName workerName) {<a name="line.463"></a>
<span class="sourceLineNo">464</span> // resubmit the tasks on the TimeoutMonitor thread. Makes it easier<a name="line.464"></a>
<span class="sourceLineNo">465</span> // to reason about concurrency. Makes it easier to retry.<a name="line.465"></a>
<span class="sourceLineNo">466</span> synchronized (deadWorkersLock) {<a name="line.466"></a>
<span class="sourceLineNo">467</span> if (deadWorkers == null) {<a name="line.467"></a>
<span class="sourceLineNo">468</span> deadWorkers = new HashSet&lt;&gt;(100);<a name="line.468"></a>
<span class="sourceLineNo">469</span> }<a name="line.469"></a>
<span class="sourceLineNo">470</span> deadWorkers.add(workerName);<a name="line.470"></a>
<span class="sourceLineNo">471</span> }<a name="line.471"></a>
<span class="sourceLineNo">472</span> LOG.info("Dead splitlog worker {}", workerName);<a name="line.472"></a>
<span class="sourceLineNo">473</span> }<a name="line.473"></a>
<span class="sourceLineNo">474</span><a name="line.474"></a>
<span class="sourceLineNo">475</span> void handleDeadWorkers(Set&lt;ServerName&gt; serverNames) {<a name="line.475"></a>
<span class="sourceLineNo">476</span> synchronized (deadWorkersLock) {<a name="line.476"></a>
<span class="sourceLineNo">477</span> if (deadWorkers == null) {<a name="line.477"></a>
<span class="sourceLineNo">478</span> deadWorkers = new HashSet&lt;&gt;(100);<a name="line.478"></a>
<span class="sourceLineNo">479</span> }<a name="line.479"></a>
<span class="sourceLineNo">480</span> deadWorkers.addAll(serverNames);<a name="line.480"></a>
<span class="sourceLineNo">481</span> }<a name="line.481"></a>
<span class="sourceLineNo">482</span> LOG.info("dead splitlog workers " + serverNames);<a name="line.482"></a>
<span class="sourceLineNo">483</span> }<a name="line.483"></a>
<span class="sourceLineNo">484</span><a name="line.484"></a>
<span class="sourceLineNo">485</span> /**<a name="line.485"></a>
<span class="sourceLineNo">486</span> * Keeps track of the batch of tasks submitted together by a caller in splitLogDistributed().<a name="line.486"></a>
<span class="sourceLineNo">487</span> * Clients threads use this object to wait for all their tasks to be done.<a name="line.487"></a>
<span class="sourceLineNo">488</span> * &lt;p&gt;<a name="line.488"></a>
<span class="sourceLineNo">489</span> * All access is synchronized.<a name="line.489"></a>
<span class="sourceLineNo">490</span> */<a name="line.490"></a>
<span class="sourceLineNo">491</span> @InterfaceAudience.Private<a name="line.491"></a>
<span class="sourceLineNo">492</span> public static class TaskBatch {<a name="line.492"></a>
<span class="sourceLineNo">493</span> public int installed = 0;<a name="line.493"></a>
<span class="sourceLineNo">494</span> public int done = 0;<a name="line.494"></a>
<span class="sourceLineNo">495</span> public int error = 0;<a name="line.495"></a>
<span class="sourceLineNo">496</span> public volatile boolean isDead = false;<a name="line.496"></a>
<span class="sourceLineNo">497</span><a name="line.497"></a>
<span class="sourceLineNo">498</span> @Override<a name="line.498"></a>
<span class="sourceLineNo">499</span> public String toString() {<a name="line.499"></a>
<span class="sourceLineNo">500</span> return ("installed = " + installed + " done = " + done + " error = " + error);<a name="line.500"></a>
<span class="sourceLineNo">501</span> }<a name="line.501"></a>
<span class="sourceLineNo">502</span> }<a name="line.502"></a>
<span class="sourceLineNo">503</span><a name="line.503"></a>
<span class="sourceLineNo">504</span> /**<a name="line.504"></a>
<span class="sourceLineNo">505</span> * in memory state of an active task.<a name="line.505"></a>
<span class="sourceLineNo">506</span> */<a name="line.506"></a>
<span class="sourceLineNo">507</span> @InterfaceAudience.Private<a name="line.507"></a>
<span class="sourceLineNo">508</span> public static class Task {<a name="line.508"></a>
<span class="sourceLineNo">509</span> public volatile long last_update;<a name="line.509"></a>
<span class="sourceLineNo">510</span> public volatile int last_version;<a name="line.510"></a>
<span class="sourceLineNo">511</span> public volatile ServerName cur_worker_name;<a name="line.511"></a>
<span class="sourceLineNo">512</span> public volatile TaskBatch batch;<a name="line.512"></a>
<span class="sourceLineNo">513</span> public volatile TerminationStatus status;<a name="line.513"></a>
<span class="sourceLineNo">514</span> public volatile AtomicInteger incarnation = new AtomicInteger(0);<a name="line.514"></a>
<span class="sourceLineNo">515</span> public final AtomicInteger unforcedResubmits = new AtomicInteger();<a name="line.515"></a>
<span class="sourceLineNo">516</span> public volatile boolean resubmitThresholdReached;<a name="line.516"></a>
<span class="sourceLineNo">517</span><a name="line.517"></a>
<span class="sourceLineNo">518</span> @Override<a name="line.518"></a>
<span class="sourceLineNo">519</span> public String toString() {<a name="line.519"></a>
<span class="sourceLineNo">520</span> return ("last_update = " + last_update + " last_version = " + last_version<a name="line.520"></a>
<span class="sourceLineNo">521</span> + " cur_worker_name = " + cur_worker_name + " status = " + status + " incarnation = "<a name="line.521"></a>
<span class="sourceLineNo">522</span> + incarnation + " resubmits = " + unforcedResubmits.get() + " batch = " + batch);<a name="line.522"></a>
<span class="sourceLineNo">523</span> }<a name="line.523"></a>
<span class="sourceLineNo">524</span><a name="line.524"></a>
<span class="sourceLineNo">525</span> public Task() {<a name="line.525"></a>
<span class="sourceLineNo">526</span> last_version = -1;<a name="line.526"></a>
<span class="sourceLineNo">527</span> status = IN_PROGRESS;<a name="line.527"></a>
<span class="sourceLineNo">528</span> setUnassigned();<a name="line.528"></a>
<span class="sourceLineNo">529</span> }<a name="line.529"></a>
<span class="sourceLineNo">530</span><a name="line.530"></a>
<span class="sourceLineNo">531</span> public boolean isOrphan() {<a name="line.531"></a>
<span class="sourceLineNo">532</span> return (batch == null || batch.isDead);<a name="line.532"></a>
<span class="sourceLineNo">533</span> }<a name="line.533"></a>
<span class="sourceLineNo">534</span><a name="line.534"></a>
<span class="sourceLineNo">535</span> public boolean isUnassigned() {<a name="line.535"></a>
<span class="sourceLineNo">536</span> return (cur_worker_name == null);<a name="line.536"></a>
<span class="sourceLineNo">537</span> }<a name="line.537"></a>
<span class="sourceLineNo">538</span><a name="line.538"></a>
<span class="sourceLineNo">539</span> public void heartbeatNoDetails(long time) {<a name="line.539"></a>
<span class="sourceLineNo">540</span> last_update = time;<a name="line.540"></a>
<span class="sourceLineNo">541</span> }<a name="line.541"></a>
<span class="sourceLineNo">542</span><a name="line.542"></a>
<span class="sourceLineNo">543</span> public void heartbeat(long time, int version, ServerName worker) {<a name="line.543"></a>
<span class="sourceLineNo">544</span> last_version = version;<a name="line.544"></a>
<span class="sourceLineNo">545</span> last_update = time;<a name="line.545"></a>
<span class="sourceLineNo">546</span> cur_worker_name = worker;<a name="line.546"></a>
<span class="sourceLineNo">547</span> }<a name="line.547"></a>
<span class="sourceLineNo">548</span><a name="line.548"></a>
<span class="sourceLineNo">549</span> public void setUnassigned() {<a name="line.549"></a>
<span class="sourceLineNo">550</span> cur_worker_name = null;<a name="line.550"></a>
<span class="sourceLineNo">551</span> last_update = -1;<a name="line.551"></a>
<span class="sourceLineNo">552</span> }<a name="line.552"></a>
<span class="sourceLineNo">553</span> }<a name="line.553"></a>
<span class="sourceLineNo">554</span><a name="line.554"></a>
<span class="sourceLineNo">555</span> /**<a name="line.555"></a>
<span class="sourceLineNo">556</span> * Periodically checks all active tasks and resubmits the ones that have timed out<a name="line.556"></a>
<span class="sourceLineNo">557</span> */<a name="line.557"></a>
<span class="sourceLineNo">558</span> private class TimeoutMonitor extends ScheduledChore {<a name="line.558"></a>
<span class="sourceLineNo">559</span> private long lastLog = 0;<a name="line.559"></a>
<span class="sourceLineNo">560</span><a name="line.560"></a>
<span class="sourceLineNo">561</span> public TimeoutMonitor(final int period, Stoppable stopper) {<a name="line.561"></a>
<span class="sourceLineNo">562</span> super("SplitLogManager Timeout Monitor", stopper, period);<a name="line.562"></a>
<span class="sourceLineNo">563</span> }<a name="line.563"></a>
<span class="sourceLineNo">564</span><a name="line.564"></a>
<span class="sourceLineNo">565</span> @Override<a name="line.565"></a>
<span class="sourceLineNo">566</span> protected void chore() {<a name="line.566"></a>
<span class="sourceLineNo">567</span> if (server.getCoordinatedStateManager() == null) {<a name="line.567"></a>
<span class="sourceLineNo">568</span> return;<a name="line.568"></a>
<span class="sourceLineNo">569</span> }<a name="line.569"></a>
<span class="sourceLineNo">570</span><a name="line.570"></a>
<span class="sourceLineNo">571</span> int resubmitted = 0;<a name="line.571"></a>
<span class="sourceLineNo">572</span> int unassigned = 0;<a name="line.572"></a>
<span class="sourceLineNo">573</span> int tot = 0;<a name="line.573"></a>
<span class="sourceLineNo">574</span> boolean found_assigned_task = false;<a name="line.574"></a>
<span class="sourceLineNo">575</span> Set&lt;ServerName&gt; localDeadWorkers;<a name="line.575"></a>
<span class="sourceLineNo">576</span><a name="line.576"></a>
<span class="sourceLineNo">577</span> synchronized (deadWorkersLock) {<a name="line.577"></a>
<span class="sourceLineNo">578</span> localDeadWorkers = deadWorkers;<a name="line.578"></a>
<span class="sourceLineNo">579</span> deadWorkers = null;<a name="line.579"></a>
<span class="sourceLineNo">580</span> }<a name="line.580"></a>
<span class="sourceLineNo">581</span><a name="line.581"></a>
<span class="sourceLineNo">582</span> for (Map.Entry&lt;String, Task&gt; e : tasks.entrySet()) {<a name="line.582"></a>
<span class="sourceLineNo">583</span> String path = e.getKey();<a name="line.583"></a>
<span class="sourceLineNo">584</span> Task task = e.getValue();<a name="line.584"></a>
<span class="sourceLineNo">585</span> ServerName cur_worker = task.cur_worker_name;<a name="line.585"></a>
<span class="sourceLineNo">586</span> tot++;<a name="line.586"></a>
<span class="sourceLineNo">587</span> // don't easily resubmit a task which hasn't been picked up yet. It<a name="line.587"></a>
<span class="sourceLineNo">588</span> // might be a long while before a SplitLogWorker is free to pick up a<a name="line.588"></a>
<span class="sourceLineNo">589</span> // task. This is because a SplitLogWorker picks up a task one at a<a name="line.589"></a>
<span class="sourceLineNo">590</span> // time. If we want progress when there are no region servers then we<a name="line.590"></a>
<span class="sourceLineNo">591</span> // will have to run a SplitLogWorker thread in the Master.<a name="line.591"></a>
<span class="sourceLineNo">592</span> if (task.isUnassigned()) {<a name="line.592"></a>
<span class="sourceLineNo">593</span> unassigned++;<a name="line.593"></a>
<span class="sourceLineNo">594</span> continue;<a name="line.594"></a>
<span class="sourceLineNo">595</span> }<a name="line.595"></a>
<span class="sourceLineNo">596</span> found_assigned_task = true;<a name="line.596"></a>
<span class="sourceLineNo">597</span> if (localDeadWorkers != null &amp;&amp; localDeadWorkers.contains(cur_worker)) {<a name="line.597"></a>
<span class="sourceLineNo">598</span> SplitLogCounters.tot_mgr_resubmit_dead_server_task.increment();<a name="line.598"></a>
<span class="sourceLineNo">599</span> if (getSplitLogManagerCoordination().resubmitTask(path, task, FORCE)) {<a name="line.599"></a>
<span class="sourceLineNo">600</span> resubmitted++;<a name="line.600"></a>
<span class="sourceLineNo">601</span> } else {<a name="line.601"></a>
<span class="sourceLineNo">602</span> handleDeadWorker(cur_worker);<a name="line.602"></a>
<span class="sourceLineNo">603</span> LOG.warn("Failed to resubmit task " + path + " owned by dead " + cur_worker<a name="line.603"></a>
<span class="sourceLineNo">604</span> + ", will retry.");<a name="line.604"></a>
<span class="sourceLineNo">605</span> }<a name="line.605"></a>
<span class="sourceLineNo">606</span> } else if (getSplitLogManagerCoordination().resubmitTask(path, task, CHECK)) {<a name="line.606"></a>
<span class="sourceLineNo">607</span> resubmitted++;<a name="line.607"></a>
<span class="sourceLineNo">608</span> }<a name="line.608"></a>
<span class="sourceLineNo">609</span> }<a name="line.609"></a>
<span class="sourceLineNo">610</span> if (tot &gt; 0) {<a name="line.610"></a>
<span class="sourceLineNo">611</span> long now = EnvironmentEdgeManager.currentTime();<a name="line.611"></a>
<span class="sourceLineNo">612</span> if (now &gt; lastLog + 5000) {<a name="line.612"></a>
<span class="sourceLineNo">613</span> lastLog = now;<a name="line.613"></a>
<span class="sourceLineNo">614</span> LOG.info("total=" + tot + ", unassigned=" + unassigned + ", tasks=" + tasks);<a name="line.614"></a>
<span class="sourceLineNo">615</span> }<a name="line.615"></a>
<span class="sourceLineNo">616</span> }<a name="line.616"></a>
<span class="sourceLineNo">617</span> if (resubmitted &gt; 0) {<a name="line.617"></a>
<span class="sourceLineNo">618</span> LOG.info("resubmitted " + resubmitted + " out of " + tot + " tasks");<a name="line.618"></a>
<span class="sourceLineNo">619</span> }<a name="line.619"></a>
<span class="sourceLineNo">620</span> // If there are pending tasks and all of them have been unassigned for<a name="line.620"></a>
<span class="sourceLineNo">621</span> // some time then put up a RESCAN node to ping the workers.<a name="line.621"></a>
<span class="sourceLineNo">622</span> // ZKSplitlog.DEFAULT_UNASSIGNED_TIMEOUT is of the order of minutes<a name="line.622"></a>
<span class="sourceLineNo">623</span> // because a. it is very unlikely that every worker had a<a name="line.623"></a>
<span class="sourceLineNo">624</span> // transient error when trying to grab the task b. if there are no<a name="line.624"></a>
<span class="sourceLineNo">625</span> // workers then all tasks wills stay unassigned indefinitely and the<a name="line.625"></a>
<span class="sourceLineNo">626</span> // manager will be indefinitely creating RESCAN nodes. TODO may be the<a name="line.626"></a>
<span class="sourceLineNo">627</span> // master should spawn both a manager and a worker thread to guarantee<a name="line.627"></a>
<span class="sourceLineNo">628</span> // that there is always one worker in the system<a name="line.628"></a>
<span class="sourceLineNo">629</span> if (tot &gt; 0<a name="line.629"></a>
<span class="sourceLineNo">630</span> &amp;&amp; !found_assigned_task<a name="line.630"></a>
<span class="sourceLineNo">631</span> &amp;&amp; ((EnvironmentEdgeManager.currentTime() - lastTaskCreateTime) &gt; unassignedTimeout)) {<a name="line.631"></a>
<span class="sourceLineNo">632</span> for (Map.Entry&lt;String, Task&gt; e : tasks.entrySet()) {<a name="line.632"></a>
<span class="sourceLineNo">633</span> String key = e.getKey();<a name="line.633"></a>
<span class="sourceLineNo">634</span> Task task = e.getValue();<a name="line.634"></a>
<span class="sourceLineNo">635</span> // we have to do task.isUnassigned() check again because tasks might<a name="line.635"></a>
<span class="sourceLineNo">636</span> // have been asynchronously assigned. There is no locking required<a name="line.636"></a>
<span class="sourceLineNo">637</span> // for these checks ... it is OK even if tryGetDataSetWatch() is<a name="line.637"></a>
<span class="sourceLineNo">638</span> // called unnecessarily for a taskpath<a name="line.638"></a>
<span class="sourceLineNo">639</span> if (task.isUnassigned() &amp;&amp; (task.status != FAILURE)) {<a name="line.639"></a>
<span class="sourceLineNo">640</span> // We just touch the znode to make sure its still there<a name="line.640"></a>
<span class="sourceLineNo">641</span> getSplitLogManagerCoordination().checkTaskStillAvailable(key);<a name="line.641"></a>
<span class="sourceLineNo">642</span> }<a name="line.642"></a>
<span class="sourceLineNo">643</span> }<a name="line.643"></a>
<span class="sourceLineNo">644</span> getSplitLogManagerCoordination().checkTasks();<a name="line.644"></a>
<span class="sourceLineNo">645</span> SplitLogCounters.tot_mgr_resubmit_unassigned.increment();<a name="line.645"></a>
<span class="sourceLineNo">646</span> LOG.debug("resubmitting unassigned task(s) after timeout");<a name="line.646"></a>
<span class="sourceLineNo">647</span> }<a name="line.647"></a>
<span class="sourceLineNo">648</span> Set&lt;String&gt; failedDeletions =<a name="line.648"></a>
<span class="sourceLineNo">649</span> getSplitLogManagerCoordination().getDetails().getFailedDeletions();<a name="line.649"></a>
<span class="sourceLineNo">650</span> // Retry previously failed deletes<a name="line.650"></a>
<span class="sourceLineNo">651</span> if (failedDeletions.size() &gt; 0) {<a name="line.651"></a>
<span class="sourceLineNo">652</span> List&lt;String&gt; tmpPaths = new ArrayList&lt;&gt;(failedDeletions);<a name="line.652"></a>
<span class="sourceLineNo">653</span> for (String tmpPath : tmpPaths) {<a name="line.653"></a>
<span class="sourceLineNo">654</span> // deleteNode is an async call<a name="line.654"></a>
<span class="sourceLineNo">655</span> getSplitLogManagerCoordination().deleteTask(tmpPath);<a name="line.655"></a>
<span class="sourceLineNo">656</span> }<a name="line.656"></a>
<span class="sourceLineNo">657</span> failedDeletions.removeAll(tmpPaths);<a name="line.657"></a>
<span class="sourceLineNo">658</span> }<a name="line.658"></a>
<span class="sourceLineNo">659</span> }<a name="line.659"></a>
<span class="sourceLineNo">660</span> }<a name="line.660"></a>
<span class="sourceLineNo">661</span><a name="line.661"></a>
<span class="sourceLineNo">662</span> public enum ResubmitDirective {<a name="line.662"></a>
<span class="sourceLineNo">663</span> CHECK(), FORCE()<a name="line.663"></a>
<span class="sourceLineNo">664</span> }<a name="line.664"></a>
<span class="sourceLineNo">665</span><a name="line.665"></a>
<span class="sourceLineNo">666</span> public enum TerminationStatus {<a name="line.666"></a>
<span class="sourceLineNo">667</span> IN_PROGRESS("in_progress"), SUCCESS("success"), FAILURE("failure"), DELETED("deleted");<a name="line.667"></a>
<span class="sourceLineNo">668</span><a name="line.668"></a>
<span class="sourceLineNo">669</span> final String statusMsg;<a name="line.669"></a>
<span class="sourceLineNo">670</span><a name="line.670"></a>
<span class="sourceLineNo">671</span> TerminationStatus(String msg) {<a name="line.671"></a>
<span class="sourceLineNo">672</span> statusMsg = msg;<a name="line.672"></a>
<span class="sourceLineNo">673</span> }<a name="line.673"></a>
<span class="sourceLineNo">674</span><a name="line.674"></a>
<span class="sourceLineNo">675</span> @Override<a name="line.675"></a>
<span class="sourceLineNo">676</span> public String toString() {<a name="line.676"></a>
<span class="sourceLineNo">677</span> return statusMsg;<a name="line.677"></a>
<span class="sourceLineNo">678</span> }<a name="line.678"></a>
<span class="sourceLineNo">679</span> }<a name="line.679"></a>
<span class="sourceLineNo">680</span>}<a name="line.680"></a>
</pre>
</div>
</body>
</html>