| <!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, class: MemStoreFlusher, class: FlushRegionEntry"> |
| <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;</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 io.opentelemetry.api.trace.Span;</span> |
| <span class="source-line-no">021</span><span id="line-21">import io.opentelemetry.context.Scope;</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.lang.Thread.UncaughtExceptionHandler;</span> |
| <span class="source-line-no">024</span><span id="line-24">import java.util.ArrayList;</span> |
| <span class="source-line-no">025</span><span id="line-25">import java.util.Arrays;</span> |
| <span class="source-line-no">026</span><span id="line-26">import java.util.Collection;</span> |
| <span class="source-line-no">027</span><span id="line-27">import java.util.ConcurrentModificationException;</span> |
| <span class="source-line-no">028</span><span id="line-28">import java.util.HashMap;</span> |
| <span class="source-line-no">029</span><span id="line-29">import java.util.HashSet;</span> |
| <span class="source-line-no">030</span><span id="line-30">import java.util.List;</span> |
| <span class="source-line-no">031</span><span id="line-31">import java.util.Map;</span> |
| <span class="source-line-no">032</span><span id="line-32">import java.util.Set;</span> |
| <span class="source-line-no">033</span><span id="line-33">import java.util.SortedMap;</span> |
| <span class="source-line-no">034</span><span id="line-34">import java.util.concurrent.BlockingQueue;</span> |
| <span class="source-line-no">035</span><span id="line-35">import java.util.concurrent.DelayQueue;</span> |
| <span class="source-line-no">036</span><span id="line-36">import java.util.concurrent.Delayed;</span> |
| <span class="source-line-no">037</span><span id="line-37">import java.util.concurrent.ThreadFactory;</span> |
| <span class="source-line-no">038</span><span id="line-38">import java.util.concurrent.TimeUnit;</span> |
| <span class="source-line-no">039</span><span id="line-39">import java.util.concurrent.atomic.AtomicBoolean;</span> |
| <span class="source-line-no">040</span><span id="line-40">import java.util.concurrent.atomic.AtomicInteger;</span> |
| <span class="source-line-no">041</span><span id="line-41">import java.util.concurrent.atomic.LongAdder;</span> |
| <span class="source-line-no">042</span><span id="line-42">import java.util.concurrent.locks.ReentrantReadWriteLock;</span> |
| <span class="source-line-no">043</span><span id="line-43">import org.apache.hadoop.conf.Configuration;</span> |
| <span class="source-line-no">044</span><span id="line-44">import org.apache.hadoop.hbase.DroppedSnapshotException;</span> |
| <span class="source-line-no">045</span><span id="line-45">import org.apache.hadoop.hbase.HConstants;</span> |
| <span class="source-line-no">046</span><span id="line-46">import org.apache.hadoop.hbase.client.RegionReplicaUtil;</span> |
| <span class="source-line-no">047</span><span id="line-47">import org.apache.hadoop.hbase.conf.ConfigurationObserver;</span> |
| <span class="source-line-no">048</span><span id="line-48">import org.apache.hadoop.hbase.regionserver.HRegion.FlushResult;</span> |
| <span class="source-line-no">049</span><span id="line-49">import org.apache.hadoop.hbase.trace.TraceUtil;</span> |
| <span class="source-line-no">050</span><span id="line-50">import org.apache.hadoop.hbase.util.Bytes;</span> |
| <span class="source-line-no">051</span><span id="line-51">import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;</span> |
| <span class="source-line-no">052</span><span id="line-52">import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;</span> |
| <span class="source-line-no">053</span><span id="line-53">import org.apache.hadoop.hbase.util.Threads;</span> |
| <span class="source-line-no">054</span><span id="line-54">import org.apache.hadoop.ipc.RemoteException;</span> |
| <span class="source-line-no">055</span><span id="line-55">import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;</span> |
| <span class="source-line-no">056</span><span id="line-56">import org.apache.yetus.audience.InterfaceAudience;</span> |
| <span class="source-line-no">057</span><span id="line-57">import org.slf4j.Logger;</span> |
| <span class="source-line-no">058</span><span id="line-58">import org.slf4j.LoggerFactory;</span> |
| <span class="source-line-no">059</span><span id="line-59"></span> |
| <span class="source-line-no">060</span><span id="line-60">import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;</span> |
| <span class="source-line-no">061</span><span id="line-61"></span> |
| <span class="source-line-no">062</span><span id="line-62">/**</span> |
| <span class="source-line-no">063</span><span id="line-63"> * Thread that flushes cache on request NOTE: This class extends Thread rather than Chore because</span> |
| <span class="source-line-no">064</span><span id="line-64"> * the sleep time can be interrupted when there is something to do, rather than the Chore sleep time</span> |
| <span class="source-line-no">065</span><span id="line-65"> * which is invariant.</span> |
| <span class="source-line-no">066</span><span id="line-66"> * @see FlushRequester</span> |
| <span class="source-line-no">067</span><span id="line-67"> */</span> |
| <span class="source-line-no">068</span><span id="line-68">@InterfaceAudience.Private</span> |
| <span class="source-line-no">069</span><span id="line-69">public class MemStoreFlusher implements FlushRequester, ConfigurationObserver {</span> |
| <span class="source-line-no">070</span><span id="line-70"> private static final Logger LOG = LoggerFactory.getLogger(MemStoreFlusher.class);</span> |
| <span class="source-line-no">071</span><span id="line-71"></span> |
| <span class="source-line-no">072</span><span id="line-72"> private Configuration conf;</span> |
| <span class="source-line-no">073</span><span id="line-73"> // These two data members go together. Any entry in the one must have</span> |
| <span class="source-line-no">074</span><span id="line-74"> // a corresponding entry in the other.</span> |
| <span class="source-line-no">075</span><span id="line-75"> private final BlockingQueue<FlushQueueEntry> flushQueue = new DelayQueue<>();</span> |
| <span class="source-line-no">076</span><span id="line-76"> protected final Map<Region, FlushRegionEntry> regionsInQueue = new HashMap<>();</span> |
| <span class="source-line-no">077</span><span id="line-77"> private AtomicBoolean wakeupPending = new AtomicBoolean();</span> |
| <span class="source-line-no">078</span><span id="line-78"></span> |
| <span class="source-line-no">079</span><span id="line-79"> private final long threadWakeFrequency;</span> |
| <span class="source-line-no">080</span><span id="line-80"> private final HRegionServer server;</span> |
| <span class="source-line-no">081</span><span id="line-81"> private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();</span> |
| <span class="source-line-no">082</span><span id="line-82"> private final Object blockSignal = new Object();</span> |
| <span class="source-line-no">083</span><span id="line-83"></span> |
| <span class="source-line-no">084</span><span id="line-84"> private long blockingWaitTime;</span> |
| <span class="source-line-no">085</span><span id="line-85"> private final LongAdder updatesBlockedMsHighWater = new LongAdder();</span> |
| <span class="source-line-no">086</span><span id="line-86"></span> |
| <span class="source-line-no">087</span><span id="line-87"> private FlushHandler[] flushHandlers;</span> |
| <span class="source-line-no">088</span><span id="line-88"></span> |
| <span class="source-line-no">089</span><span id="line-89"> private final AtomicInteger flusherIdGen = new AtomicInteger();</span> |
| <span class="source-line-no">090</span><span id="line-90"></span> |
| <span class="source-line-no">091</span><span id="line-91"> private ThreadFactory flusherThreadFactory;</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 List<FlushRequestListener> flushRequestListeners = new ArrayList<>(1);</span> |
| <span class="source-line-no">094</span><span id="line-94"></span> |
| <span class="source-line-no">095</span><span id="line-95"> /**</span> |
| <span class="source-line-no">096</span><span id="line-96"> * Singleton instance inserted into flush queue used for signaling.</span> |
| <span class="source-line-no">097</span><span id="line-97"> */</span> |
| <span class="source-line-no">098</span><span id="line-98"> private static final FlushQueueEntry WAKEUPFLUSH_INSTANCE = new FlushQueueEntry() {</span> |
| <span class="source-line-no">099</span><span id="line-99"> @Override</span> |
| <span class="source-line-no">100</span><span id="line-100"> public long getDelay(TimeUnit unit) {</span> |
| <span class="source-line-no">101</span><span id="line-101"> return 0;</span> |
| <span class="source-line-no">102</span><span id="line-102"> }</span> |
| <span class="source-line-no">103</span><span id="line-103"></span> |
| <span class="source-line-no">104</span><span id="line-104"> @Override</span> |
| <span class="source-line-no">105</span><span id="line-105"> public int compareTo(Delayed o) {</span> |
| <span class="source-line-no">106</span><span id="line-106"> return -1;</span> |
| <span class="source-line-no">107</span><span id="line-107"> }</span> |
| <span class="source-line-no">108</span><span id="line-108"></span> |
| <span class="source-line-no">109</span><span id="line-109"> @Override</span> |
| <span class="source-line-no">110</span><span id="line-110"> public boolean equals(Object obj) {</span> |
| <span class="source-line-no">111</span><span id="line-111"> return obj == this;</span> |
| <span class="source-line-no">112</span><span id="line-112"> }</span> |
| <span class="source-line-no">113</span><span id="line-113"></span> |
| <span class="source-line-no">114</span><span id="line-114"> @Override</span> |
| <span class="source-line-no">115</span><span id="line-115"> public int hashCode() {</span> |
| <span class="source-line-no">116</span><span id="line-116"> return 42;</span> |
| <span class="source-line-no">117</span><span id="line-117"> }</span> |
| <span class="source-line-no">118</span><span id="line-118"> };</span> |
| <span class="source-line-no">119</span><span id="line-119"></span> |
| <span class="source-line-no">120</span><span id="line-120"> /**</span> |
| <span class="source-line-no">121</span><span id="line-121"> * */</span> |
| <span class="source-line-no">122</span><span id="line-122"> public MemStoreFlusher(final Configuration conf, final HRegionServer server) {</span> |
| <span class="source-line-no">123</span><span id="line-123"> super();</span> |
| <span class="source-line-no">124</span><span id="line-124"> this.conf = conf;</span> |
| <span class="source-line-no">125</span><span id="line-125"> this.server = server;</span> |
| <span class="source-line-no">126</span><span id="line-126"> this.threadWakeFrequency = conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);</span> |
| <span class="source-line-no">127</span><span id="line-127"> this.blockingWaitTime = conf.getInt("hbase.hstore.blockingWaitTime", 90000);</span> |
| <span class="source-line-no">128</span><span id="line-128"> int handlerCount = 0;</span> |
| <span class="source-line-no">129</span><span id="line-129"> if (server != null) {</span> |
| <span class="source-line-no">130</span><span id="line-130"> handlerCount = getHandlerCount(conf);</span> |
| <span class="source-line-no">131</span><span id="line-131"> LOG.info("globalMemStoreLimit="</span> |
| <span class="source-line-no">132</span><span id="line-132"> + TraditionalBinaryPrefix</span> |
| <span class="source-line-no">133</span><span id="line-133"> .long2String(this.server.getRegionServerAccounting().getGlobalMemStoreLimit(), "", 1)</span> |
| <span class="source-line-no">134</span><span id="line-134"> + ", globalMemStoreLimitLowMark="</span> |
| <span class="source-line-no">135</span><span id="line-135"> + TraditionalBinaryPrefix.long2String(</span> |
| <span class="source-line-no">136</span><span id="line-136"> this.server.getRegionServerAccounting().getGlobalMemStoreLimitLowMark(), "", 1)</span> |
| <span class="source-line-no">137</span><span id="line-137"> + ", Offheap=" + (this.server.getRegionServerAccounting().isOffheap()));</span> |
| <span class="source-line-no">138</span><span id="line-138"> }</span> |
| <span class="source-line-no">139</span><span id="line-139"> this.flushHandlers = new FlushHandler[handlerCount];</span> |
| <span class="source-line-no">140</span><span id="line-140"> }</span> |
| <span class="source-line-no">141</span><span id="line-141"></span> |
| <span class="source-line-no">142</span><span id="line-142"> public LongAdder getUpdatesBlockedMsHighWater() {</span> |
| <span class="source-line-no">143</span><span id="line-143"> return this.updatesBlockedMsHighWater;</span> |
| <span class="source-line-no">144</span><span id="line-144"> }</span> |
| <span class="source-line-no">145</span><span id="line-145"></span> |
| <span class="source-line-no">146</span><span id="line-146"> /**</span> |
| <span class="source-line-no">147</span><span id="line-147"> * The memstore across all regions has exceeded the low water mark. Pick one region to flush and</span> |
| <span class="source-line-no">148</span><span id="line-148"> * flush it synchronously (this is called from the flush thread)</span> |
| <span class="source-line-no">149</span><span id="line-149"> * @return true if successful</span> |
| <span class="source-line-no">150</span><span id="line-150"> */</span> |
| <span class="source-line-no">151</span><span id="line-151"> private boolean flushOneForGlobalPressure(FlushType flushType) {</span> |
| <span class="source-line-no">152</span><span id="line-152"> SortedMap<Long, Collection<HRegion>> regionsBySize = null;</span> |
| <span class="source-line-no">153</span><span id="line-153"> switch (flushType) {</span> |
| <span class="source-line-no">154</span><span id="line-154"> case ABOVE_OFFHEAP_HIGHER_MARK:</span> |
| <span class="source-line-no">155</span><span id="line-155"> case ABOVE_OFFHEAP_LOWER_MARK:</span> |
| <span class="source-line-no">156</span><span id="line-156"> regionsBySize = server.getCopyOfOnlineRegionsSortedByOffHeapSize();</span> |
| <span class="source-line-no">157</span><span id="line-157"> break;</span> |
| <span class="source-line-no">158</span><span id="line-158"> case ABOVE_ONHEAP_HIGHER_MARK:</span> |
| <span class="source-line-no">159</span><span id="line-159"> case ABOVE_ONHEAP_LOWER_MARK:</span> |
| <span class="source-line-no">160</span><span id="line-160"> default:</span> |
| <span class="source-line-no">161</span><span id="line-161"> regionsBySize = server.getCopyOfOnlineRegionsSortedByOnHeapSize();</span> |
| <span class="source-line-no">162</span><span id="line-162"> }</span> |
| <span class="source-line-no">163</span><span id="line-163"> Set<HRegion> excludedRegions = new HashSet<>();</span> |
| <span class="source-line-no">164</span><span id="line-164"></span> |
| <span class="source-line-no">165</span><span id="line-165"> double secondaryMultiplier =</span> |
| <span class="source-line-no">166</span><span id="line-166"> ServerRegionReplicaUtil.getRegionReplicaStoreFileRefreshMultiplier(conf);</span> |
| <span class="source-line-no">167</span><span id="line-167"></span> |
| <span class="source-line-no">168</span><span id="line-168"> boolean flushedOne = false;</span> |
| <span class="source-line-no">169</span><span id="line-169"> while (!flushedOne) {</span> |
| <span class="source-line-no">170</span><span id="line-170"> // Find the biggest region that doesn't have too many storefiles (might be null!)</span> |
| <span class="source-line-no">171</span><span id="line-171"> HRegion bestFlushableRegion = getBiggestMemStoreRegion(regionsBySize, excludedRegions, true);</span> |
| <span class="source-line-no">172</span><span id="line-172"> // Find the biggest region, total, even if it might have too many flushes.</span> |
| <span class="source-line-no">173</span><span id="line-173"> HRegion bestAnyRegion = getBiggestMemStoreRegion(regionsBySize, excludedRegions, false);</span> |
| <span class="source-line-no">174</span><span id="line-174"> // Find the biggest region that is a secondary region</span> |
| <span class="source-line-no">175</span><span id="line-175"> HRegion bestRegionReplica = getBiggestMemStoreOfRegionReplica(regionsBySize, excludedRegions);</span> |
| <span class="source-line-no">176</span><span id="line-176"> if (bestAnyRegion == null) {</span> |
| <span class="source-line-no">177</span><span id="line-177"> // If bestAnyRegion is null, assign replica. It may be null too. Next step is check for null</span> |
| <span class="source-line-no">178</span><span id="line-178"> bestAnyRegion = bestRegionReplica;</span> |
| <span class="source-line-no">179</span><span id="line-179"> }</span> |
| <span class="source-line-no">180</span><span id="line-180"> if (bestAnyRegion == null) {</span> |
| <span class="source-line-no">181</span><span id="line-181"> LOG.error("Above memory mark but there are no flushable regions!");</span> |
| <span class="source-line-no">182</span><span id="line-182"> return false;</span> |
| <span class="source-line-no">183</span><span id="line-183"> }</span> |
| <span class="source-line-no">184</span><span id="line-184"></span> |
| <span class="source-line-no">185</span><span id="line-185"> HRegion regionToFlush;</span> |
| <span class="source-line-no">186</span><span id="line-186"> long bestAnyRegionSize;</span> |
| <span class="source-line-no">187</span><span id="line-187"> long bestFlushableRegionSize;</span> |
| <span class="source-line-no">188</span><span id="line-188"> switch (flushType) {</span> |
| <span class="source-line-no">189</span><span id="line-189"> case ABOVE_OFFHEAP_HIGHER_MARK:</span> |
| <span class="source-line-no">190</span><span id="line-190"> case ABOVE_OFFHEAP_LOWER_MARK:</span> |
| <span class="source-line-no">191</span><span id="line-191"> bestAnyRegionSize = bestAnyRegion.getMemStoreOffHeapSize();</span> |
| <span class="source-line-no">192</span><span id="line-192"> bestFlushableRegionSize = getMemStoreOffHeapSize(bestFlushableRegion);</span> |
| <span class="source-line-no">193</span><span id="line-193"> break;</span> |
| <span class="source-line-no">194</span><span id="line-194"></span> |
| <span class="source-line-no">195</span><span id="line-195"> case ABOVE_ONHEAP_HIGHER_MARK:</span> |
| <span class="source-line-no">196</span><span id="line-196"> case ABOVE_ONHEAP_LOWER_MARK:</span> |
| <span class="source-line-no">197</span><span id="line-197"> bestAnyRegionSize = bestAnyRegion.getMemStoreHeapSize();</span> |
| <span class="source-line-no">198</span><span id="line-198"> bestFlushableRegionSize = getMemStoreHeapSize(bestFlushableRegion);</span> |
| <span class="source-line-no">199</span><span id="line-199"> break;</span> |
| <span class="source-line-no">200</span><span id="line-200"></span> |
| <span class="source-line-no">201</span><span id="line-201"> default:</span> |
| <span class="source-line-no">202</span><span id="line-202"> bestAnyRegionSize = bestAnyRegion.getMemStoreDataSize();</span> |
| <span class="source-line-no">203</span><span id="line-203"> bestFlushableRegionSize = getMemStoreDataSize(bestFlushableRegion);</span> |
| <span class="source-line-no">204</span><span id="line-204"> }</span> |
| <span class="source-line-no">205</span><span id="line-205"> if (bestAnyRegionSize > 2 * bestFlushableRegionSize) {</span> |
| <span class="source-line-no">206</span><span id="line-206"> // Even if it's not supposed to be flushed, pick a region if it's more than twice</span> |
| <span class="source-line-no">207</span><span id="line-207"> // as big as the best flushable one - otherwise when we're under pressure we make</span> |
| <span class="source-line-no">208</span><span id="line-208"> // lots of little flushes and cause lots of compactions, etc, which just makes</span> |
| <span class="source-line-no">209</span><span id="line-209"> // life worse!</span> |
| <span class="source-line-no">210</span><span id="line-210"> if (LOG.isDebugEnabled()) {</span> |
| <span class="source-line-no">211</span><span id="line-211"> LOG.debug("Under global heap pressure: " + "Region "</span> |
| <span class="source-line-no">212</span><span id="line-212"> + bestAnyRegion.getRegionInfo().getRegionNameAsString() + " has too many "</span> |
| <span class="source-line-no">213</span><span id="line-213"> + "store files, but is " + TraditionalBinaryPrefix.long2String(bestAnyRegionSize, "", 1)</span> |
| <span class="source-line-no">214</span><span id="line-214"> + " vs best flushable region's "</span> |
| <span class="source-line-no">215</span><span id="line-215"> + TraditionalBinaryPrefix.long2String(bestFlushableRegionSize, "", 1)</span> |
| <span class="source-line-no">216</span><span id="line-216"> + ". Choosing the bigger.");</span> |
| <span class="source-line-no">217</span><span id="line-217"> }</span> |
| <span class="source-line-no">218</span><span id="line-218"> regionToFlush = bestAnyRegion;</span> |
| <span class="source-line-no">219</span><span id="line-219"> } else {</span> |
| <span class="source-line-no">220</span><span id="line-220"> if (bestFlushableRegion == null) {</span> |
| <span class="source-line-no">221</span><span id="line-221"> regionToFlush = bestAnyRegion;</span> |
| <span class="source-line-no">222</span><span id="line-222"> } else {</span> |
| <span class="source-line-no">223</span><span id="line-223"> regionToFlush = bestFlushableRegion;</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"> long regionToFlushSize;</span> |
| <span class="source-line-no">228</span><span id="line-228"> long bestRegionReplicaSize;</span> |
| <span class="source-line-no">229</span><span id="line-229"> switch (flushType) {</span> |
| <span class="source-line-no">230</span><span id="line-230"> case ABOVE_OFFHEAP_HIGHER_MARK:</span> |
| <span class="source-line-no">231</span><span id="line-231"> case ABOVE_OFFHEAP_LOWER_MARK:</span> |
| <span class="source-line-no">232</span><span id="line-232"> regionToFlushSize = regionToFlush.getMemStoreOffHeapSize();</span> |
| <span class="source-line-no">233</span><span id="line-233"> bestRegionReplicaSize = getMemStoreOffHeapSize(bestRegionReplica);</span> |
| <span class="source-line-no">234</span><span id="line-234"> break;</span> |
| <span class="source-line-no">235</span><span id="line-235"></span> |
| <span class="source-line-no">236</span><span id="line-236"> case ABOVE_ONHEAP_HIGHER_MARK:</span> |
| <span class="source-line-no">237</span><span id="line-237"> case ABOVE_ONHEAP_LOWER_MARK:</span> |
| <span class="source-line-no">238</span><span id="line-238"> regionToFlushSize = regionToFlush.getMemStoreHeapSize();</span> |
| <span class="source-line-no">239</span><span id="line-239"> bestRegionReplicaSize = getMemStoreHeapSize(bestRegionReplica);</span> |
| <span class="source-line-no">240</span><span id="line-240"> break;</span> |
| <span class="source-line-no">241</span><span id="line-241"></span> |
| <span class="source-line-no">242</span><span id="line-242"> default:</span> |
| <span class="source-line-no">243</span><span id="line-243"> regionToFlushSize = regionToFlush.getMemStoreDataSize();</span> |
| <span class="source-line-no">244</span><span id="line-244"> bestRegionReplicaSize = getMemStoreDataSize(bestRegionReplica);</span> |
| <span class="source-line-no">245</span><span id="line-245"> }</span> |
| <span class="source-line-no">246</span><span id="line-246"></span> |
| <span class="source-line-no">247</span><span id="line-247"> if ((regionToFlush == null || regionToFlushSize == 0) && bestRegionReplicaSize == 0) {</span> |
| <span class="source-line-no">248</span><span id="line-248"> // A concurrency issue (such as splitting region) may happen such that the online region</span> |
| <span class="source-line-no">249</span><span id="line-249"> // seen by getCopyOfOnlineRegionsSortedByXX() method is no longer eligible to</span> |
| <span class="source-line-no">250</span><span id="line-250"> // getBiggestMemStoreRegion(). This means that we can come out of the loop</span> |
| <span class="source-line-no">251</span><span id="line-251"> LOG.debug("Above memory mark but there is no flushable region");</span> |
| <span class="source-line-no">252</span><span id="line-252"> return false;</span> |
| <span class="source-line-no">253</span><span id="line-253"> }</span> |
| <span class="source-line-no">254</span><span id="line-254"></span> |
| <span class="source-line-no">255</span><span id="line-255"> if (</span> |
| <span class="source-line-no">256</span><span id="line-256"> regionToFlush == null || (bestRegionReplica != null</span> |
| <span class="source-line-no">257</span><span id="line-257"> && ServerRegionReplicaUtil.isRegionReplicaStoreFileRefreshEnabled(conf)</span> |
| <span class="source-line-no">258</span><span id="line-258"> && (bestRegionReplicaSize > secondaryMultiplier * regionToFlushSize))</span> |
| <span class="source-line-no">259</span><span id="line-259"> ) {</span> |
| <span class="source-line-no">260</span><span id="line-260"> LOG.info("Refreshing storefiles of region " + bestRegionReplica</span> |
| <span class="source-line-no">261</span><span id="line-261"> + " due to global heap pressure. Total memstore off heap size="</span> |
| <span class="source-line-no">262</span><span id="line-262"> + TraditionalBinaryPrefix</span> |
| <span class="source-line-no">263</span><span id="line-263"> .long2String(server.getRegionServerAccounting().getGlobalMemStoreOffHeapSize(), "", 1)</span> |
| <span class="source-line-no">264</span><span id="line-264"> + " memstore heap size=" + TraditionalBinaryPrefix</span> |
| <span class="source-line-no">265</span><span id="line-265"> .long2String(server.getRegionServerAccounting().getGlobalMemStoreHeapSize(), "", 1));</span> |
| <span class="source-line-no">266</span><span id="line-266"> flushedOne = refreshStoreFilesAndReclaimMemory(bestRegionReplica);</span> |
| <span class="source-line-no">267</span><span id="line-267"> if (!flushedOne) {</span> |
| <span class="source-line-no">268</span><span id="line-268"> LOG.info("Excluding secondary region " + bestRegionReplica</span> |
| <span class="source-line-no">269</span><span id="line-269"> + " - trying to find a different region to refresh files.");</span> |
| <span class="source-line-no">270</span><span id="line-270"> excludedRegions.add(bestRegionReplica);</span> |
| <span class="source-line-no">271</span><span id="line-271"> }</span> |
| <span class="source-line-no">272</span><span id="line-272"> } else {</span> |
| <span class="source-line-no">273</span><span id="line-273"> LOG.info("Flush of region " + regionToFlush + " due to global heap pressure. "</span> |
| <span class="source-line-no">274</span><span id="line-274"> + "Flush type=" + flushType.toString() + ", Total Memstore Heap size="</span> |
| <span class="source-line-no">275</span><span id="line-275"> + TraditionalBinaryPrefix</span> |
| <span class="source-line-no">276</span><span id="line-276"> .long2String(server.getRegionServerAccounting().getGlobalMemStoreHeapSize(), "", 1)</span> |
| <span class="source-line-no">277</span><span id="line-277"> + ", Total Memstore Off-Heap size="</span> |
| <span class="source-line-no">278</span><span id="line-278"> + TraditionalBinaryPrefix</span> |
| <span class="source-line-no">279</span><span id="line-279"> .long2String(server.getRegionServerAccounting().getGlobalMemStoreOffHeapSize(), "", 1)</span> |
| <span class="source-line-no">280</span><span id="line-280"> + ", Region memstore size="</span> |
| <span class="source-line-no">281</span><span id="line-281"> + TraditionalBinaryPrefix.long2String(regionToFlushSize, "", 1));</span> |
| <span class="source-line-no">282</span><span id="line-282"> flushedOne = flushRegion(regionToFlush, true, null, FlushLifeCycleTracker.DUMMY);</span> |
| <span class="source-line-no">283</span><span id="line-283"></span> |
| <span class="source-line-no">284</span><span id="line-284"> if (!flushedOne) {</span> |
| <span class="source-line-no">285</span><span id="line-285"> LOG.info("Excluding unflushable region " + regionToFlush</span> |
| <span class="source-line-no">286</span><span id="line-286"> + " - trying to find a different region to flush.");</span> |
| <span class="source-line-no">287</span><span id="line-287"> excludedRegions.add(regionToFlush);</span> |
| <span class="source-line-no">288</span><span id="line-288"> }</span> |
| <span class="source-line-no">289</span><span id="line-289"> }</span> |
| <span class="source-line-no">290</span><span id="line-290"> }</span> |
| <span class="source-line-no">291</span><span id="line-291"> return true;</span> |
| <span class="source-line-no">292</span><span id="line-292"> }</span> |
| <span class="source-line-no">293</span><span id="line-293"></span> |
| <span class="source-line-no">294</span><span id="line-294"> /** Returns Return memstore offheap size or null if <code>r</code> is null */</span> |
| <span class="source-line-no">295</span><span id="line-295"> private static long getMemStoreOffHeapSize(HRegion r) {</span> |
| <span class="source-line-no">296</span><span id="line-296"> return r == null ? 0 : r.getMemStoreOffHeapSize();</span> |
| <span class="source-line-no">297</span><span id="line-297"> }</span> |
| <span class="source-line-no">298</span><span id="line-298"></span> |
| <span class="source-line-no">299</span><span id="line-299"> /** Returns Return memstore heap size or null if <code>r</code> is null */</span> |
| <span class="source-line-no">300</span><span id="line-300"> private static long getMemStoreHeapSize(HRegion r) {</span> |
| <span class="source-line-no">301</span><span id="line-301"> return r == null ? 0 : r.getMemStoreHeapSize();</span> |
| <span class="source-line-no">302</span><span id="line-302"> }</span> |
| <span class="source-line-no">303</span><span id="line-303"></span> |
| <span class="source-line-no">304</span><span id="line-304"> /** Returns Return memstore data size or null if <code>r</code> is null */</span> |
| <span class="source-line-no">305</span><span id="line-305"> private static long getMemStoreDataSize(HRegion r) {</span> |
| <span class="source-line-no">306</span><span id="line-306"> return r == null ? 0 : r.getMemStoreDataSize();</span> |
| <span class="source-line-no">307</span><span id="line-307"> }</span> |
| <span class="source-line-no">308</span><span id="line-308"></span> |
| <span class="source-line-no">309</span><span id="line-309"> private class FlushHandler extends Thread {</span> |
| <span class="source-line-no">310</span><span id="line-310"></span> |
| <span class="source-line-no">311</span><span id="line-311"> private final AtomicBoolean running = new AtomicBoolean(true);</span> |
| <span class="source-line-no">312</span><span id="line-312"></span> |
| <span class="source-line-no">313</span><span id="line-313"> private FlushHandler(String name) {</span> |
| <span class="source-line-no">314</span><span id="line-314"> super(name);</span> |
| <span class="source-line-no">315</span><span id="line-315"> }</span> |
| <span class="source-line-no">316</span><span id="line-316"></span> |
| <span class="source-line-no">317</span><span id="line-317"> @Override</span> |
| <span class="source-line-no">318</span><span id="line-318"> public void run() {</span> |
| <span class="source-line-no">319</span><span id="line-319"> while (!server.isStopped() && running.get()) {</span> |
| <span class="source-line-no">320</span><span id="line-320"> FlushQueueEntry fqe = null;</span> |
| <span class="source-line-no">321</span><span id="line-321"> try {</span> |
| <span class="source-line-no">322</span><span id="line-322"> wakeupPending.set(false); // allow someone to wake us up again</span> |
| <span class="source-line-no">323</span><span id="line-323"> fqe = flushQueue.poll(threadWakeFrequency, TimeUnit.MILLISECONDS);</span> |
| <span class="source-line-no">324</span><span id="line-324"> if (fqe == null || fqe == WAKEUPFLUSH_INSTANCE) {</span> |
| <span class="source-line-no">325</span><span id="line-325"> FlushType type = isAboveLowWaterMark();</span> |
| <span class="source-line-no">326</span><span id="line-326"> if (type != FlushType.NORMAL) {</span> |
| <span class="source-line-no">327</span><span id="line-327"> LOG.debug("Flush thread woke up because memory above low water="</span> |
| <span class="source-line-no">328</span><span id="line-328"> + TraditionalBinaryPrefix.long2String(</span> |
| <span class="source-line-no">329</span><span id="line-329"> server.getRegionServerAccounting().getGlobalMemStoreLimitLowMark(), "", 1));</span> |
| <span class="source-line-no">330</span><span id="line-330"> // For offheap memstore, even if the lower water mark was breached due to heap</span> |
| <span class="source-line-no">331</span><span id="line-331"> // overhead</span> |
| <span class="source-line-no">332</span><span id="line-332"> // we still select the regions based on the region's memstore data size.</span> |
| <span class="source-line-no">333</span><span id="line-333"> // TODO : If we want to decide based on heap over head it can be done without tracking</span> |
| <span class="source-line-no">334</span><span id="line-334"> // it per region.</span> |
| <span class="source-line-no">335</span><span id="line-335"> if (!flushOneForGlobalPressure(type)) {</span> |
| <span class="source-line-no">336</span><span id="line-336"> // Wasn't able to flush any region, but we're above low water mark</span> |
| <span class="source-line-no">337</span><span id="line-337"> // This is unlikely to happen, but might happen when closing the</span> |
| <span class="source-line-no">338</span><span id="line-338"> // entire server - another thread is flushing regions. We'll just</span> |
| <span class="source-line-no">339</span><span id="line-339"> // sleep a little bit to avoid spinning, and then pretend that</span> |
| <span class="source-line-no">340</span><span id="line-340"> // we flushed one, so anyone blocked will check again</span> |
| <span class="source-line-no">341</span><span id="line-341"> Thread.sleep(1000);</span> |
| <span class="source-line-no">342</span><span id="line-342"> wakeUpIfBlocking();</span> |
| <span class="source-line-no">343</span><span id="line-343"> }</span> |
| <span class="source-line-no">344</span><span id="line-344"> // Enqueue another one of these tokens so we'll wake up again</span> |
| <span class="source-line-no">345</span><span id="line-345"> wakeupFlushThread();</span> |
| <span class="source-line-no">346</span><span id="line-346"> }</span> |
| <span class="source-line-no">347</span><span id="line-347"> continue;</span> |
| <span class="source-line-no">348</span><span id="line-348"> }</span> |
| <span class="source-line-no">349</span><span id="line-349"> FlushRegionEntry fre = (FlushRegionEntry) fqe;</span> |
| <span class="source-line-no">350</span><span id="line-350"> if (!flushRegion(fre)) {</span> |
| <span class="source-line-no">351</span><span id="line-351"> break;</span> |
| <span class="source-line-no">352</span><span id="line-352"> }</span> |
| <span class="source-line-no">353</span><span id="line-353"> } catch (InterruptedException ex) {</span> |
| <span class="source-line-no">354</span><span id="line-354"> continue;</span> |
| <span class="source-line-no">355</span><span id="line-355"> } catch (ConcurrentModificationException ex) {</span> |
| <span class="source-line-no">356</span><span id="line-356"> continue;</span> |
| <span class="source-line-no">357</span><span id="line-357"> } catch (Exception ex) {</span> |
| <span class="source-line-no">358</span><span id="line-358"> LOG.error("Cache flusher failed for entry " + fqe, ex);</span> |
| <span class="source-line-no">359</span><span id="line-359"> if (!server.checkFileSystem()) {</span> |
| <span class="source-line-no">360</span><span id="line-360"> break;</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"> if (server.isStopped()) {</span> |
| <span class="source-line-no">366</span><span id="line-366"> synchronized (regionsInQueue) {</span> |
| <span class="source-line-no">367</span><span id="line-367"> regionsInQueue.clear();</span> |
| <span class="source-line-no">368</span><span id="line-368"> flushQueue.clear();</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"> // Signal anyone waiting, so they see the close flag</span> |
| <span class="source-line-no">372</span><span id="line-372"> wakeUpIfBlocking();</span> |
| <span class="source-line-no">373</span><span id="line-373"> }</span> |
| <span class="source-line-no">374</span><span id="line-374"> LOG.info(getName() + " exiting");</span> |
| <span class="source-line-no">375</span><span id="line-375"> }</span> |
| <span class="source-line-no">376</span><span id="line-376"></span> |
| <span class="source-line-no">377</span><span id="line-377"> public void shutdown() {</span> |
| <span class="source-line-no">378</span><span id="line-378"> if (!running.compareAndSet(true, false)) {</span> |
| <span class="source-line-no">379</span><span id="line-379"> LOG.warn("{} is already signaled to shutdown", getName());</span> |
| <span class="source-line-no">380</span><span id="line-380"> }</span> |
| <span class="source-line-no">381</span><span id="line-381"> }</span> |
| <span class="source-line-no">382</span><span id="line-382"> }</span> |
| <span class="source-line-no">383</span><span id="line-383"></span> |
| <span class="source-line-no">384</span><span id="line-384"> private void wakeupFlushThread() {</span> |
| <span class="source-line-no">385</span><span id="line-385"> if (wakeupPending.compareAndSet(false, true)) {</span> |
| <span class="source-line-no">386</span><span id="line-386"> flushQueue.add(WAKEUPFLUSH_INSTANCE);</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"></span> |
| <span class="source-line-no">390</span><span id="line-390"> private HRegion getBiggestMemStoreRegion(SortedMap<Long, Collection<HRegion>> regionsBySize,</span> |
| <span class="source-line-no">391</span><span id="line-391"> Set<HRegion> excludedRegions, boolean checkStoreFileCount) {</span> |
| <span class="source-line-no">392</span><span id="line-392"> synchronized (regionsInQueue) {</span> |
| <span class="source-line-no">393</span><span id="line-393"> for (Map.Entry<Long, Collection<HRegion>> entry : regionsBySize.entrySet()) {</span> |
| <span class="source-line-no">394</span><span id="line-394"> for (HRegion region : entry.getValue()) {</span> |
| <span class="source-line-no">395</span><span id="line-395"> if (excludedRegions.contains(region)) {</span> |
| <span class="source-line-no">396</span><span id="line-396"> continue;</span> |
| <span class="source-line-no">397</span><span id="line-397"> }</span> |
| <span class="source-line-no">398</span><span id="line-398"></span> |
| <span class="source-line-no">399</span><span id="line-399"> if (region.writestate.flushing || !region.writestate.writesEnabled) {</span> |
| <span class="source-line-no">400</span><span id="line-400"> continue;</span> |
| <span class="source-line-no">401</span><span id="line-401"> }</span> |
| <span class="source-line-no">402</span><span id="line-402"></span> |
| <span class="source-line-no">403</span><span id="line-403"> if (checkStoreFileCount && isTooManyStoreFiles(region)) {</span> |
| <span class="source-line-no">404</span><span id="line-404"> continue;</span> |
| <span class="source-line-no">405</span><span id="line-405"> }</span> |
| <span class="source-line-no">406</span><span id="line-406"> return region;</span> |
| <span class="source-line-no">407</span><span id="line-407"> }</span> |
| <span class="source-line-no">408</span><span id="line-408"> }</span> |
| <span class="source-line-no">409</span><span id="line-409"> }</span> |
| <span class="source-line-no">410</span><span id="line-410"> return null;</span> |
| <span class="source-line-no">411</span><span id="line-411"> }</span> |
| <span class="source-line-no">412</span><span id="line-412"></span> |
| <span class="source-line-no">413</span><span id="line-413"> private HRegion getBiggestMemStoreOfRegionReplica(</span> |
| <span class="source-line-no">414</span><span id="line-414"> SortedMap<Long, Collection<HRegion>> regionsBySize, Set<HRegion> excludedRegions) {</span> |
| <span class="source-line-no">415</span><span id="line-415"> synchronized (regionsInQueue) {</span> |
| <span class="source-line-no">416</span><span id="line-416"> for (Map.Entry<Long, Collection<HRegion>> entry : regionsBySize.entrySet()) {</span> |
| <span class="source-line-no">417</span><span id="line-417"> for (HRegion region : entry.getValue()) {</span> |
| <span class="source-line-no">418</span><span id="line-418"> if (excludedRegions.contains(region)) {</span> |
| <span class="source-line-no">419</span><span id="line-419"> continue;</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"> if (RegionReplicaUtil.isDefaultReplica(region.getRegionInfo())) {</span> |
| <span class="source-line-no">423</span><span id="line-423"> continue;</span> |
| <span class="source-line-no">424</span><span id="line-424"> }</span> |
| <span class="source-line-no">425</span><span id="line-425"> return region;</span> |
| <span class="source-line-no">426</span><span id="line-426"> }</span> |
| <span class="source-line-no">427</span><span id="line-427"> }</span> |
| <span class="source-line-no">428</span><span id="line-428"> }</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"></span> |
| <span class="source-line-no">432</span><span id="line-432"> private boolean refreshStoreFilesAndReclaimMemory(Region region) {</span> |
| <span class="source-line-no">433</span><span id="line-433"> try {</span> |
| <span class="source-line-no">434</span><span id="line-434"> return region.refreshStoreFiles();</span> |
| <span class="source-line-no">435</span><span id="line-435"> } catch (IOException e) {</span> |
| <span class="source-line-no">436</span><span id="line-436"> LOG.warn("Refreshing store files failed with exception", e);</span> |
| <span class="source-line-no">437</span><span id="line-437"> }</span> |
| <span class="source-line-no">438</span><span id="line-438"> return false;</span> |
| <span class="source-line-no">439</span><span id="line-439"> }</span> |
| <span class="source-line-no">440</span><span id="line-440"></span> |
| <span class="source-line-no">441</span><span id="line-441"> /**</span> |
| <span class="source-line-no">442</span><span id="line-442"> * Return the FlushType if global memory usage is above the high watermark</span> |
| <span class="source-line-no">443</span><span id="line-443"> */</span> |
| <span class="source-line-no">444</span><span id="line-444"> private FlushType isAboveHighWaterMark() {</span> |
| <span class="source-line-no">445</span><span id="line-445"> return server.getRegionServerAccounting().isAboveHighWaterMark();</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"> * Return the FlushType if we're above the low watermark</span> |
| <span class="source-line-no">450</span><span id="line-450"> */</span> |
| <span class="source-line-no">451</span><span id="line-451"> private FlushType isAboveLowWaterMark() {</span> |
| <span class="source-line-no">452</span><span id="line-452"> return server.getRegionServerAccounting().isAboveLowWaterMark();</span> |
| <span class="source-line-no">453</span><span id="line-453"> }</span> |
| <span class="source-line-no">454</span><span id="line-454"></span> |
| <span class="source-line-no">455</span><span id="line-455"> @Override</span> |
| <span class="source-line-no">456</span><span id="line-456"> public boolean requestFlush(HRegion r, FlushLifeCycleTracker tracker) {</span> |
| <span class="source-line-no">457</span><span id="line-457"> return this.requestFlush(r, null, tracker);</span> |
| <span class="source-line-no">458</span><span id="line-458"> }</span> |
| <span class="source-line-no">459</span><span id="line-459"></span> |
| <span class="source-line-no">460</span><span id="line-460"> @Override</span> |
| <span class="source-line-no">461</span><span id="line-461"> public boolean requestFlush(HRegion r, List<byte[]> families, FlushLifeCycleTracker tracker) {</span> |
| <span class="source-line-no">462</span><span id="line-462"> synchronized (regionsInQueue) {</span> |
| <span class="source-line-no">463</span><span id="line-463"> FlushRegionEntry existFqe = regionsInQueue.get(r);</span> |
| <span class="source-line-no">464</span><span id="line-464"> if (existFqe != null) {</span> |
| <span class="source-line-no">465</span><span id="line-465"> // if a delayed one exists and not reach the time to execute, just remove it</span> |
| <span class="source-line-no">466</span><span id="line-466"> if (existFqe.isDelay() && existFqe.whenToExpire > EnvironmentEdgeManager.currentTime()) {</span> |
| <span class="source-line-no">467</span><span id="line-467"> LOG.info("Remove the existing delayed flush entry for {}, "</span> |
| <span class="source-line-no">468</span><span id="line-468"> + "because we need to flush it immediately", r);</span> |
| <span class="source-line-no">469</span><span id="line-469"> this.regionsInQueue.remove(r);</span> |
| <span class="source-line-no">470</span><span id="line-470"> this.flushQueue.remove(existFqe);</span> |
| <span class="source-line-no">471</span><span id="line-471"> r.decrementFlushesQueuedCount();</span> |
| <span class="source-line-no">472</span><span id="line-472"> } else {</span> |
| <span class="source-line-no">473</span><span id="line-473"> tracker.notExecuted("Flush already requested on " + r);</span> |
| <span class="source-line-no">474</span><span id="line-474"> return false;</span> |
| <span class="source-line-no">475</span><span id="line-475"> }</span> |
| <span class="source-line-no">476</span><span id="line-476"> }</span> |
| <span class="source-line-no">477</span><span id="line-477"></span> |
| <span class="source-line-no">478</span><span id="line-478"> // This entry has no delay so it will be added at the top of the flush</span> |
| <span class="source-line-no">479</span><span id="line-479"> // queue. It'll come out near immediately.</span> |
| <span class="source-line-no">480</span><span id="line-480"> FlushRegionEntry fqe = new FlushRegionEntry(r, families, tracker);</span> |
| <span class="source-line-no">481</span><span id="line-481"> this.regionsInQueue.put(r, fqe);</span> |
| <span class="source-line-no">482</span><span id="line-482"> this.flushQueue.add(fqe);</span> |
| <span class="source-line-no">483</span><span id="line-483"> r.incrementFlushesQueuedCount();</span> |
| <span class="source-line-no">484</span><span id="line-484"> return true;</span> |
| <span class="source-line-no">485</span><span id="line-485"> }</span> |
| <span class="source-line-no">486</span><span id="line-486"> }</span> |
| <span class="source-line-no">487</span><span id="line-487"></span> |
| <span class="source-line-no">488</span><span id="line-488"> @Override</span> |
| <span class="source-line-no">489</span><span id="line-489"> public boolean requestDelayedFlush(HRegion r, long delay) {</span> |
| <span class="source-line-no">490</span><span id="line-490"> synchronized (regionsInQueue) {</span> |
| <span class="source-line-no">491</span><span id="line-491"> if (!regionsInQueue.containsKey(r)) {</span> |
| <span class="source-line-no">492</span><span id="line-492"> // This entry has some delay</span> |
| <span class="source-line-no">493</span><span id="line-493"> FlushRegionEntry fqe = new FlushRegionEntry(r, null, FlushLifeCycleTracker.DUMMY);</span> |
| <span class="source-line-no">494</span><span id="line-494"> fqe.requeue(delay);</span> |
| <span class="source-line-no">495</span><span id="line-495"> this.regionsInQueue.put(r, fqe);</span> |
| <span class="source-line-no">496</span><span id="line-496"> this.flushQueue.add(fqe);</span> |
| <span class="source-line-no">497</span><span id="line-497"> r.incrementFlushesQueuedCount();</span> |
| <span class="source-line-no">498</span><span id="line-498"> return true;</span> |
| <span class="source-line-no">499</span><span id="line-499"> }</span> |
| <span class="source-line-no">500</span><span id="line-500"> return false;</span> |
| <span class="source-line-no">501</span><span id="line-501"> }</span> |
| <span class="source-line-no">502</span><span id="line-502"> }</span> |
| <span class="source-line-no">503</span><span id="line-503"></span> |
| <span class="source-line-no">504</span><span id="line-504"> public int getFlushQueueSize() {</span> |
| <span class="source-line-no">505</span><span id="line-505"> return flushQueue.size();</span> |
| <span class="source-line-no">506</span><span id="line-506"> }</span> |
| <span class="source-line-no">507</span><span id="line-507"></span> |
| <span class="source-line-no">508</span><span id="line-508"> /**</span> |
| <span class="source-line-no">509</span><span id="line-509"> * Only interrupt once it's done with a run through the work loop.</span> |
| <span class="source-line-no">510</span><span id="line-510"> */</span> |
| <span class="source-line-no">511</span><span id="line-511"> void interruptIfNecessary() {</span> |
| <span class="source-line-no">512</span><span id="line-512"> lock.writeLock().lock();</span> |
| <span class="source-line-no">513</span><span id="line-513"> try {</span> |
| <span class="source-line-no">514</span><span id="line-514"> for (FlushHandler flushHandler : flushHandlers) {</span> |
| <span class="source-line-no">515</span><span id="line-515"> if (flushHandler != null) {</span> |
| <span class="source-line-no">516</span><span id="line-516"> flushHandler.interrupt();</span> |
| <span class="source-line-no">517</span><span id="line-517"> }</span> |
| <span class="source-line-no">518</span><span id="line-518"> }</span> |
| <span class="source-line-no">519</span><span id="line-519"> } finally {</span> |
| <span class="source-line-no">520</span><span id="line-520"> lock.writeLock().unlock();</span> |
| <span class="source-line-no">521</span><span id="line-521"> }</span> |
| <span class="source-line-no">522</span><span id="line-522"> }</span> |
| <span class="source-line-no">523</span><span id="line-523"></span> |
| <span class="source-line-no">524</span><span id="line-524"> synchronized void start(UncaughtExceptionHandler eh) {</span> |
| <span class="source-line-no">525</span><span id="line-525"> this.flusherThreadFactory =</span> |
| <span class="source-line-no">526</span><span id="line-526"> new ThreadFactoryBuilder().setDaemon(true).setUncaughtExceptionHandler(eh).build();</span> |
| <span class="source-line-no">527</span><span id="line-527"> lock.readLock().lock();</span> |
| <span class="source-line-no">528</span><span id="line-528"> try {</span> |
| <span class="source-line-no">529</span><span id="line-529"> startFlushHandlerThreads(flushHandlers, 0, flushHandlers.length);</span> |
| <span class="source-line-no">530</span><span id="line-530"> } finally {</span> |
| <span class="source-line-no">531</span><span id="line-531"> lock.readLock().unlock();</span> |
| <span class="source-line-no">532</span><span id="line-532"> }</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"> boolean isAlive() {</span> |
| <span class="source-line-no">536</span><span id="line-536"> lock.readLock().lock();</span> |
| <span class="source-line-no">537</span><span id="line-537"> try {</span> |
| <span class="source-line-no">538</span><span id="line-538"> for (FlushHandler flushHandler : flushHandlers) {</span> |
| <span class="source-line-no">539</span><span id="line-539"> if (flushHandler != null && flushHandler.isAlive()) {</span> |
| <span class="source-line-no">540</span><span id="line-540"> return true;</span> |
| <span class="source-line-no">541</span><span id="line-541"> }</span> |
| <span class="source-line-no">542</span><span id="line-542"> }</span> |
| <span class="source-line-no">543</span><span id="line-543"> return false;</span> |
| <span class="source-line-no">544</span><span id="line-544"> } finally {</span> |
| <span class="source-line-no">545</span><span id="line-545"> lock.readLock().unlock();</span> |
| <span class="source-line-no">546</span><span id="line-546"> }</span> |
| <span class="source-line-no">547</span><span id="line-547"> }</span> |
| <span class="source-line-no">548</span><span id="line-548"></span> |
| <span class="source-line-no">549</span><span id="line-549"> void shutdown() {</span> |
| <span class="source-line-no">550</span><span id="line-550"> lock.readLock().lock();</span> |
| <span class="source-line-no">551</span><span id="line-551"> try {</span> |
| <span class="source-line-no">552</span><span id="line-552"> for (FlushHandler flushHandler : flushHandlers) {</span> |
| <span class="source-line-no">553</span><span id="line-553"> if (flushHandler != null) {</span> |
| <span class="source-line-no">554</span><span id="line-554"> Threads.shutdown(flushHandler);</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"> } finally {</span> |
| <span class="source-line-no">558</span><span id="line-558"> lock.readLock().unlock();</span> |
| <span class="source-line-no">559</span><span id="line-559"> }</span> |
| <span class="source-line-no">560</span><span id="line-560"> }</span> |
| <span class="source-line-no">561</span><span id="line-561"></span> |
| <span class="source-line-no">562</span><span id="line-562"> /**</span> |
| <span class="source-line-no">563</span><span id="line-563"> * A flushRegion that checks store file count. If too many, puts the flush on delay queue to retry</span> |
| <span class="source-line-no">564</span><span id="line-564"> * later.</span> |
| <span class="source-line-no">565</span><span id="line-565"> * @return true if the region was successfully flushed, false otherwise. If false, there will be</span> |
| <span class="source-line-no">566</span><span id="line-566"> * accompanying log messages explaining why the region was not flushed.</span> |
| <span class="source-line-no">567</span><span id="line-567"> */</span> |
| <span class="source-line-no">568</span><span id="line-568"> private boolean flushRegion(final FlushRegionEntry fqe) {</span> |
| <span class="source-line-no">569</span><span id="line-569"> HRegion region = fqe.region;</span> |
| <span class="source-line-no">570</span><span id="line-570"> if (!region.getRegionInfo().isMetaRegion() && isTooManyStoreFiles(region)) {</span> |
| <span class="source-line-no">571</span><span id="line-571"> if (fqe.isMaximumWait(this.blockingWaitTime)) {</span> |
| <span class="source-line-no">572</span><span id="line-572"> LOG.info("Waited " + (EnvironmentEdgeManager.currentTime() - fqe.createTime)</span> |
| <span class="source-line-no">573</span><span id="line-573"> + "ms on a compaction to clean up 'too many store files'; waited "</span> |
| <span class="source-line-no">574</span><span id="line-574"> + "long enough... proceeding with flush of "</span> |
| <span class="source-line-no">575</span><span id="line-575"> + region.getRegionInfo().getRegionNameAsString());</span> |
| <span class="source-line-no">576</span><span id="line-576"> } else {</span> |
| <span class="source-line-no">577</span><span id="line-577"> // If this is first time we've been put off, then emit a log message.</span> |
| <span class="source-line-no">578</span><span id="line-578"> if (fqe.getRequeueCount() <= 0) {</span> |
| <span class="source-line-no">579</span><span id="line-579"> // Note: We don't impose blockingStoreFiles constraint on meta regions</span> |
| <span class="source-line-no">580</span><span id="line-580"> LOG.warn("{} has too many store files({}); delaying flush up to {} ms",</span> |
| <span class="source-line-no">581</span><span id="line-581"> region.getRegionInfo().getEncodedName(), getStoreFileCount(region),</span> |
| <span class="source-line-no">582</span><span id="line-582"> this.blockingWaitTime);</span> |
| <span class="source-line-no">583</span><span id="line-583"> final CompactSplit compactSplitThread = server.getCompactSplitThread();</span> |
| <span class="source-line-no">584</span><span id="line-584"> if (!compactSplitThread.requestSplit(region)) {</span> |
| <span class="source-line-no">585</span><span id="line-585"> try {</span> |
| <span class="source-line-no">586</span><span id="line-586"> compactSplitThread.requestSystemCompaction(region, Thread.currentThread().getName());</span> |
| <span class="source-line-no">587</span><span id="line-587"> } catch (IOException e) {</span> |
| <span class="source-line-no">588</span><span id="line-588"> e = e instanceof RemoteException ? ((RemoteException) e).unwrapRemoteException() : e;</span> |
| <span class="source-line-no">589</span><span id="line-589"> LOG.error("Cache flush failed for region "</span> |
| <span class="source-line-no">590</span><span id="line-590"> + Bytes.toStringBinary(region.getRegionInfo().getRegionName()), e);</span> |
| <span class="source-line-no">591</span><span id="line-591"> }</span> |
| <span class="source-line-no">592</span><span id="line-592"> }</span> |
| <span class="source-line-no">593</span><span id="line-593"> }</span> |
| <span class="source-line-no">594</span><span id="line-594"></span> |
| <span class="source-line-no">595</span><span id="line-595"> // Put back on the queue. Have it come back out of the queue</span> |
| <span class="source-line-no">596</span><span id="line-596"> // after a delay of this.blockingWaitTime / 100 ms.</span> |
| <span class="source-line-no">597</span><span id="line-597"> this.flushQueue.add(fqe.requeue(this.blockingWaitTime / 100));</span> |
| <span class="source-line-no">598</span><span id="line-598"> // Tell a lie, it's not flushed but it's ok</span> |
| <span class="source-line-no">599</span><span id="line-599"> return true;</span> |
| <span class="source-line-no">600</span><span id="line-600"> }</span> |
| <span class="source-line-no">601</span><span id="line-601"> }</span> |
| <span class="source-line-no">602</span><span id="line-602"> return flushRegion(region, false, fqe.families, fqe.getTracker());</span> |
| <span class="source-line-no">603</span><span id="line-603"> }</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"> * Flush a region.</span> |
| <span class="source-line-no">607</span><span id="line-607"> * @param region Region to flush.</span> |
| <span class="source-line-no">608</span><span id="line-608"> * @param emergencyFlush Set if we are being force flushed. If true the region needs to be removed</span> |
| <span class="source-line-no">609</span><span id="line-609"> * from the flush queue. If false, when we were called from the main flusher</span> |
| <span class="source-line-no">610</span><span id="line-610"> * run loop and we got the entry to flush by calling poll on the flush queue</span> |
| <span class="source-line-no">611</span><span id="line-611"> * (which removed it).</span> |
| <span class="source-line-no">612</span><span id="line-612"> * @param families stores of region to flush.</span> |
| <span class="source-line-no">613</span><span id="line-613"> * @return true if the region was successfully flushed, false otherwise. If false, there will be</span> |
| <span class="source-line-no">614</span><span id="line-614"> * accompanying log messages explaining why the region was not flushed.</span> |
| <span class="source-line-no">615</span><span id="line-615"> */</span> |
| <span class="source-line-no">616</span><span id="line-616"> private boolean flushRegion(HRegion region, boolean emergencyFlush, List<byte[]> families,</span> |
| <span class="source-line-no">617</span><span id="line-617"> FlushLifeCycleTracker tracker) {</span> |
| <span class="source-line-no">618</span><span id="line-618"> synchronized (this.regionsInQueue) {</span> |
| <span class="source-line-no">619</span><span id="line-619"> FlushRegionEntry fqe = this.regionsInQueue.remove(region);</span> |
| <span class="source-line-no">620</span><span id="line-620"> // Use the start time of the FlushRegionEntry if available</span> |
| <span class="source-line-no">621</span><span id="line-621"> if (fqe != null && emergencyFlush) {</span> |
| <span class="source-line-no">622</span><span id="line-622"> // Need to remove from region from delay queue. When NOT an</span> |
| <span class="source-line-no">623</span><span id="line-623"> // emergencyFlush, then item was removed via a flushQueue.poll.</span> |
| <span class="source-line-no">624</span><span id="line-624"> flushQueue.remove(fqe);</span> |
| <span class="source-line-no">625</span><span id="line-625"> }</span> |
| <span class="source-line-no">626</span><span id="line-626"> }</span> |
| <span class="source-line-no">627</span><span id="line-627"></span> |
| <span class="source-line-no">628</span><span id="line-628"> tracker.beforeExecution();</span> |
| <span class="source-line-no">629</span><span id="line-629"> lock.readLock().lock();</span> |
| <span class="source-line-no">630</span><span id="line-630"> final CompactSplit compactSplitThread = server.getCompactSplitThread();</span> |
| <span class="source-line-no">631</span><span id="line-631"> try {</span> |
| <span class="source-line-no">632</span><span id="line-632"> notifyFlushRequest(region, emergencyFlush);</span> |
| <span class="source-line-no">633</span><span id="line-633"> FlushResult flushResult = region.flushcache(families, false, tracker);</span> |
| <span class="source-line-no">634</span><span id="line-634"> boolean shouldCompact = flushResult.isCompactionNeeded();</span> |
| <span class="source-line-no">635</span><span id="line-635"> // We just want to check the size</span> |
| <span class="source-line-no">636</span><span id="line-636"> boolean shouldSplit = region.checkSplit().isPresent();</span> |
| <span class="source-line-no">637</span><span id="line-637"> if (shouldSplit) {</span> |
| <span class="source-line-no">638</span><span id="line-638"> compactSplitThread.requestSplit(region);</span> |
| <span class="source-line-no">639</span><span id="line-639"> } else if (shouldCompact) {</span> |
| <span class="source-line-no">640</span><span id="line-640"> compactSplitThread.requestSystemCompaction(region, Thread.currentThread().getName());</span> |
| <span class="source-line-no">641</span><span id="line-641"> }</span> |
| <span class="source-line-no">642</span><span id="line-642"> } catch (DroppedSnapshotException ex) {</span> |
| <span class="source-line-no">643</span><span id="line-643"> // Cache flush can fail in a few places. If it fails in a critical</span> |
| <span class="source-line-no">644</span><span id="line-644"> // section, we get a DroppedSnapshotException and a replay of wal</span> |
| <span class="source-line-no">645</span><span id="line-645"> // is required. Currently the only way to do this is a restart of</span> |
| <span class="source-line-no">646</span><span id="line-646"> // the server. Abort because hdfs is probably bad (HBASE-644 is a case</span> |
| <span class="source-line-no">647</span><span id="line-647"> // where hdfs was bad but passed the hdfs check).</span> |
| <span class="source-line-no">648</span><span id="line-648"> server.abort("Replay of WAL required. Forcing server shutdown", ex);</span> |
| <span class="source-line-no">649</span><span id="line-649"> return false;</span> |
| <span class="source-line-no">650</span><span id="line-650"> } catch (IOException ex) {</span> |
| <span class="source-line-no">651</span><span id="line-651"> ex = ex instanceof RemoteException ? ((RemoteException) ex).unwrapRemoteException() : ex;</span> |
| <span class="source-line-no">652</span><span id="line-652"> LOG.error("Cache flush failed" + (region != null</span> |
| <span class="source-line-no">653</span><span id="line-653"> ? (" for region " + Bytes.toStringBinary(region.getRegionInfo().getRegionName()))</span> |
| <span class="source-line-no">654</span><span id="line-654"> : ""), ex);</span> |
| <span class="source-line-no">655</span><span id="line-655"> if (!server.checkFileSystem()) {</span> |
| <span class="source-line-no">656</span><span id="line-656"> return false;</span> |
| <span class="source-line-no">657</span><span id="line-657"> }</span> |
| <span class="source-line-no">658</span><span id="line-658"> } finally {</span> |
| <span class="source-line-no">659</span><span id="line-659"> lock.readLock().unlock();</span> |
| <span class="source-line-no">660</span><span id="line-660"> wakeUpIfBlocking();</span> |
| <span class="source-line-no">661</span><span id="line-661"> tracker.afterExecution();</span> |
| <span class="source-line-no">662</span><span id="line-662"> }</span> |
| <span class="source-line-no">663</span><span id="line-663"> return true;</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"> private void notifyFlushRequest(Region region, boolean emergencyFlush) {</span> |
| <span class="source-line-no">667</span><span id="line-667"> FlushType type = null;</span> |
| <span class="source-line-no">668</span><span id="line-668"> if (emergencyFlush) {</span> |
| <span class="source-line-no">669</span><span id="line-669"> type = isAboveHighWaterMark();</span> |
| <span class="source-line-no">670</span><span id="line-670"> }</span> |
| <span class="source-line-no">671</span><span id="line-671"> if (type == null) {</span> |
| <span class="source-line-no">672</span><span id="line-672"> type = isAboveLowWaterMark();</span> |
| <span class="source-line-no">673</span><span id="line-673"> }</span> |
| <span class="source-line-no">674</span><span id="line-674"> for (FlushRequestListener listener : flushRequestListeners) {</span> |
| <span class="source-line-no">675</span><span id="line-675"> listener.flushRequested(type, region);</span> |
| <span class="source-line-no">676</span><span id="line-676"> }</span> |
| <span class="source-line-no">677</span><span id="line-677"> }</span> |
| <span class="source-line-no">678</span><span id="line-678"></span> |
| <span class="source-line-no">679</span><span id="line-679"> private void wakeUpIfBlocking() {</span> |
| <span class="source-line-no">680</span><span id="line-680"> synchronized (blockSignal) {</span> |
| <span class="source-line-no">681</span><span id="line-681"> blockSignal.notifyAll();</span> |
| <span class="source-line-no">682</span><span id="line-682"> }</span> |
| <span class="source-line-no">683</span><span id="line-683"> }</span> |
| <span class="source-line-no">684</span><span id="line-684"></span> |
| <span class="source-line-no">685</span><span id="line-685"> private boolean isTooManyStoreFiles(Region region) {</span> |
| <span class="source-line-no">686</span><span id="line-686"></span> |
| <span class="source-line-no">687</span><span id="line-687"> // When compaction is disabled, the region is flushable</span> |
| <span class="source-line-no">688</span><span id="line-688"> if (!region.getTableDescriptor().isCompactionEnabled()) {</span> |
| <span class="source-line-no">689</span><span id="line-689"> return false;</span> |
| <span class="source-line-no">690</span><span id="line-690"> }</span> |
| <span class="source-line-no">691</span><span id="line-691"></span> |
| <span class="source-line-no">692</span><span id="line-692"> for (Store store : region.getStores()) {</span> |
| <span class="source-line-no">693</span><span id="line-693"> if (store.hasTooManyStoreFiles()) {</span> |
| <span class="source-line-no">694</span><span id="line-694"> return true;</span> |
| <span class="source-line-no">695</span><span id="line-695"> }</span> |
| <span class="source-line-no">696</span><span id="line-696"> }</span> |
| <span class="source-line-no">697</span><span id="line-697"> return false;</span> |
| <span class="source-line-no">698</span><span id="line-698"> }</span> |
| <span class="source-line-no">699</span><span id="line-699"></span> |
| <span class="source-line-no">700</span><span id="line-700"> private int getStoreFileCount(Region region) {</span> |
| <span class="source-line-no">701</span><span id="line-701"> int count = 0;</span> |
| <span class="source-line-no">702</span><span id="line-702"> for (Store store : region.getStores()) {</span> |
| <span class="source-line-no">703</span><span id="line-703"> count += store.getStorefilesCount();</span> |
| <span class="source-line-no">704</span><span id="line-704"> }</span> |
| <span class="source-line-no">705</span><span id="line-705"> return count;</span> |
| <span class="source-line-no">706</span><span id="line-706"> }</span> |
| <span class="source-line-no">707</span><span id="line-707"></span> |
| <span class="source-line-no">708</span><span id="line-708"> /**</span> |
| <span class="source-line-no">709</span><span id="line-709"> * Check if the regionserver's memstore memory usage is greater than the limit. If so, flush</span> |
| <span class="source-line-no">710</span><span id="line-710"> * regions with the biggest memstores until we're down to the lower limit. This method blocks</span> |
| <span class="source-line-no">711</span><span id="line-711"> * callers until we're down to a safe amount of memstore consumption.</span> |
| <span class="source-line-no">712</span><span id="line-712"> */</span> |
| <span class="source-line-no">713</span><span id="line-713"> public void reclaimMemStoreMemory() {</span> |
| <span class="source-line-no">714</span><span id="line-714"> Span span =</span> |
| <span class="source-line-no">715</span><span id="line-715"> TraceUtil.getGlobalTracer().spanBuilder("MemStoreFluser.reclaimMemStoreMemory").startSpan();</span> |
| <span class="source-line-no">716</span><span id="line-716"> try (Scope scope = span.makeCurrent()) {</span> |
| <span class="source-line-no">717</span><span id="line-717"> FlushType flushType = isAboveHighWaterMark();</span> |
| <span class="source-line-no">718</span><span id="line-718"> if (flushType != FlushType.NORMAL) {</span> |
| <span class="source-line-no">719</span><span id="line-719"> span.addEvent("Force Flush. We're above high water mark.");</span> |
| <span class="source-line-no">720</span><span id="line-720"> long start = EnvironmentEdgeManager.currentTime();</span> |
| <span class="source-line-no">721</span><span id="line-721"> long nextLogTimeMs = start;</span> |
| <span class="source-line-no">722</span><span id="line-722"> synchronized (this.blockSignal) {</span> |
| <span class="source-line-no">723</span><span id="line-723"> boolean blocked = false;</span> |
| <span class="source-line-no">724</span><span id="line-724"> long startTime = 0;</span> |
| <span class="source-line-no">725</span><span id="line-725"> boolean interrupted = false;</span> |
| <span class="source-line-no">726</span><span id="line-726"> try {</span> |
| <span class="source-line-no">727</span><span id="line-727"> flushType = isAboveHighWaterMark();</span> |
| <span class="source-line-no">728</span><span id="line-728"> while (flushType != FlushType.NORMAL && !server.isStopped()) {</span> |
| <span class="source-line-no">729</span><span id="line-729"> if (!blocked) {</span> |
| <span class="source-line-no">730</span><span id="line-730"> startTime = EnvironmentEdgeManager.currentTime();</span> |
| <span class="source-line-no">731</span><span id="line-731"> if (!server.getRegionServerAccounting().isOffheap()) {</span> |
| <span class="source-line-no">732</span><span id="line-732"> logMsg("global memstore heapsize",</span> |
| <span class="source-line-no">733</span><span id="line-733"> server.getRegionServerAccounting().getGlobalMemStoreHeapSize(),</span> |
| <span class="source-line-no">734</span><span id="line-734"> server.getRegionServerAccounting().getGlobalMemStoreLimit());</span> |
| <span class="source-line-no">735</span><span id="line-735"> } else {</span> |
| <span class="source-line-no">736</span><span id="line-736"> switch (flushType) {</span> |
| <span class="source-line-no">737</span><span id="line-737"> case ABOVE_OFFHEAP_HIGHER_MARK:</span> |
| <span class="source-line-no">738</span><span id="line-738"> logMsg("the global offheap memstore datasize",</span> |
| <span class="source-line-no">739</span><span id="line-739"> server.getRegionServerAccounting().getGlobalMemStoreOffHeapSize(),</span> |
| <span class="source-line-no">740</span><span id="line-740"> server.getRegionServerAccounting().getGlobalMemStoreLimit());</span> |
| <span class="source-line-no">741</span><span id="line-741"> break;</span> |
| <span class="source-line-no">742</span><span id="line-742"> case ABOVE_ONHEAP_HIGHER_MARK:</span> |
| <span class="source-line-no">743</span><span id="line-743"> logMsg("global memstore heapsize",</span> |
| <span class="source-line-no">744</span><span id="line-744"> server.getRegionServerAccounting().getGlobalMemStoreHeapSize(),</span> |
| <span class="source-line-no">745</span><span id="line-745"> server.getRegionServerAccounting().getGlobalOnHeapMemStoreLimit());</span> |
| <span class="source-line-no">746</span><span id="line-746"> break;</span> |
| <span class="source-line-no">747</span><span id="line-747"> default:</span> |
| <span class="source-line-no">748</span><span id="line-748"> break;</span> |
| <span class="source-line-no">749</span><span id="line-749"> }</span> |
| <span class="source-line-no">750</span><span id="line-750"> }</span> |
| <span class="source-line-no">751</span><span id="line-751"> }</span> |
| <span class="source-line-no">752</span><span id="line-752"> blocked = true;</span> |
| <span class="source-line-no">753</span><span id="line-753"> wakeupFlushThread();</span> |
| <span class="source-line-no">754</span><span id="line-754"> try {</span> |
| <span class="source-line-no">755</span><span id="line-755"> // we should be able to wait forever, but we've seen a bug where</span> |
| <span class="source-line-no">756</span><span id="line-756"> // we miss a notify, so put a 5 second bound on it at least.</span> |
| <span class="source-line-no">757</span><span id="line-757"> blockSignal.wait(5 * 1000);</span> |
| <span class="source-line-no">758</span><span id="line-758"> } catch (InterruptedException ie) {</span> |
| <span class="source-line-no">759</span><span id="line-759"> LOG.warn("Interrupted while waiting");</span> |
| <span class="source-line-no">760</span><span id="line-760"> interrupted = true;</span> |
| <span class="source-line-no">761</span><span id="line-761"> }</span> |
| <span class="source-line-no">762</span><span id="line-762"> long nowMs = EnvironmentEdgeManager.currentTime();</span> |
| <span class="source-line-no">763</span><span id="line-763"> if (nowMs >= nextLogTimeMs) {</span> |
| <span class="source-line-no">764</span><span id="line-764"> LOG.warn("Memstore is above high water mark and block {} ms", nowMs - start);</span> |
| <span class="source-line-no">765</span><span id="line-765"> nextLogTimeMs = nowMs + 1000;</span> |
| <span class="source-line-no">766</span><span id="line-766"> }</span> |
| <span class="source-line-no">767</span><span id="line-767"> flushType = isAboveHighWaterMark();</span> |
| <span class="source-line-no">768</span><span id="line-768"> }</span> |
| <span class="source-line-no">769</span><span id="line-769"> } finally {</span> |
| <span class="source-line-no">770</span><span id="line-770"> if (interrupted) {</span> |
| <span class="source-line-no">771</span><span id="line-771"> Thread.currentThread().interrupt();</span> |
| <span class="source-line-no">772</span><span id="line-772"> }</span> |
| <span class="source-line-no">773</span><span id="line-773"> }</span> |
| <span class="source-line-no">774</span><span id="line-774"></span> |
| <span class="source-line-no">775</span><span id="line-775"> if (blocked) {</span> |
| <span class="source-line-no">776</span><span id="line-776"> final long totalTime = EnvironmentEdgeManager.currentTime() - startTime;</span> |
| <span class="source-line-no">777</span><span id="line-777"> if (totalTime > 0) {</span> |
| <span class="source-line-no">778</span><span id="line-778"> this.updatesBlockedMsHighWater.add(totalTime);</span> |
| <span class="source-line-no">779</span><span id="line-779"> }</span> |
| <span class="source-line-no">780</span><span id="line-780"> LOG.info("Unblocking updates for server " + server.toString());</span> |
| <span class="source-line-no">781</span><span id="line-781"> }</span> |
| <span class="source-line-no">782</span><span id="line-782"> }</span> |
| <span class="source-line-no">783</span><span id="line-783"> } else {</span> |
| <span class="source-line-no">784</span><span id="line-784"> flushType = isAboveLowWaterMark();</span> |
| <span class="source-line-no">785</span><span id="line-785"> if (flushType != FlushType.NORMAL) {</span> |
| <span class="source-line-no">786</span><span id="line-786"> wakeupFlushThread();</span> |
| <span class="source-line-no">787</span><span id="line-787"> }</span> |
| <span class="source-line-no">788</span><span id="line-788"> span.end();</span> |
| <span class="source-line-no">789</span><span id="line-789"> }</span> |
| <span class="source-line-no">790</span><span id="line-790"> }</span> |
| <span class="source-line-no">791</span><span id="line-791"> }</span> |
| <span class="source-line-no">792</span><span id="line-792"></span> |
| <span class="source-line-no">793</span><span id="line-793"> private void logMsg(String type, long val, long max) {</span> |
| <span class="source-line-no">794</span><span id="line-794"> LOG.info("Blocking updates: {} {} is >= blocking {}", type,</span> |
| <span class="source-line-no">795</span><span id="line-795"> TraditionalBinaryPrefix.long2String(val, "", 1),</span> |
| <span class="source-line-no">796</span><span id="line-796"> TraditionalBinaryPrefix.long2String(max, "", 1));</span> |
| <span class="source-line-no">797</span><span id="line-797"> }</span> |
| <span class="source-line-no">798</span><span id="line-798"></span> |
| <span class="source-line-no">799</span><span id="line-799"> @Override</span> |
| <span class="source-line-no">800</span><span id="line-800"> public String toString() {</span> |
| <span class="source-line-no">801</span><span id="line-801"> return "flush_queue=" + flushQueue.size();</span> |
| <span class="source-line-no">802</span><span id="line-802"> }</span> |
| <span class="source-line-no">803</span><span id="line-803"></span> |
| <span class="source-line-no">804</span><span id="line-804"> public String dumpQueue() {</span> |
| <span class="source-line-no">805</span><span id="line-805"> StringBuilder queueList = new StringBuilder();</span> |
| <span class="source-line-no">806</span><span id="line-806"> queueList.append("Flush Queue Queue dump:\n");</span> |
| <span class="source-line-no">807</span><span id="line-807"> queueList.append(" Flush Queue:\n");</span> |
| <span class="source-line-no">808</span><span id="line-808"> java.util.Iterator<FlushQueueEntry> it = flushQueue.iterator();</span> |
| <span class="source-line-no">809</span><span id="line-809"></span> |
| <span class="source-line-no">810</span><span id="line-810"> while (it.hasNext()) {</span> |
| <span class="source-line-no">811</span><span id="line-811"> queueList.append(" " + it.next().toString());</span> |
| <span class="source-line-no">812</span><span id="line-812"> queueList.append("\n");</span> |
| <span class="source-line-no">813</span><span id="line-813"> }</span> |
| <span class="source-line-no">814</span><span id="line-814"></span> |
| <span class="source-line-no">815</span><span id="line-815"> return queueList.toString();</span> |
| <span class="source-line-no">816</span><span id="line-816"> }</span> |
| <span class="source-line-no">817</span><span id="line-817"></span> |
| <span class="source-line-no">818</span><span id="line-818"> /**</span> |
| <span class="source-line-no">819</span><span id="line-819"> * Register a MemstoreFlushListener</span> |
| <span class="source-line-no">820</span><span id="line-820"> */</span> |
| <span class="source-line-no">821</span><span id="line-821"> @Override</span> |
| <span class="source-line-no">822</span><span id="line-822"> public void registerFlushRequestListener(final FlushRequestListener listener) {</span> |
| <span class="source-line-no">823</span><span id="line-823"> this.flushRequestListeners.add(listener);</span> |
| <span class="source-line-no">824</span><span id="line-824"> }</span> |
| <span class="source-line-no">825</span><span id="line-825"></span> |
| <span class="source-line-no">826</span><span id="line-826"> /**</span> |
| <span class="source-line-no">827</span><span id="line-827"> * Unregister the listener from MemstoreFlushListeners</span> |
| <span class="source-line-no">828</span><span id="line-828"> * @return true when passed listener is unregistered successfully.</span> |
| <span class="source-line-no">829</span><span id="line-829"> */</span> |
| <span class="source-line-no">830</span><span id="line-830"> @Override</span> |
| <span class="source-line-no">831</span><span id="line-831"> public boolean unregisterFlushRequestListener(final FlushRequestListener listener) {</span> |
| <span class="source-line-no">832</span><span id="line-832"> return this.flushRequestListeners.remove(listener);</span> |
| <span class="source-line-no">833</span><span id="line-833"> }</span> |
| <span class="source-line-no">834</span><span id="line-834"></span> |
| <span class="source-line-no">835</span><span id="line-835"> /**</span> |
| <span class="source-line-no">836</span><span id="line-836"> * Sets the global memstore limit to a new size.</span> |
| <span class="source-line-no">837</span><span id="line-837"> */</span> |
| <span class="source-line-no">838</span><span id="line-838"> @Override</span> |
| <span class="source-line-no">839</span><span id="line-839"> public void setGlobalMemStoreLimit(long globalMemStoreSize) {</span> |
| <span class="source-line-no">840</span><span id="line-840"> this.server.getRegionServerAccounting().setGlobalMemStoreLimits(globalMemStoreSize);</span> |
| <span class="source-line-no">841</span><span id="line-841"> reclaimMemStoreMemory();</span> |
| <span class="source-line-no">842</span><span id="line-842"> }</span> |
| <span class="source-line-no">843</span><span id="line-843"></span> |
| <span class="source-line-no">844</span><span id="line-844"> interface FlushQueueEntry extends Delayed {</span> |
| <span class="source-line-no">845</span><span id="line-845"> }</span> |
| <span class="source-line-no">846</span><span id="line-846"></span> |
| <span class="source-line-no">847</span><span id="line-847"> /**</span> |
| <span class="source-line-no">848</span><span id="line-848"> * Datastructure used in the flush queue. Holds region and retry count. Keeps tabs on how old this</span> |
| <span class="source-line-no">849</span><span id="line-849"> * object is. Implements {@link Delayed}. On construction, the delay is zero. When added to a</span> |
| <span class="source-line-no">850</span><span id="line-850"> * delay queue, we'll come out near immediately. Call {@link #requeue(long)} passing delay in</span> |
| <span class="source-line-no">851</span><span id="line-851"> * milliseconds before readding to delay queue if you want it to stay there a while.</span> |
| <span class="source-line-no">852</span><span id="line-852"> */</span> |
| <span class="source-line-no">853</span><span id="line-853"> static class FlushRegionEntry implements FlushQueueEntry {</span> |
| <span class="source-line-no">854</span><span id="line-854"> private final HRegion region;</span> |
| <span class="source-line-no">855</span><span id="line-855"></span> |
| <span class="source-line-no">856</span><span id="line-856"> private final long createTime;</span> |
| <span class="source-line-no">857</span><span id="line-857"> private long whenToExpire;</span> |
| <span class="source-line-no">858</span><span id="line-858"> private int requeueCount = 0;</span> |
| <span class="source-line-no">859</span><span id="line-859"></span> |
| <span class="source-line-no">860</span><span id="line-860"> private final List<byte[]> families;</span> |
| <span class="source-line-no">861</span><span id="line-861"></span> |
| <span class="source-line-no">862</span><span id="line-862"> private final FlushLifeCycleTracker tracker;</span> |
| <span class="source-line-no">863</span><span id="line-863"></span> |
| <span class="source-line-no">864</span><span id="line-864"> FlushRegionEntry(final HRegion r, List<byte[]> families, FlushLifeCycleTracker tracker) {</span> |
| <span class="source-line-no">865</span><span id="line-865"> this.region = r;</span> |
| <span class="source-line-no">866</span><span id="line-866"> this.createTime = EnvironmentEdgeManager.currentTime();</span> |
| <span class="source-line-no">867</span><span id="line-867"> this.whenToExpire = this.createTime;</span> |
| <span class="source-line-no">868</span><span id="line-868"> this.families = families;</span> |
| <span class="source-line-no">869</span><span id="line-869"> this.tracker = tracker;</span> |
| <span class="source-line-no">870</span><span id="line-870"> }</span> |
| <span class="source-line-no">871</span><span id="line-871"></span> |
| <span class="source-line-no">872</span><span id="line-872"> /** Returns True if we have been delayed > <code>maximumWait</code> milliseconds. */</span> |
| <span class="source-line-no">873</span><span id="line-873"> public boolean isMaximumWait(final long maximumWait) {</span> |
| <span class="source-line-no">874</span><span id="line-874"> return (EnvironmentEdgeManager.currentTime() - this.createTime) > maximumWait;</span> |
| <span class="source-line-no">875</span><span id="line-875"> }</span> |
| <span class="source-line-no">876</span><span id="line-876"></span> |
| <span class="source-line-no">877</span><span id="line-877"> /** Returns True if the entry is a delay flush task */</span> |
| <span class="source-line-no">878</span><span id="line-878"> protected boolean isDelay() {</span> |
| <span class="source-line-no">879</span><span id="line-879"> return this.whenToExpire > this.createTime;</span> |
| <span class="source-line-no">880</span><span id="line-880"> }</span> |
| <span class="source-line-no">881</span><span id="line-881"></span> |
| <span class="source-line-no">882</span><span id="line-882"> /**</span> |
| <span class="source-line-no">883</span><span id="line-883"> * @return Count of times {@link #requeue(long)} was called; i.e this is number of times we've</span> |
| <span class="source-line-no">884</span><span id="line-884"> * been requeued.</span> |
| <span class="source-line-no">885</span><span id="line-885"> */</span> |
| <span class="source-line-no">886</span><span id="line-886"> public int getRequeueCount() {</span> |
| <span class="source-line-no">887</span><span id="line-887"> return this.requeueCount;</span> |
| <span class="source-line-no">888</span><span id="line-888"> }</span> |
| <span class="source-line-no">889</span><span id="line-889"></span> |
| <span class="source-line-no">890</span><span id="line-890"> public FlushLifeCycleTracker getTracker() {</span> |
| <span class="source-line-no">891</span><span id="line-891"> return tracker;</span> |
| <span class="source-line-no">892</span><span id="line-892"> }</span> |
| <span class="source-line-no">893</span><span id="line-893"></span> |
| <span class="source-line-no">894</span><span id="line-894"> /**</span> |
| <span class="source-line-no">895</span><span id="line-895"> * @param when When to expire, when to come up out of the queue. Specify in milliseconds. This</span> |
| <span class="source-line-no">896</span><span id="line-896"> * method adds EnvironmentEdgeManager.currentTime() to whatever you pass.</span> |
| <span class="source-line-no">897</span><span id="line-897"> * @return This.</span> |
| <span class="source-line-no">898</span><span id="line-898"> */</span> |
| <span class="source-line-no">899</span><span id="line-899"> public FlushRegionEntry requeue(final long when) {</span> |
| <span class="source-line-no">900</span><span id="line-900"> this.whenToExpire = EnvironmentEdgeManager.currentTime() + when;</span> |
| <span class="source-line-no">901</span><span id="line-901"> this.requeueCount++;</span> |
| <span class="source-line-no">902</span><span id="line-902"> return this;</span> |
| <span class="source-line-no">903</span><span id="line-903"> }</span> |
| <span class="source-line-no">904</span><span id="line-904"></span> |
| <span class="source-line-no">905</span><span id="line-905"> @Override</span> |
| <span class="source-line-no">906</span><span id="line-906"> public long getDelay(TimeUnit unit) {</span> |
| <span class="source-line-no">907</span><span id="line-907"> return unit.convert(this.whenToExpire - EnvironmentEdgeManager.currentTime(),</span> |
| <span class="source-line-no">908</span><span id="line-908"> TimeUnit.MILLISECONDS);</span> |
| <span class="source-line-no">909</span><span id="line-909"> }</span> |
| <span class="source-line-no">910</span><span id="line-910"></span> |
| <span class="source-line-no">911</span><span id="line-911"> @Override</span> |
| <span class="source-line-no">912</span><span id="line-912"> public int compareTo(Delayed other) {</span> |
| <span class="source-line-no">913</span><span id="line-913"> // Delay is compared first. If there is a tie, compare region's hash code</span> |
| <span class="source-line-no">914</span><span id="line-914"> int ret =</span> |
| <span class="source-line-no">915</span><span id="line-915"> Long.valueOf(getDelay(TimeUnit.MILLISECONDS) - other.getDelay(TimeUnit.MILLISECONDS))</span> |
| <span class="source-line-no">916</span><span id="line-916"> .intValue();</span> |
| <span class="source-line-no">917</span><span id="line-917"> if (ret != 0) {</span> |
| <span class="source-line-no">918</span><span id="line-918"> return ret;</span> |
| <span class="source-line-no">919</span><span id="line-919"> }</span> |
| <span class="source-line-no">920</span><span id="line-920"> FlushQueueEntry otherEntry = (FlushQueueEntry) other;</span> |
| <span class="source-line-no">921</span><span id="line-921"> return hashCode() - otherEntry.hashCode();</span> |
| <span class="source-line-no">922</span><span id="line-922"> }</span> |
| <span class="source-line-no">923</span><span id="line-923"></span> |
| <span class="source-line-no">924</span><span id="line-924"> @Override</span> |
| <span class="source-line-no">925</span><span id="line-925"> public String toString() {</span> |
| <span class="source-line-no">926</span><span id="line-926"> return "[flush region " + Bytes.toStringBinary(region.getRegionInfo().getRegionName()) + "]";</span> |
| <span class="source-line-no">927</span><span id="line-927"> }</span> |
| <span class="source-line-no">928</span><span id="line-928"></span> |
| <span class="source-line-no">929</span><span id="line-929"> @Override</span> |
| <span class="source-line-no">930</span><span id="line-930"> public int hashCode() {</span> |
| <span class="source-line-no">931</span><span id="line-931"> int hash = (int) getDelay(TimeUnit.MILLISECONDS);</span> |
| <span class="source-line-no">932</span><span id="line-932"> return hash ^ region.hashCode();</span> |
| <span class="source-line-no">933</span><span id="line-933"> }</span> |
| <span class="source-line-no">934</span><span id="line-934"></span> |
| <span class="source-line-no">935</span><span id="line-935"> @Override</span> |
| <span class="source-line-no">936</span><span id="line-936"> public boolean equals(Object obj) {</span> |
| <span class="source-line-no">937</span><span id="line-937"> if (this == obj) {</span> |
| <span class="source-line-no">938</span><span id="line-938"> return true;</span> |
| <span class="source-line-no">939</span><span id="line-939"> }</span> |
| <span class="source-line-no">940</span><span id="line-940"> if (obj == null || getClass() != obj.getClass()) {</span> |
| <span class="source-line-no">941</span><span id="line-941"> return false;</span> |
| <span class="source-line-no">942</span><span id="line-942"> }</span> |
| <span class="source-line-no">943</span><span id="line-943"> FlushRegionEntry other = (FlushRegionEntry) obj;</span> |
| <span class="source-line-no">944</span><span id="line-944"> if (</span> |
| <span class="source-line-no">945</span><span id="line-945"> !Bytes.equals(this.region.getRegionInfo().getRegionName(),</span> |
| <span class="source-line-no">946</span><span id="line-946"> other.region.getRegionInfo().getRegionName())</span> |
| <span class="source-line-no">947</span><span id="line-947"> ) {</span> |
| <span class="source-line-no">948</span><span id="line-948"> return false;</span> |
| <span class="source-line-no">949</span><span id="line-949"> }</span> |
| <span class="source-line-no">950</span><span id="line-950"> return compareTo(other) == 0;</span> |
| <span class="source-line-no">951</span><span id="line-951"> }</span> |
| <span class="source-line-no">952</span><span id="line-952"> }</span> |
| <span class="source-line-no">953</span><span id="line-953"></span> |
| <span class="source-line-no">954</span><span id="line-954"> private int getHandlerCount(Configuration conf) {</span> |
| <span class="source-line-no">955</span><span id="line-955"> int handlerCount = conf.getInt("hbase.hstore.flusher.count", 2);</span> |
| <span class="source-line-no">956</span><span id="line-956"> if (handlerCount < 1) {</span> |
| <span class="source-line-no">957</span><span id="line-957"> LOG.warn(</span> |
| <span class="source-line-no">958</span><span id="line-958"> "hbase.hstore.flusher.count was configed to {} which is less than 1, " + "corrected to 1",</span> |
| <span class="source-line-no">959</span><span id="line-959"> handlerCount);</span> |
| <span class="source-line-no">960</span><span id="line-960"> handlerCount = 1;</span> |
| <span class="source-line-no">961</span><span id="line-961"> }</span> |
| <span class="source-line-no">962</span><span id="line-962"> return handlerCount;</span> |
| <span class="source-line-no">963</span><span id="line-963"> }</span> |
| <span class="source-line-no">964</span><span id="line-964"></span> |
| <span class="source-line-no">965</span><span id="line-965"> @Override</span> |
| <span class="source-line-no">966</span><span id="line-966"> public void onConfigurationChange(Configuration newConf) {</span> |
| <span class="source-line-no">967</span><span id="line-967"> int newHandlerCount = getHandlerCount(newConf);</span> |
| <span class="source-line-no">968</span><span id="line-968"> if (newHandlerCount != flushHandlers.length) {</span> |
| <span class="source-line-no">969</span><span id="line-969"> LOG.info("update hbase.hstore.flusher.count from {} to {}", flushHandlers.length,</span> |
| <span class="source-line-no">970</span><span id="line-970"> newHandlerCount);</span> |
| <span class="source-line-no">971</span><span id="line-971"> lock.writeLock().lock();</span> |
| <span class="source-line-no">972</span><span id="line-972"> try {</span> |
| <span class="source-line-no">973</span><span id="line-973"> FlushHandler[] newFlushHandlers = Arrays.copyOf(flushHandlers, newHandlerCount);</span> |
| <span class="source-line-no">974</span><span id="line-974"> if (newHandlerCount > flushHandlers.length) {</span> |
| <span class="source-line-no">975</span><span id="line-975"> startFlushHandlerThreads(newFlushHandlers, flushHandlers.length, newFlushHandlers.length);</span> |
| <span class="source-line-no">976</span><span id="line-976"> } else {</span> |
| <span class="source-line-no">977</span><span id="line-977"> stopFlushHandlerThreads(flushHandlers, newHandlerCount, flushHandlers.length);</span> |
| <span class="source-line-no">978</span><span id="line-978"> }</span> |
| <span class="source-line-no">979</span><span id="line-979"> flusherIdGen.compareAndSet(flushHandlers.length, newFlushHandlers.length);</span> |
| <span class="source-line-no">980</span><span id="line-980"> this.flushHandlers = newFlushHandlers;</span> |
| <span class="source-line-no">981</span><span id="line-981"> } finally {</span> |
| <span class="source-line-no">982</span><span id="line-982"> lock.writeLock().unlock();</span> |
| <span class="source-line-no">983</span><span id="line-983"> }</span> |
| <span class="source-line-no">984</span><span id="line-984"> }</span> |
| <span class="source-line-no">985</span><span id="line-985"> }</span> |
| <span class="source-line-no">986</span><span id="line-986"></span> |
| <span class="source-line-no">987</span><span id="line-987"> private void startFlushHandlerThreads(FlushHandler[] flushHandlers, int start, int end) {</span> |
| <span class="source-line-no">988</span><span id="line-988"> if (flusherThreadFactory != null) {</span> |
| <span class="source-line-no">989</span><span id="line-989"> for (int i = start; i < end; i++) {</span> |
| <span class="source-line-no">990</span><span id="line-990"> flushHandlers[i] = new FlushHandler("MemStoreFlusher." + flusherIdGen.getAndIncrement());</span> |
| <span class="source-line-no">991</span><span id="line-991"> flusherThreadFactory.newThread(flushHandlers[i]);</span> |
| <span class="source-line-no">992</span><span id="line-992"> flushHandlers[i].start();</span> |
| <span class="source-line-no">993</span><span id="line-993"> }</span> |
| <span class="source-line-no">994</span><span id="line-994"> }</span> |
| <span class="source-line-no">995</span><span id="line-995"> }</span> |
| <span class="source-line-no">996</span><span id="line-996"></span> |
| <span class="source-line-no">997</span><span id="line-997"> private void stopFlushHandlerThreads(FlushHandler[] flushHandlers, int start, int end) {</span> |
| <span class="source-line-no">998</span><span id="line-998"> for (int i = start; i < end; i++) {</span> |
| <span class="source-line-no">999</span><span id="line-999"> flushHandlers[i].shutdown();</span> |
| <span class="source-line-no">1000</span><span id="line-1000"> if (LOG.isDebugEnabled()) {</span> |
| <span class="source-line-no">1001</span><span id="line-1001"> LOG.debug("send shutdown signal to {}", flushHandlers[i].getName());</span> |
| <span class="source-line-no">1002</span><span id="line-1002"> }</span> |
| <span class="source-line-no">1003</span><span id="line-1003"> }</span> |
| <span class="source-line-no">1004</span><span id="line-1004"> }</span> |
| <span class="source-line-no">1005</span><span id="line-1005"></span> |
| <span class="source-line-no">1006</span><span id="line-1006"> public int getFlusherCount() {</span> |
| <span class="source-line-no">1007</span><span id="line-1007"> return flusherIdGen.get();</span> |
| <span class="source-line-no">1008</span><span id="line-1008"> }</span> |
| <span class="source-line-no">1009</span><span id="line-1009">}</span> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </pre> |
| </div> |
| </main> |
| </body> |
| </html> |