blob: e5f9638fc6274acc2dbf16ca15116c805e74b389 [file] [log] [blame]
<!DOCTYPE HTML>
<html lang="en">
<head>
<!-- Generated by javadoc (17) -->
<title>Source code</title>
<meta name="viewport" content="width=device-width, initial-scale=1">
<meta name="description" content="source: package: org.apache.hadoop.hbase.regionserver.wal, class: FSHLog, class: SyncRunner">
<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.regionserver.wal;</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.regionserver.wal.WALActionsListener.RollRequestReason.SLOW_SYNC;</span>
<span class="source-line-no">021</span><span id="line-21"></span>
<span class="source-line-no">022</span><span id="line-22">import java.io.IOException;</span>
<span class="source-line-no">023</span><span id="line-23">import java.io.OutputStream;</span>
<span class="source-line-no">024</span><span id="line-24">import java.util.Arrays;</span>
<span class="source-line-no">025</span><span id="line-25">import java.util.List;</span>
<span class="source-line-no">026</span><span id="line-26">import java.util.concurrent.BlockingQueue;</span>
<span class="source-line-no">027</span><span id="line-27">import java.util.concurrent.CompletableFuture;</span>
<span class="source-line-no">028</span><span id="line-28">import java.util.concurrent.LinkedBlockingQueue;</span>
<span class="source-line-no">029</span><span id="line-29">import java.util.concurrent.atomic.AtomicInteger;</span>
<span class="source-line-no">030</span><span id="line-30">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">031</span><span id="line-31">import org.apache.hadoop.fs.FSDataOutputStream;</span>
<span class="source-line-no">032</span><span id="line-32">import org.apache.hadoop.fs.FileSystem;</span>
<span class="source-line-no">033</span><span id="line-33">import org.apache.hadoop.fs.Path;</span>
<span class="source-line-no">034</span><span id="line-34">import org.apache.hadoop.hbase.Abortable;</span>
<span class="source-line-no">035</span><span id="line-35">import org.apache.hadoop.hbase.HConstants;</span>
<span class="source-line-no">036</span><span id="line-36">import org.apache.hadoop.hbase.util.Bytes;</span>
<span class="source-line-no">037</span><span id="line-37">import org.apache.hadoop.hbase.util.ClassSize;</span>
<span class="source-line-no">038</span><span id="line-38">import org.apache.hadoop.hbase.util.CommonFSUtils;</span>
<span class="source-line-no">039</span><span id="line-39">import org.apache.hadoop.hbase.wal.FSHLogProvider;</span>
<span class="source-line-no">040</span><span id="line-40">import org.apache.hadoop.hbase.wal.WALProvider.Writer;</span>
<span class="source-line-no">041</span><span id="line-41">import org.apache.hadoop.hdfs.DFSOutputStream;</span>
<span class="source-line-no">042</span><span id="line-42">import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;</span>
<span class="source-line-no">043</span><span id="line-43">import org.apache.hadoop.hdfs.protocol.DatanodeInfo;</span>
<span class="source-line-no">044</span><span id="line-44">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">045</span><span id="line-45">import org.slf4j.Logger;</span>
<span class="source-line-no">046</span><span id="line-46">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">047</span><span id="line-47"></span>
<span class="source-line-no">048</span><span id="line-48">/**</span>
<span class="source-line-no">049</span><span id="line-49"> * The original implementation of FSWAL.</span>
<span class="source-line-no">050</span><span id="line-50"> */</span>
<span class="source-line-no">051</span><span id="line-51">@InterfaceAudience.Private</span>
<span class="source-line-no">052</span><span id="line-52">public class FSHLog extends AbstractFSWAL&lt;Writer&gt; {</span>
<span class="source-line-no">053</span><span id="line-53"> // IMPLEMENTATION NOTES:</span>
<span class="source-line-no">054</span><span id="line-54"> //</span>
<span class="source-line-no">055</span><span id="line-55"> // At the core is a ring buffer. Our ring buffer is the LMAX Disruptor. It tries to</span>
<span class="source-line-no">056</span><span id="line-56"> // minimize synchronizations and volatile writes when multiple contending threads as is the case</span>
<span class="source-line-no">057</span><span id="line-57"> // here appending and syncing on a single WAL. The Disruptor is configured to handle multiple</span>
<span class="source-line-no">058</span><span id="line-58"> // producers but it has one consumer only (the producers in HBase are IPC Handlers calling append</span>
<span class="source-line-no">059</span><span id="line-59"> // and then sync). The single consumer/writer pulls the appends and syncs off the ring buffer.</span>
<span class="source-line-no">060</span><span id="line-60"> // When a handler calls sync, it is given back a future. The producer 'blocks' on the future so</span>
<span class="source-line-no">061</span><span id="line-61"> // it does not return until the sync completes. The future is passed over the ring buffer from</span>
<span class="source-line-no">062</span><span id="line-62"> // the producer/handler to the consumer thread where it does its best to batch up the producer</span>
<span class="source-line-no">063</span><span id="line-63"> // syncs so one WAL sync actually spans multiple producer sync invocations. How well the</span>
<span class="source-line-no">064</span><span id="line-64"> // batching works depends on the write rate; i.e. we tend to batch more in times of</span>
<span class="source-line-no">065</span><span id="line-65"> // high writes/syncs.</span>
<span class="source-line-no">066</span><span id="line-66"> //</span>
<span class="source-line-no">067</span><span id="line-67"> // Calls to append now also wait until the append has been done on the consumer side of the</span>
<span class="source-line-no">068</span><span id="line-68"> // disruptor. We used to not wait but it makes the implementation easier to grok if we have</span>
<span class="source-line-no">069</span><span id="line-69"> // the region edit/sequence id after the append returns.</span>
<span class="source-line-no">070</span><span id="line-70"> //</span>
<span class="source-line-no">071</span><span id="line-71"> // TODO: Handlers need to coordinate appending AND syncing. Can we have the threads contend</span>
<span class="source-line-no">072</span><span id="line-72"> // once only? Probably hard given syncs take way longer than an append.</span>
<span class="source-line-no">073</span><span id="line-73"> //</span>
<span class="source-line-no">074</span><span id="line-74"> // The consumer threads pass the syncs off to multiple syncing threads in a round robin fashion</span>
<span class="source-line-no">075</span><span id="line-75"> // to ensure we keep up back-to-back FS sync calls (FS sync calls are the long poll writing the</span>
<span class="source-line-no">076</span><span id="line-76"> // WAL). The consumer thread passes the futures to the sync threads for it to complete</span>
<span class="source-line-no">077</span><span id="line-77"> // the futures when done.</span>
<span class="source-line-no">078</span><span id="line-78"> //</span>
<span class="source-line-no">079</span><span id="line-79"> // The 'sequence' in the below is the sequence of the append/sync on the ringbuffer. It</span>
<span class="source-line-no">080</span><span id="line-80"> // acts as a sort-of transaction id. It is always incrementing.</span>
<span class="source-line-no">081</span><span id="line-81"> //</span>
<span class="source-line-no">082</span><span id="line-82"> // The RingBufferEventHandler class hosts the ring buffer consuming code. The threads that</span>
<span class="source-line-no">083</span><span id="line-83"> // do the actual FS sync are implementations of SyncRunner. SafePointZigZagLatch is a</span>
<span class="source-line-no">084</span><span id="line-84"> // synchronization class used to halt the consumer at a safe point -- just after all outstanding</span>
<span class="source-line-no">085</span><span id="line-85"> // syncs and appends have completed -- so the log roller can swap the WAL out under it.</span>
<span class="source-line-no">086</span><span id="line-86"> //</span>
<span class="source-line-no">087</span><span id="line-87"> // We use ring buffer sequence as txid of FSWALEntry and SyncFuture.</span>
<span class="source-line-no">088</span><span id="line-88"> private static final Logger LOG = LoggerFactory.getLogger(FSHLog.class);</span>
<span class="source-line-no">089</span><span id="line-89"></span>
<span class="source-line-no">090</span><span id="line-90"> private static final String TOLERABLE_LOW_REPLICATION =</span>
<span class="source-line-no">091</span><span id="line-91"> "hbase.regionserver.hlog.tolerable.lowreplication";</span>
<span class="source-line-no">092</span><span id="line-92"> private static final String LOW_REPLICATION_ROLL_LIMIT =</span>
<span class="source-line-no">093</span><span id="line-93"> "hbase.regionserver.hlog.lowreplication.rolllimit";</span>
<span class="source-line-no">094</span><span id="line-94"> private static final int DEFAULT_LOW_REPLICATION_ROLL_LIMIT = 5;</span>
<span class="source-line-no">095</span><span id="line-95"> private static final String SYNCER_COUNT = "hbase.regionserver.hlog.syncer.count";</span>
<span class="source-line-no">096</span><span id="line-96"> private static final int DEFAULT_SYNCER_COUNT = 5;</span>
<span class="source-line-no">097</span><span id="line-97"> private static final String MAX_BATCH_COUNT = "hbase.regionserver.wal.sync.batch.count";</span>
<span class="source-line-no">098</span><span id="line-98"> private static final int DEFAULT_MAX_BATCH_COUNT = 200;</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 static final String FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS =</span>
<span class="source-line-no">101</span><span id="line-101"> "hbase.wal.fshlog.wait.on.shutdown.seconds";</span>
<span class="source-line-no">102</span><span id="line-102"> private static final int DEFAULT_FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS = 5;</span>
<span class="source-line-no">103</span><span id="line-103"></span>
<span class="source-line-no">104</span><span id="line-104"> private static final IOException WITER_REPLACED_EXCEPTION =</span>
<span class="source-line-no">105</span><span id="line-105"> new IOException("Writer was replaced!");</span>
<span class="source-line-no">106</span><span id="line-106"> private static final IOException WITER_BROKEN_EXCEPTION = new IOException("Wirter was broken!");</span>
<span class="source-line-no">107</span><span id="line-107"> private static final IOException WAL_CLOSE_EXCEPTION = new IOException("WAL was closed!");</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"> * FSDataOutputStream associated with the current SequenceFile.writer</span>
<span class="source-line-no">111</span><span id="line-111"> */</span>
<span class="source-line-no">112</span><span id="line-112"> private FSDataOutputStream hdfs_out;</span>
<span class="source-line-no">113</span><span id="line-113"></span>
<span class="source-line-no">114</span><span id="line-114"> // All about log rolling if not enough replicas outstanding.</span>
<span class="source-line-no">115</span><span id="line-115"></span>
<span class="source-line-no">116</span><span id="line-116"> // Minimum tolerable replicas, if the actual value is lower than it, rollWriter will be triggered</span>
<span class="source-line-no">117</span><span id="line-117"> private final int minTolerableReplication;</span>
<span class="source-line-no">118</span><span id="line-118"></span>
<span class="source-line-no">119</span><span id="line-119"> // If live datanode count is lower than the default replicas value,</span>
<span class="source-line-no">120</span><span id="line-120"> // RollWriter will be triggered in each sync(So the RollWriter will be</span>
<span class="source-line-no">121</span><span id="line-121"> // triggered one by one in a short time). Using it as a workaround to slow</span>
<span class="source-line-no">122</span><span id="line-122"> // down the roll frequency triggered by checkLowReplication().</span>
<span class="source-line-no">123</span><span id="line-123"> private final AtomicInteger consecutiveLogRolls = new AtomicInteger(0);</span>
<span class="source-line-no">124</span><span id="line-124"></span>
<span class="source-line-no">125</span><span id="line-125"> private final int lowReplicationRollLimit;</span>
<span class="source-line-no">126</span><span id="line-126"></span>
<span class="source-line-no">127</span><span id="line-127"> // If consecutiveLogRolls is larger than lowReplicationRollLimit,</span>
<span class="source-line-no">128</span><span id="line-128"> // then disable the rolling in checkLowReplication().</span>
<span class="source-line-no">129</span><span id="line-129"> // Enable it if the replications recover.</span>
<span class="source-line-no">130</span><span id="line-130"> private volatile boolean lowReplicationRollEnabled = true;</span>
<span class="source-line-no">131</span><span id="line-131"></span>
<span class="source-line-no">132</span><span id="line-132"> private final int syncerCount;</span>
<span class="source-line-no">133</span><span id="line-133"> private final int maxSyncRequestCount;</span>
<span class="source-line-no">134</span><span id="line-134"></span>
<span class="source-line-no">135</span><span id="line-135"> /**</span>
<span class="source-line-no">136</span><span id="line-136"> * Which syncrunner to use next.</span>
<span class="source-line-no">137</span><span id="line-137"> */</span>
<span class="source-line-no">138</span><span id="line-138"> private int syncRunnerIndex = 0;</span>
<span class="source-line-no">139</span><span id="line-139"></span>
<span class="source-line-no">140</span><span id="line-140"> private SyncRunner[] syncRunners = null;</span>
<span class="source-line-no">141</span><span id="line-141"></span>
<span class="source-line-no">142</span><span id="line-142"> /**</span>
<span class="source-line-no">143</span><span id="line-143"> * Constructor.</span>
<span class="source-line-no">144</span><span id="line-144"> * @param fs filesystem handle</span>
<span class="source-line-no">145</span><span id="line-145"> * @param root path for stored and archived wals</span>
<span class="source-line-no">146</span><span id="line-146"> * @param logDir dir where wals are stored</span>
<span class="source-line-no">147</span><span id="line-147"> * @param conf configuration to use</span>
<span class="source-line-no">148</span><span id="line-148"> */</span>
<span class="source-line-no">149</span><span id="line-149"> public FSHLog(final FileSystem fs, final Path root, final String logDir, final Configuration conf)</span>
<span class="source-line-no">150</span><span id="line-150"> throws IOException {</span>
<span class="source-line-no">151</span><span id="line-151"> this(fs, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null, null);</span>
<span class="source-line-no">152</span><span id="line-152"> }</span>
<span class="source-line-no">153</span><span id="line-153"></span>
<span class="source-line-no">154</span><span id="line-154"> public FSHLog(final FileSystem fs, Abortable abortable, final Path root, final String logDir,</span>
<span class="source-line-no">155</span><span id="line-155"> final Configuration conf) throws IOException {</span>
<span class="source-line-no">156</span><span id="line-156"> this(fs, abortable, root, logDir, HConstants.HREGION_OLDLOGDIR_NAME, conf, null, true, null,</span>
<span class="source-line-no">157</span><span id="line-157"> null, null, null);</span>
<span class="source-line-no">158</span><span id="line-158"> }</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 FSHLog(final FileSystem fs, final Path rootDir, final String logDir,</span>
<span class="source-line-no">161</span><span id="line-161"> final String archiveDir, final Configuration conf, final List&lt;WALActionsListener&gt; listeners,</span>
<span class="source-line-no">162</span><span id="line-162"> final boolean failIfWALExists, final String prefix, final String suffix) throws IOException {</span>
<span class="source-line-no">163</span><span id="line-163"> this(fs, null, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix, suffix,</span>
<span class="source-line-no">164</span><span id="line-164"> null, null);</span>
<span class="source-line-no">165</span><span id="line-165"> }</span>
<span class="source-line-no">166</span><span id="line-166"></span>
<span class="source-line-no">167</span><span id="line-167"> /**</span>
<span class="source-line-no">168</span><span id="line-168"> * Create an edit log at the given &lt;code&gt;dir&lt;/code&gt; location. You should never have to load an</span>
<span class="source-line-no">169</span><span id="line-169"> * existing log. If there is a log at startup, it should have already been processed and deleted</span>
<span class="source-line-no">170</span><span id="line-170"> * by the time the WAL object is started up.</span>
<span class="source-line-no">171</span><span id="line-171"> * @param fs filesystem handle</span>
<span class="source-line-no">172</span><span id="line-172"> * @param abortable Abortable - the server here</span>
<span class="source-line-no">173</span><span id="line-173"> * @param rootDir path to where logs and oldlogs</span>
<span class="source-line-no">174</span><span id="line-174"> * @param logDir dir where wals are stored</span>
<span class="source-line-no">175</span><span id="line-175"> * @param archiveDir dir where wals are archived</span>
<span class="source-line-no">176</span><span id="line-176"> * @param conf configuration to use</span>
<span class="source-line-no">177</span><span id="line-177"> * @param listeners Listeners on WAL events. Listeners passed here will be registered before</span>
<span class="source-line-no">178</span><span id="line-178"> * we do anything else; e.g. the Constructor {@link #rollWriter()}.</span>
<span class="source-line-no">179</span><span id="line-179"> * @param failIfWALExists If true IOException will be thrown if files related to this wal already</span>
<span class="source-line-no">180</span><span id="line-180"> * exist.</span>
<span class="source-line-no">181</span><span id="line-181"> * @param prefix should always be hostname and port in distributed env and it will be URL</span>
<span class="source-line-no">182</span><span id="line-182"> * encoded before being used. If prefix is null, "wal" will be used</span>
<span class="source-line-no">183</span><span id="line-183"> * @param suffix will be url encoded. null is treated as empty. non-empty must start with</span>
<span class="source-line-no">184</span><span id="line-184"> * {@link org.apache.hadoop.hbase.wal.AbstractFSWALProvider#WAL_FILE_NAME_DELIMITER}</span>
<span class="source-line-no">185</span><span id="line-185"> */</span>
<span class="source-line-no">186</span><span id="line-186"> public FSHLog(final FileSystem fs, final Abortable abortable, final Path rootDir,</span>
<span class="source-line-no">187</span><span id="line-187"> final String logDir, final String archiveDir, final Configuration conf,</span>
<span class="source-line-no">188</span><span id="line-188"> final List&lt;WALActionsListener&gt; listeners, final boolean failIfWALExists, final String prefix,</span>
<span class="source-line-no">189</span><span id="line-189"> final String suffix, FileSystem remoteFs, Path remoteWALDir) throws IOException {</span>
<span class="source-line-no">190</span><span id="line-190"> super(fs, abortable, rootDir, logDir, archiveDir, conf, listeners, failIfWALExists, prefix,</span>
<span class="source-line-no">191</span><span id="line-191"> suffix, remoteFs, remoteWALDir);</span>
<span class="source-line-no">192</span><span id="line-192"> this.minTolerableReplication =</span>
<span class="source-line-no">193</span><span id="line-193"> conf.getInt(TOLERABLE_LOW_REPLICATION, CommonFSUtils.getDefaultReplication(fs, this.walDir));</span>
<span class="source-line-no">194</span><span id="line-194"> this.lowReplicationRollLimit =</span>
<span class="source-line-no">195</span><span id="line-195"> conf.getInt(LOW_REPLICATION_ROLL_LIMIT, DEFAULT_LOW_REPLICATION_ROLL_LIMIT);</span>
<span class="source-line-no">196</span><span id="line-196"></span>
<span class="source-line-no">197</span><span id="line-197"> // Advance the ring buffer sequence so that it starts from 1 instead of 0,</span>
<span class="source-line-no">198</span><span id="line-198"> // because SyncFuture.NOT_DONE = 0.</span>
<span class="source-line-no">199</span><span id="line-199"></span>
<span class="source-line-no">200</span><span id="line-200"> this.syncerCount = conf.getInt(SYNCER_COUNT, DEFAULT_SYNCER_COUNT);</span>
<span class="source-line-no">201</span><span id="line-201"> this.maxSyncRequestCount = conf.getInt(MAX_BATCH_COUNT,</span>
<span class="source-line-no">202</span><span id="line-202"> conf.getInt(HConstants.REGION_SERVER_HANDLER_COUNT, DEFAULT_MAX_BATCH_COUNT));</span>
<span class="source-line-no">203</span><span id="line-203"></span>
<span class="source-line-no">204</span><span id="line-204"> this.createSingleThreadPoolConsumeExecutor("FSHLog", rootDir, prefix);</span>
<span class="source-line-no">205</span><span id="line-205"></span>
<span class="source-line-no">206</span><span id="line-206"> this.setWaitOnShutdownInSeconds(</span>
<span class="source-line-no">207</span><span id="line-207"> conf.getInt(FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS, DEFAULT_FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS),</span>
<span class="source-line-no">208</span><span id="line-208"> FSHLOG_WAIT_ON_SHUTDOWN_IN_SECONDS);</span>
<span class="source-line-no">209</span><span id="line-209"> }</span>
<span class="source-line-no">210</span><span id="line-210"></span>
<span class="source-line-no">211</span><span id="line-211"> @Override</span>
<span class="source-line-no">212</span><span id="line-212"> public void init() throws IOException {</span>
<span class="source-line-no">213</span><span id="line-213"> super.init();</span>
<span class="source-line-no">214</span><span id="line-214"> this.createSyncRunnersAndStart();</span>
<span class="source-line-no">215</span><span id="line-215"> }</span>
<span class="source-line-no">216</span><span id="line-216"></span>
<span class="source-line-no">217</span><span id="line-217"> private void createSyncRunnersAndStart() {</span>
<span class="source-line-no">218</span><span id="line-218"> this.syncRunnerIndex = 0;</span>
<span class="source-line-no">219</span><span id="line-219"> this.syncRunners = new SyncRunner[syncerCount];</span>
<span class="source-line-no">220</span><span id="line-220"> for (int i = 0; i &lt; syncerCount; i++) {</span>
<span class="source-line-no">221</span><span id="line-221"> this.syncRunners[i] = new SyncRunner("sync." + i, maxSyncRequestCount);</span>
<span class="source-line-no">222</span><span id="line-222"> this.syncRunners[i].start();</span>
<span class="source-line-no">223</span><span id="line-223"> }</span>
<span class="source-line-no">224</span><span id="line-224"> }</span>
<span class="source-line-no">225</span><span id="line-225"></span>
<span class="source-line-no">226</span><span id="line-226"> /**</span>
<span class="source-line-no">227</span><span id="line-227"> * Currently, we need to expose the writer's OutputStream to tests so that they can manipulate the</span>
<span class="source-line-no">228</span><span id="line-228"> * default behavior (such as setting the maxRecoveryErrorCount value). This is done using</span>
<span class="source-line-no">229</span><span id="line-229"> * reflection on the underlying HDFS OutputStream. NOTE: This could be removed once Hadoop1</span>
<span class="source-line-no">230</span><span id="line-230"> * support is removed.</span>
<span class="source-line-no">231</span><span id="line-231"> * @return null if underlying stream is not ready.</span>
<span class="source-line-no">232</span><span id="line-232"> */</span>
<span class="source-line-no">233</span><span id="line-233"> OutputStream getOutputStream() {</span>
<span class="source-line-no">234</span><span id="line-234"> FSDataOutputStream fsdos = this.hdfs_out;</span>
<span class="source-line-no">235</span><span id="line-235"> return fsdos != null ? fsdos.getWrappedStream() : null;</span>
<span class="source-line-no">236</span><span id="line-236"> }</span>
<span class="source-line-no">237</span><span id="line-237"></span>
<span class="source-line-no">238</span><span id="line-238"> /**</span>
<span class="source-line-no">239</span><span id="line-239"> * Run a sync after opening to set up the pipeline.</span>
<span class="source-line-no">240</span><span id="line-240"> */</span>
<span class="source-line-no">241</span><span id="line-241"> private void preemptiveSync(final ProtobufLogWriter nextWriter) {</span>
<span class="source-line-no">242</span><span id="line-242"> long startTimeNanos = System.nanoTime();</span>
<span class="source-line-no">243</span><span id="line-243"> try {</span>
<span class="source-line-no">244</span><span id="line-244"> nextWriter.sync(useHsync);</span>
<span class="source-line-no">245</span><span id="line-245"> postSync(System.nanoTime() - startTimeNanos, 0);</span>
<span class="source-line-no">246</span><span id="line-246"> } catch (IOException e) {</span>
<span class="source-line-no">247</span><span id="line-247"> // optimization failed, no need to abort here.</span>
<span class="source-line-no">248</span><span id="line-248"> LOG.warn("pre-sync failed but an optimization so keep going", e);</span>
<span class="source-line-no">249</span><span id="line-249"> }</span>
<span class="source-line-no">250</span><span id="line-250"> }</span>
<span class="source-line-no">251</span><span id="line-251"></span>
<span class="source-line-no">252</span><span id="line-252"> /**</span>
<span class="source-line-no">253</span><span id="line-253"> * This method allows subclasses to inject different writers without having to extend other</span>
<span class="source-line-no">254</span><span id="line-254"> * methods like rollWriter().</span>
<span class="source-line-no">255</span><span id="line-255"> * @return Writer instance</span>
<span class="source-line-no">256</span><span id="line-256"> */</span>
<span class="source-line-no">257</span><span id="line-257"> @Override</span>
<span class="source-line-no">258</span><span id="line-258"> protected Writer createWriterInstance(FileSystem fs, Path path) throws IOException {</span>
<span class="source-line-no">259</span><span id="line-259"> Writer writer = FSHLogProvider.createWriter(conf, fs, path, false, this.blocksize);</span>
<span class="source-line-no">260</span><span id="line-260"> if (writer instanceof ProtobufLogWriter) {</span>
<span class="source-line-no">261</span><span id="line-261"> preemptiveSync((ProtobufLogWriter) writer);</span>
<span class="source-line-no">262</span><span id="line-262"> }</span>
<span class="source-line-no">263</span><span id="line-263"> return writer;</span>
<span class="source-line-no">264</span><span id="line-264"> }</span>
<span class="source-line-no">265</span><span id="line-265"></span>
<span class="source-line-no">266</span><span id="line-266"> @Override</span>
<span class="source-line-no">267</span><span id="line-267"> protected void doAppend(Writer writer, FSWALEntry entry) throws IOException {</span>
<span class="source-line-no">268</span><span id="line-268"> writer.append(entry);</span>
<span class="source-line-no">269</span><span id="line-269"> }</span>
<span class="source-line-no">270</span><span id="line-270"></span>
<span class="source-line-no">271</span><span id="line-271"> @Override</span>
<span class="source-line-no">272</span><span id="line-272"> protected void onWriterReplaced(Writer nextWriter) {</span>
<span class="source-line-no">273</span><span id="line-273"> if (nextWriter != null &amp;&amp; nextWriter instanceof ProtobufLogWriter) {</span>
<span class="source-line-no">274</span><span id="line-274"> this.hdfs_out = ((ProtobufLogWriter) nextWriter).getStream();</span>
<span class="source-line-no">275</span><span id="line-275"> } else {</span>
<span class="source-line-no">276</span><span id="line-276"> this.hdfs_out = null;</span>
<span class="source-line-no">277</span><span id="line-277"> }</span>
<span class="source-line-no">278</span><span id="line-278"> this.createSyncRunnersAndStart();</span>
<span class="source-line-no">279</span><span id="line-279"> }</span>
<span class="source-line-no">280</span><span id="line-280"></span>
<span class="source-line-no">281</span><span id="line-281"> @Override</span>
<span class="source-line-no">282</span><span id="line-282"> protected void doCleanUpResources() {</span>
<span class="source-line-no">283</span><span id="line-283"> this.shutDownSyncRunners();</span>
<span class="source-line-no">284</span><span id="line-284"> };</span>
<span class="source-line-no">285</span><span id="line-285"></span>
<span class="source-line-no">286</span><span id="line-286"> private void shutDownSyncRunners() {</span>
<span class="source-line-no">287</span><span id="line-287"> SyncRunner[] syncRunnersToUse = this.syncRunners;</span>
<span class="source-line-no">288</span><span id="line-288"> if (syncRunnersToUse != null) {</span>
<span class="source-line-no">289</span><span id="line-289"> for (SyncRunner syncRunner : syncRunnersToUse) {</span>
<span class="source-line-no">290</span><span id="line-290"> syncRunner.shutDown();</span>
<span class="source-line-no">291</span><span id="line-291"> }</span>
<span class="source-line-no">292</span><span id="line-292"> }</span>
<span class="source-line-no">293</span><span id="line-293"> this.syncRunners = null;</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"> @Override</span>
<span class="source-line-no">297</span><span id="line-297"> protected CompletableFuture&lt;Long&gt; doWriterSync(Writer writer, boolean shouldUseHSync,</span>
<span class="source-line-no">298</span><span id="line-298"> long txidWhenSync) {</span>
<span class="source-line-no">299</span><span id="line-299"> CompletableFuture&lt;Long&gt; future = new CompletableFuture&lt;&gt;();</span>
<span class="source-line-no">300</span><span id="line-300"> SyncRequest syncRequest = new SyncRequest(writer, shouldUseHSync, txidWhenSync, future);</span>
<span class="source-line-no">301</span><span id="line-301"> this.offerSyncRequest(syncRequest);</span>
<span class="source-line-no">302</span><span id="line-302"> return future;</span>
<span class="source-line-no">303</span><span id="line-303"> }</span>
<span class="source-line-no">304</span><span id="line-304"></span>
<span class="source-line-no">305</span><span id="line-305"> private void offerSyncRequest(SyncRequest syncRequest) {</span>
<span class="source-line-no">306</span><span id="line-306"> for (int i = 0; i &lt; this.syncRunners.length; i++) {</span>
<span class="source-line-no">307</span><span id="line-307"> this.syncRunnerIndex = (this.syncRunnerIndex + 1) % this.syncRunners.length;</span>
<span class="source-line-no">308</span><span id="line-308"> if (this.syncRunners[this.syncRunnerIndex].offer(syncRequest)) {</span>
<span class="source-line-no">309</span><span id="line-309"> return;</span>
<span class="source-line-no">310</span><span id="line-310"> }</span>
<span class="source-line-no">311</span><span id="line-311"> }</span>
<span class="source-line-no">312</span><span id="line-312"> syncRequest.completableFuture</span>
<span class="source-line-no">313</span><span id="line-313"> .completeExceptionally(new IOException("There is no available syncRunner."));</span>
<span class="source-line-no">314</span><span id="line-314"> }</span>
<span class="source-line-no">315</span><span id="line-315"></span>
<span class="source-line-no">316</span><span id="line-316"> static class SyncRequest {</span>
<span class="source-line-no">317</span><span id="line-317"> private final Writer writer;</span>
<span class="source-line-no">318</span><span id="line-318"> private final boolean shouldUseHSync;</span>
<span class="source-line-no">319</span><span id="line-319"> private final long sequenceWhenSync;</span>
<span class="source-line-no">320</span><span id="line-320"> private final CompletableFuture&lt;Long&gt; completableFuture;</span>
<span class="source-line-no">321</span><span id="line-321"></span>
<span class="source-line-no">322</span><span id="line-322"> public SyncRequest(Writer writer, boolean shouldUseHSync, long txidWhenSync,</span>
<span class="source-line-no">323</span><span id="line-323"> CompletableFuture&lt;Long&gt; completableFuture) {</span>
<span class="source-line-no">324</span><span id="line-324"> this.writer = writer;</span>
<span class="source-line-no">325</span><span id="line-325"> this.shouldUseHSync = shouldUseHSync;</span>
<span class="source-line-no">326</span><span id="line-326"> this.sequenceWhenSync = txidWhenSync;</span>
<span class="source-line-no">327</span><span id="line-327"> this.completableFuture = completableFuture;</span>
<span class="source-line-no">328</span><span id="line-328"> }</span>
<span class="source-line-no">329</span><span id="line-329"></span>
<span class="source-line-no">330</span><span id="line-330"> }</span>
<span class="source-line-no">331</span><span id="line-331"></span>
<span class="source-line-no">332</span><span id="line-332"> /**</span>
<span class="source-line-no">333</span><span id="line-333"> * Thread to runs the hdfs sync call. This call takes a while to complete. This is the longest</span>
<span class="source-line-no">334</span><span id="line-334"> * pole adding edits to the WAL and this must complete to be sure all edits persisted. We run</span>
<span class="source-line-no">335</span><span id="line-335"> * multiple threads sync'ng rather than one that just syncs in series so we have better latencies;</span>
<span class="source-line-no">336</span><span id="line-336"> * otherwise, an edit that arrived just after a sync started, might have to wait almost the length</span>
<span class="source-line-no">337</span><span id="line-337"> * of two sync invocations before it is marked done.</span>
<span class="source-line-no">338</span><span id="line-338"> * &lt;p&gt;</span>
<span class="source-line-no">339</span><span id="line-339"> * When the sync completes, it marks all the passed in futures done. On the other end of the sync</span>
<span class="source-line-no">340</span><span id="line-340"> * future is a blocked thread, usually a regionserver Handler. There may be more than one future</span>
<span class="source-line-no">341</span><span id="line-341"> * passed in the case where a few threads arrive at about the same time and all invoke 'sync'. In</span>
<span class="source-line-no">342</span><span id="line-342"> * this case we'll batch up the invocations and run one filesystem sync only for a batch of</span>
<span class="source-line-no">343</span><span id="line-343"> * Handler sync invocations. Do not confuse these Handler SyncFutures with the futures an</span>
<span class="source-line-no">344</span><span id="line-344"> * ExecutorService returns when you call submit. We have no use for these in this model. These</span>
<span class="source-line-no">345</span><span id="line-345"> * SyncFutures are 'artificial', something to hold the Handler until the filesystem sync</span>
<span class="source-line-no">346</span><span id="line-346"> * completes.</span>
<span class="source-line-no">347</span><span id="line-347"> */</span>
<span class="source-line-no">348</span><span id="line-348"> private class SyncRunner extends Thread {</span>
<span class="source-line-no">349</span><span id="line-349"> // Keep around last exception thrown. Clear on successful sync.</span>
<span class="source-line-no">350</span><span id="line-350"> private final BlockingQueue&lt;SyncRequest&gt; syncRequests;</span>
<span class="source-line-no">351</span><span id="line-351"> private volatile boolean shutDown = false;</span>
<span class="source-line-no">352</span><span id="line-352"></span>
<span class="source-line-no">353</span><span id="line-353"> SyncRunner(final String name, final int maxHandlersCount) {</span>
<span class="source-line-no">354</span><span id="line-354"> super(name);</span>
<span class="source-line-no">355</span><span id="line-355"> // LinkedBlockingQueue because of</span>
<span class="source-line-no">356</span><span id="line-356"> // http://www.javacodegeeks.com/2010/09/java-best-practices-queue-battle-and.html</span>
<span class="source-line-no">357</span><span id="line-357"> // Could use other blockingqueues here or concurrent queues.</span>
<span class="source-line-no">358</span><span id="line-358"> //</span>
<span class="source-line-no">359</span><span id="line-359"> // We could let the capacity be 'open' but bound it so we get alerted in pathological case</span>
<span class="source-line-no">360</span><span id="line-360"> // where we cannot sync and we have a bunch of threads all backed up waiting on their syncs</span>
<span class="source-line-no">361</span><span id="line-361"> // to come in. LinkedBlockingQueue actually shrinks when you remove elements so Q should</span>
<span class="source-line-no">362</span><span id="line-362"> // stay neat and tidy in usual case. Let the max size be three times the maximum handlers.</span>
<span class="source-line-no">363</span><span id="line-363"> // The passed in maxHandlerCount is the user-level handlers which is what we put up most of</span>
<span class="source-line-no">364</span><span id="line-364"> // but HBase has other handlers running too -- opening region handlers which want to write</span>
<span class="source-line-no">365</span><span id="line-365"> // the meta table when succesful (i.e. sync), closing handlers -- etc. These are usually</span>
<span class="source-line-no">366</span><span id="line-366"> // much fewer in number than the user-space handlers so Q-size should be user handlers plus</span>
<span class="source-line-no">367</span><span id="line-367"> // some space for these other handlers. Lets multiply by 3 for good-measure.</span>
<span class="source-line-no">368</span><span id="line-368"> this.syncRequests = new LinkedBlockingQueue&lt;&gt;(maxHandlersCount * 3);</span>
<span class="source-line-no">369</span><span id="line-369"> }</span>
<span class="source-line-no">370</span><span id="line-370"></span>
<span class="source-line-no">371</span><span id="line-371"> boolean offer(SyncRequest syncRequest) {</span>
<span class="source-line-no">372</span><span id="line-372"> if (this.shutDown) {</span>
<span class="source-line-no">373</span><span id="line-373"> return false;</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"> if (!this.syncRequests.offer(syncRequest)) {</span>
<span class="source-line-no">377</span><span id="line-377"> return false;</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"> // recheck</span>
<span class="source-line-no">381</span><span id="line-381"> if (this.shutDown) {</span>
<span class="source-line-no">382</span><span id="line-382"> if (this.syncRequests.remove(syncRequest)) {</span>
<span class="source-line-no">383</span><span id="line-383"> return false;</span>
<span class="source-line-no">384</span><span id="line-384"> }</span>
<span class="source-line-no">385</span><span id="line-385"> }</span>
<span class="source-line-no">386</span><span id="line-386"> return true;</span>
<span class="source-line-no">387</span><span id="line-387"> }</span>
<span class="source-line-no">388</span><span id="line-388"></span>
<span class="source-line-no">389</span><span id="line-389"> private void completeSyncRequests(SyncRequest syncRequest, long syncedSequenceId) {</span>
<span class="source-line-no">390</span><span id="line-390"> if (syncRequest != null) {</span>
<span class="source-line-no">391</span><span id="line-391"> syncRequest.completableFuture.complete(syncedSequenceId);</span>
<span class="source-line-no">392</span><span id="line-392"> }</span>
<span class="source-line-no">393</span><span id="line-393"> while (true) {</span>
<span class="source-line-no">394</span><span id="line-394"> SyncRequest head = this.syncRequests.peek();</span>
<span class="source-line-no">395</span><span id="line-395"> if (head == null) {</span>
<span class="source-line-no">396</span><span id="line-396"> break;</span>
<span class="source-line-no">397</span><span id="line-397"> }</span>
<span class="source-line-no">398</span><span id="line-398"> if (head.sequenceWhenSync &gt; syncedSequenceId) {</span>
<span class="source-line-no">399</span><span id="line-399"> break;</span>
<span class="source-line-no">400</span><span id="line-400"> }</span>
<span class="source-line-no">401</span><span id="line-401"> head.completableFuture.complete(syncedSequenceId);</span>
<span class="source-line-no">402</span><span id="line-402"> this.syncRequests.poll();</span>
<span class="source-line-no">403</span><span id="line-403"> }</span>
<span class="source-line-no">404</span><span id="line-404"> }</span>
<span class="source-line-no">405</span><span id="line-405"></span>
<span class="source-line-no">406</span><span id="line-406"> private void completeExceptionallySyncRequests(SyncRequest syncRequest, Exception exception) {</span>
<span class="source-line-no">407</span><span id="line-407"> if (syncRequest != null) {</span>
<span class="source-line-no">408</span><span id="line-408"> syncRequest.completableFuture.completeExceptionally(exception);</span>
<span class="source-line-no">409</span><span id="line-409"> }</span>
<span class="source-line-no">410</span><span id="line-410"> while (true) {</span>
<span class="source-line-no">411</span><span id="line-411"> SyncRequest head = this.syncRequests.peek();</span>
<span class="source-line-no">412</span><span id="line-412"> if (head == null) {</span>
<span class="source-line-no">413</span><span id="line-413"> break;</span>
<span class="source-line-no">414</span><span id="line-414"> }</span>
<span class="source-line-no">415</span><span id="line-415"> if (head.writer != syncRequest.writer) {</span>
<span class="source-line-no">416</span><span id="line-416"> break;</span>
<span class="source-line-no">417</span><span id="line-417"> }</span>
<span class="source-line-no">418</span><span id="line-418"> head.completableFuture.completeExceptionally(exception);</span>
<span class="source-line-no">419</span><span id="line-419"> this.syncRequests.poll();</span>
<span class="source-line-no">420</span><span id="line-420"> }</span>
<span class="source-line-no">421</span><span id="line-421"> }</span>
<span class="source-line-no">422</span><span id="line-422"></span>
<span class="source-line-no">423</span><span id="line-423"> private SyncRequest takeSyncRequest() throws InterruptedException {</span>
<span class="source-line-no">424</span><span id="line-424"> while (true) {</span>
<span class="source-line-no">425</span><span id="line-425"> // We have to process what we 'take' from the queue</span>
<span class="source-line-no">426</span><span id="line-426"> SyncRequest syncRequest = this.syncRequests.take();</span>
<span class="source-line-no">427</span><span id="line-427"> // See if we can process any syncfutures BEFORE we go sync.</span>
<span class="source-line-no">428</span><span id="line-428"> long currentHighestSyncedSequence = highestSyncedTxid.get();</span>
<span class="source-line-no">429</span><span id="line-429"> if (syncRequest.sequenceWhenSync &lt; currentHighestSyncedSequence) {</span>
<span class="source-line-no">430</span><span id="line-430"> syncRequest.completableFuture.complete(currentHighestSyncedSequence);</span>
<span class="source-line-no">431</span><span id="line-431"> continue;</span>
<span class="source-line-no">432</span><span id="line-432"> }</span>
<span class="source-line-no">433</span><span id="line-433"> return syncRequest;</span>
<span class="source-line-no">434</span><span id="line-434"> }</span>
<span class="source-line-no">435</span><span id="line-435"> }</span>
<span class="source-line-no">436</span><span id="line-436"></span>
<span class="source-line-no">437</span><span id="line-437"> @Override</span>
<span class="source-line-no">438</span><span id="line-438"> public void run() {</span>
<span class="source-line-no">439</span><span id="line-439"> while (!this.shutDown) {</span>
<span class="source-line-no">440</span><span id="line-440"> try {</span>
<span class="source-line-no">441</span><span id="line-441"> SyncRequest syncRequest = this.takeSyncRequest();</span>
<span class="source-line-no">442</span><span id="line-442"> // I got something. Lets run. Save off current sequence number in case it changes</span>
<span class="source-line-no">443</span><span id="line-443"> // while we run.</span>
<span class="source-line-no">444</span><span id="line-444"> long currentSequenceToUse = syncRequest.sequenceWhenSync;</span>
<span class="source-line-no">445</span><span id="line-445"> boolean writerBroken = isWriterBroken();</span>
<span class="source-line-no">446</span><span id="line-446"> long currentHighestProcessedAppendTxid = highestProcessedAppendTxid;</span>
<span class="source-line-no">447</span><span id="line-447"> Writer currentWriter = writer;</span>
<span class="source-line-no">448</span><span id="line-448"> if (currentWriter != syncRequest.writer) {</span>
<span class="source-line-no">449</span><span id="line-449"> syncRequest.completableFuture.completeExceptionally(WITER_REPLACED_EXCEPTION);</span>
<span class="source-line-no">450</span><span id="line-450"> continue;</span>
<span class="source-line-no">451</span><span id="line-451"> }</span>
<span class="source-line-no">452</span><span id="line-452"> if (writerBroken) {</span>
<span class="source-line-no">453</span><span id="line-453"> syncRequest.completableFuture.completeExceptionally(WITER_BROKEN_EXCEPTION);</span>
<span class="source-line-no">454</span><span id="line-454"> continue;</span>
<span class="source-line-no">455</span><span id="line-455"> }</span>
<span class="source-line-no">456</span><span id="line-456"> if (currentHighestProcessedAppendTxid &gt; currentSequenceToUse) {</span>
<span class="source-line-no">457</span><span id="line-457"> currentSequenceToUse = currentHighestProcessedAppendTxid;</span>
<span class="source-line-no">458</span><span id="line-458"> }</span>
<span class="source-line-no">459</span><span id="line-459"> Exception lastException = null;</span>
<span class="source-line-no">460</span><span id="line-460"> try {</span>
<span class="source-line-no">461</span><span id="line-461"> writer.sync(syncRequest.shouldUseHSync);</span>
<span class="source-line-no">462</span><span id="line-462"> } catch (IOException e) {</span>
<span class="source-line-no">463</span><span id="line-463"> LOG.error("Error syncing", e);</span>
<span class="source-line-no">464</span><span id="line-464"> lastException = e;</span>
<span class="source-line-no">465</span><span id="line-465"> } catch (Exception e) {</span>
<span class="source-line-no">466</span><span id="line-466"> LOG.warn("UNEXPECTED", e);</span>
<span class="source-line-no">467</span><span id="line-467"> lastException = e;</span>
<span class="source-line-no">468</span><span id="line-468"> } finally {</span>
<span class="source-line-no">469</span><span id="line-469"> if (lastException != null) {</span>
<span class="source-line-no">470</span><span id="line-470"> this.completeExceptionallySyncRequests(syncRequest, lastException);</span>
<span class="source-line-no">471</span><span id="line-471"> } else {</span>
<span class="source-line-no">472</span><span id="line-472"> this.completeSyncRequests(syncRequest, currentSequenceToUse);</span>
<span class="source-line-no">473</span><span id="line-473"> }</span>
<span class="source-line-no">474</span><span id="line-474"> }</span>
<span class="source-line-no">475</span><span id="line-475"> } catch (InterruptedException e) {</span>
<span class="source-line-no">476</span><span id="line-476"> // Presume legit interrupt.</span>
<span class="source-line-no">477</span><span id="line-477"> LOG.info("interrupted");</span>
<span class="source-line-no">478</span><span id="line-478"> } catch (Throwable t) {</span>
<span class="source-line-no">479</span><span id="line-479"> LOG.warn("UNEXPECTED, continuing", t);</span>
<span class="source-line-no">480</span><span id="line-480"> }</span>
<span class="source-line-no">481</span><span id="line-481"> }</span>
<span class="source-line-no">482</span><span id="line-482"> this.clearSyncRequestsWhenShutDown();</span>
<span class="source-line-no">483</span><span id="line-483"> }</span>
<span class="source-line-no">484</span><span id="line-484"></span>
<span class="source-line-no">485</span><span id="line-485"> private void clearSyncRequestsWhenShutDown() {</span>
<span class="source-line-no">486</span><span id="line-486"> while (true) {</span>
<span class="source-line-no">487</span><span id="line-487"> SyncRequest syncRequest = this.syncRequests.poll();</span>
<span class="source-line-no">488</span><span id="line-488"> if (syncRequest == null) {</span>
<span class="source-line-no">489</span><span id="line-489"> break;</span>
<span class="source-line-no">490</span><span id="line-490"> }</span>
<span class="source-line-no">491</span><span id="line-491"> syncRequest.completableFuture.completeExceptionally(WAL_CLOSE_EXCEPTION);</span>
<span class="source-line-no">492</span><span id="line-492"> }</span>
<span class="source-line-no">493</span><span id="line-493"> }</span>
<span class="source-line-no">494</span><span id="line-494"></span>
<span class="source-line-no">495</span><span id="line-495"> void shutDown() {</span>
<span class="source-line-no">496</span><span id="line-496"> try {</span>
<span class="source-line-no">497</span><span id="line-497"> this.shutDown = true;</span>
<span class="source-line-no">498</span><span id="line-498"> this.interrupt();</span>
<span class="source-line-no">499</span><span id="line-499"> this.join();</span>
<span class="source-line-no">500</span><span id="line-500"> } catch (InterruptedException e) {</span>
<span class="source-line-no">501</span><span id="line-501"> LOG.warn("interrupted", e);</span>
<span class="source-line-no">502</span><span id="line-502"> Thread.currentThread().interrupt();</span>
<span class="source-line-no">503</span><span id="line-503"> }</span>
<span class="source-line-no">504</span><span id="line-504"> }</span>
<span class="source-line-no">505</span><span id="line-505"> }</span>
<span class="source-line-no">506</span><span id="line-506"></span>
<span class="source-line-no">507</span><span id="line-507"> @Override</span>
<span class="source-line-no">508</span><span id="line-508"> protected void checkSlowSyncCount() {</span>
<span class="source-line-no">509</span><span id="line-509"> if (isLogRollRequested()) {</span>
<span class="source-line-no">510</span><span id="line-510"> return;</span>
<span class="source-line-no">511</span><span id="line-511"> }</span>
<span class="source-line-no">512</span><span id="line-512"> if (doCheckSlowSync()) {</span>
<span class="source-line-no">513</span><span id="line-513"> // We log this already in checkSlowSync</span>
<span class="source-line-no">514</span><span id="line-514"> requestLogRoll(SLOW_SYNC);</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"></span>
<span class="source-line-no">518</span><span id="line-518"> /** Returns true if number of replicas for the WAL is lower than threshold */</span>
<span class="source-line-no">519</span><span id="line-519"> @Override</span>
<span class="source-line-no">520</span><span id="line-520"> protected boolean doCheckLogLowReplication() {</span>
<span class="source-line-no">521</span><span id="line-521"> boolean logRollNeeded = false;</span>
<span class="source-line-no">522</span><span id="line-522"> // if the number of replicas in HDFS has fallen below the configured</span>
<span class="source-line-no">523</span><span id="line-523"> // value, then roll logs.</span>
<span class="source-line-no">524</span><span id="line-524"> try {</span>
<span class="source-line-no">525</span><span id="line-525"> int numCurrentReplicas = getLogReplication();</span>
<span class="source-line-no">526</span><span id="line-526"> if (numCurrentReplicas != 0 &amp;&amp; numCurrentReplicas &lt; this.minTolerableReplication) {</span>
<span class="source-line-no">527</span><span id="line-527"> if (this.lowReplicationRollEnabled) {</span>
<span class="source-line-no">528</span><span id="line-528"> if (this.consecutiveLogRolls.get() &lt; this.lowReplicationRollLimit) {</span>
<span class="source-line-no">529</span><span id="line-529"> LOG.warn("HDFS pipeline error detected. " + "Found " + numCurrentReplicas</span>
<span class="source-line-no">530</span><span id="line-530"> + " replicas but expecting no less than " + this.minTolerableReplication</span>
<span class="source-line-no">531</span><span id="line-531"> + " replicas. " + " Requesting close of WAL. current pipeline: "</span>
<span class="source-line-no">532</span><span id="line-532"> + Arrays.toString(getPipeline()));</span>
<span class="source-line-no">533</span><span id="line-533"> logRollNeeded = true;</span>
<span class="source-line-no">534</span><span id="line-534"> // If rollWriter is requested, increase consecutiveLogRolls. Once it</span>
<span class="source-line-no">535</span><span id="line-535"> // is larger than lowReplicationRollLimit, disable the</span>
<span class="source-line-no">536</span><span id="line-536"> // LowReplication-Roller</span>
<span class="source-line-no">537</span><span id="line-537"> this.consecutiveLogRolls.getAndIncrement();</span>
<span class="source-line-no">538</span><span id="line-538"> } else {</span>
<span class="source-line-no">539</span><span id="line-539"> LOG.warn("Too many consecutive RollWriter requests, it's a sign of "</span>
<span class="source-line-no">540</span><span id="line-540"> + "the total number of live datanodes is lower than the tolerable replicas.");</span>
<span class="source-line-no">541</span><span id="line-541"> this.consecutiveLogRolls.set(0);</span>
<span class="source-line-no">542</span><span id="line-542"> this.lowReplicationRollEnabled = false;</span>
<span class="source-line-no">543</span><span id="line-543"> }</span>
<span class="source-line-no">544</span><span id="line-544"> }</span>
<span class="source-line-no">545</span><span id="line-545"> } else if (numCurrentReplicas &gt;= this.minTolerableReplication) {</span>
<span class="source-line-no">546</span><span id="line-546"> if (!this.lowReplicationRollEnabled) {</span>
<span class="source-line-no">547</span><span id="line-547"> // The new writer's log replicas is always the default value.</span>
<span class="source-line-no">548</span><span id="line-548"> // So we should not enable LowReplication-Roller. If numEntries</span>
<span class="source-line-no">549</span><span id="line-549"> // is lower than or equals 1, we consider it as a new writer.</span>
<span class="source-line-no">550</span><span id="line-550"> if (this.numEntries.get() &lt;= 1) {</span>
<span class="source-line-no">551</span><span id="line-551"> return logRollNeeded;</span>
<span class="source-line-no">552</span><span id="line-552"> }</span>
<span class="source-line-no">553</span><span id="line-553"> // Once the live datanode number and the replicas return to normal,</span>
<span class="source-line-no">554</span><span id="line-554"> // enable the LowReplication-Roller.</span>
<span class="source-line-no">555</span><span id="line-555"> this.lowReplicationRollEnabled = true;</span>
<span class="source-line-no">556</span><span id="line-556"> LOG.info("LowReplication-Roller was enabled.");</span>
<span class="source-line-no">557</span><span id="line-557"> }</span>
<span class="source-line-no">558</span><span id="line-558"> }</span>
<span class="source-line-no">559</span><span id="line-559"> } catch (Exception e) {</span>
<span class="source-line-no">560</span><span id="line-560"> LOG.warn("DFSOutputStream.getNumCurrentReplicas failed because of " + e + ", continuing...");</span>
<span class="source-line-no">561</span><span id="line-561"> }</span>
<span class="source-line-no">562</span><span id="line-562"> return logRollNeeded;</span>
<span class="source-line-no">563</span><span id="line-563"> }</span>
<span class="source-line-no">564</span><span id="line-564"></span>
<span class="source-line-no">565</span><span id="line-565"> /**</span>
<span class="source-line-no">566</span><span id="line-566"> * {@inheritDoc}</span>
<span class="source-line-no">567</span><span id="line-567"> * &lt;p&gt;</span>
<span class="source-line-no">568</span><span id="line-568"> * If the pipeline isn't started yet or is empty, you will get the default replication factor.</span>
<span class="source-line-no">569</span><span id="line-569"> * Therefore, if this function returns 0, it means you are not properly running with the HDFS-826</span>
<span class="source-line-no">570</span><span id="line-570"> * patch.</span>
<span class="source-line-no">571</span><span id="line-571"> */</span>
<span class="source-line-no">572</span><span id="line-572"> @Override</span>
<span class="source-line-no">573</span><span id="line-573"> int getLogReplication() {</span>
<span class="source-line-no">574</span><span id="line-574"> try {</span>
<span class="source-line-no">575</span><span id="line-575"> // in standalone mode, it will return 0</span>
<span class="source-line-no">576</span><span id="line-576"> if (this.hdfs_out instanceof HdfsDataOutputStream) {</span>
<span class="source-line-no">577</span><span id="line-577"> return ((HdfsDataOutputStream) this.hdfs_out).getCurrentBlockReplication();</span>
<span class="source-line-no">578</span><span id="line-578"> }</span>
<span class="source-line-no">579</span><span id="line-579"> } catch (IOException e) {</span>
<span class="source-line-no">580</span><span id="line-580"> LOG.info("", e);</span>
<span class="source-line-no">581</span><span id="line-581"> }</span>
<span class="source-line-no">582</span><span id="line-582"> return 0;</span>
<span class="source-line-no">583</span><span id="line-583"> }</span>
<span class="source-line-no">584</span><span id="line-584"></span>
<span class="source-line-no">585</span><span id="line-585"> boolean isLowReplicationRollEnabled() {</span>
<span class="source-line-no">586</span><span id="line-586"> return lowReplicationRollEnabled;</span>
<span class="source-line-no">587</span><span id="line-587"> }</span>
<span class="source-line-no">588</span><span id="line-588"></span>
<span class="source-line-no">589</span><span id="line-589"> public static final long FIXED_OVERHEAD =</span>
<span class="source-line-no">590</span><span id="line-590"> ClassSize.align(ClassSize.OBJECT + (5 * ClassSize.REFERENCE) + (2 * ClassSize.ATOMIC_INTEGER)</span>
<span class="source-line-no">591</span><span id="line-591"> + (3 * Bytes.SIZEOF_INT) + (4 * Bytes.SIZEOF_LONG));</span>
<span class="source-line-no">592</span><span id="line-592"></span>
<span class="source-line-no">593</span><span id="line-593"> /**</span>
<span class="source-line-no">594</span><span id="line-594"> * This method gets the pipeline for the current WAL.</span>
<span class="source-line-no">595</span><span id="line-595"> */</span>
<span class="source-line-no">596</span><span id="line-596"> @Override</span>
<span class="source-line-no">597</span><span id="line-597"> DatanodeInfo[] getPipeline() {</span>
<span class="source-line-no">598</span><span id="line-598"> if (this.hdfs_out != null) {</span>
<span class="source-line-no">599</span><span id="line-599"> if (this.hdfs_out.getWrappedStream() instanceof DFSOutputStream) {</span>
<span class="source-line-no">600</span><span id="line-600"> return ((DFSOutputStream) this.hdfs_out.getWrappedStream()).getPipeline();</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"> return new DatanodeInfo[0];</span>
<span class="source-line-no">604</span><span id="line-604"> }</span>
<span class="source-line-no">605</span><span id="line-605"></span>
<span class="source-line-no">606</span><span id="line-606"> @Override</span>
<span class="source-line-no">607</span><span id="line-607"> protected Writer createCombinedWriter(Writer localWriter, Writer remoteWriter) {</span>
<span class="source-line-no">608</span><span id="line-608"> // put remote writer first as usually it will cost more time to finish, so we write to it first</span>
<span class="source-line-no">609</span><span id="line-609"> return CombinedWriter.create(remoteWriter, localWriter);</span>
<span class="source-line-no">610</span><span id="line-610"> }</span>
<span class="source-line-no">611</span><span id="line-611">}</span>
</pre>
</div>
</main>
</body>
</html>