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