blob: beb9a5fa155ff59b203dd120802bfaecdd9beb9b [file] [log] [blame]
<!DOCTYPE HTML>
<html lang="en">
<head>
<!-- Generated by javadoc (17) -->
<title>Source code</title>
<meta name="viewport" content="width=device-width, initial-scale=1">
<meta name="description" content="source: package: org.apache.hadoop.hbase.regionserver, class: TestHStore, class: FaultyOutputStream">
<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 static org.apache.hadoop.hbase.io.hfile.CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY;</span>
<span class="source-line-no">021</span><span id="line-21">import static org.apache.hadoop.hbase.io.hfile.CacheConfig.CACHE_DATA_ON_READ_KEY;</span>
<span class="source-line-no">022</span><span id="line-22">import static org.apache.hadoop.hbase.io.hfile.CacheConfig.DEFAULT_CACHE_DATA_ON_READ;</span>
<span class="source-line-no">023</span><span id="line-23">import static org.apache.hadoop.hbase.io.hfile.CacheConfig.DEFAULT_CACHE_DATA_ON_WRITE;</span>
<span class="source-line-no">024</span><span id="line-24">import static org.apache.hadoop.hbase.io.hfile.CacheConfig.DEFAULT_EVICT_ON_CLOSE;</span>
<span class="source-line-no">025</span><span id="line-25">import static org.apache.hadoop.hbase.io.hfile.CacheConfig.EVICT_BLOCKS_ON_CLOSE_KEY;</span>
<span class="source-line-no">026</span><span id="line-26">import static org.apache.hadoop.hbase.regionserver.DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY;</span>
<span class="source-line-no">027</span><span id="line-27">import static org.junit.Assert.assertArrayEquals;</span>
<span class="source-line-no">028</span><span id="line-28">import static org.junit.Assert.assertEquals;</span>
<span class="source-line-no">029</span><span id="line-29">import static org.junit.Assert.assertFalse;</span>
<span class="source-line-no">030</span><span id="line-30">import static org.junit.Assert.assertNull;</span>
<span class="source-line-no">031</span><span id="line-31">import static org.junit.Assert.assertTrue;</span>
<span class="source-line-no">032</span><span id="line-32">import static org.junit.Assert.fail;</span>
<span class="source-line-no">033</span><span id="line-33">import static org.mockito.ArgumentMatchers.any;</span>
<span class="source-line-no">034</span><span id="line-34">import static org.mockito.Mockito.mock;</span>
<span class="source-line-no">035</span><span id="line-35">import static org.mockito.Mockito.spy;</span>
<span class="source-line-no">036</span><span id="line-36">import static org.mockito.Mockito.times;</span>
<span class="source-line-no">037</span><span id="line-37">import static org.mockito.Mockito.verify;</span>
<span class="source-line-no">038</span><span id="line-38">import static org.mockito.Mockito.when;</span>
<span class="source-line-no">039</span><span id="line-39"></span>
<span class="source-line-no">040</span><span id="line-40">import java.io.FileNotFoundException;</span>
<span class="source-line-no">041</span><span id="line-41">import java.io.IOException;</span>
<span class="source-line-no">042</span><span id="line-42">import java.lang.ref.SoftReference;</span>
<span class="source-line-no">043</span><span id="line-43">import java.security.PrivilegedExceptionAction;</span>
<span class="source-line-no">044</span><span id="line-44">import java.util.ArrayList;</span>
<span class="source-line-no">045</span><span id="line-45">import java.util.Arrays;</span>
<span class="source-line-no">046</span><span id="line-46">import java.util.Collection;</span>
<span class="source-line-no">047</span><span id="line-47">import java.util.Collections;</span>
<span class="source-line-no">048</span><span id="line-48">import java.util.Iterator;</span>
<span class="source-line-no">049</span><span id="line-49">import java.util.List;</span>
<span class="source-line-no">050</span><span id="line-50">import java.util.ListIterator;</span>
<span class="source-line-no">051</span><span id="line-51">import java.util.NavigableSet;</span>
<span class="source-line-no">052</span><span id="line-52">import java.util.Optional;</span>
<span class="source-line-no">053</span><span id="line-53">import java.util.TreeSet;</span>
<span class="source-line-no">054</span><span id="line-54">import java.util.concurrent.BrokenBarrierException;</span>
<span class="source-line-no">055</span><span id="line-55">import java.util.concurrent.ConcurrentSkipListSet;</span>
<span class="source-line-no">056</span><span id="line-56">import java.util.concurrent.CountDownLatch;</span>
<span class="source-line-no">057</span><span id="line-57">import java.util.concurrent.CyclicBarrier;</span>
<span class="source-line-no">058</span><span id="line-58">import java.util.concurrent.ExecutorService;</span>
<span class="source-line-no">059</span><span id="line-59">import java.util.concurrent.Executors;</span>
<span class="source-line-no">060</span><span id="line-60">import java.util.concurrent.Future;</span>
<span class="source-line-no">061</span><span id="line-61">import java.util.concurrent.ThreadPoolExecutor;</span>
<span class="source-line-no">062</span><span id="line-62">import java.util.concurrent.TimeUnit;</span>
<span class="source-line-no">063</span><span id="line-63">import java.util.concurrent.atomic.AtomicBoolean;</span>
<span class="source-line-no">064</span><span id="line-64">import java.util.concurrent.atomic.AtomicInteger;</span>
<span class="source-line-no">065</span><span id="line-65">import java.util.concurrent.atomic.AtomicLong;</span>
<span class="source-line-no">066</span><span id="line-66">import java.util.concurrent.atomic.AtomicReference;</span>
<span class="source-line-no">067</span><span id="line-67">import java.util.concurrent.locks.ReentrantReadWriteLock;</span>
<span class="source-line-no">068</span><span id="line-68">import java.util.function.Consumer;</span>
<span class="source-line-no">069</span><span id="line-69">import java.util.function.IntBinaryOperator;</span>
<span class="source-line-no">070</span><span id="line-70">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">071</span><span id="line-71">import org.apache.hadoop.fs.FSDataOutputStream;</span>
<span class="source-line-no">072</span><span id="line-72">import org.apache.hadoop.fs.FileStatus;</span>
<span class="source-line-no">073</span><span id="line-73">import org.apache.hadoop.fs.FileSystem;</span>
<span class="source-line-no">074</span><span id="line-74">import org.apache.hadoop.fs.FilterFileSystem;</span>
<span class="source-line-no">075</span><span id="line-75">import org.apache.hadoop.fs.LocalFileSystem;</span>
<span class="source-line-no">076</span><span id="line-76">import org.apache.hadoop.fs.Path;</span>
<span class="source-line-no">077</span><span id="line-77">import org.apache.hadoop.fs.permission.FsPermission;</span>
<span class="source-line-no">078</span><span id="line-78">import org.apache.hadoop.hbase.Cell;</span>
<span class="source-line-no">079</span><span id="line-79">import org.apache.hadoop.hbase.CellBuilderType;</span>
<span class="source-line-no">080</span><span id="line-80">import org.apache.hadoop.hbase.CellComparator;</span>
<span class="source-line-no">081</span><span id="line-81">import org.apache.hadoop.hbase.CellComparatorImpl;</span>
<span class="source-line-no">082</span><span id="line-82">import org.apache.hadoop.hbase.CellUtil;</span>
<span class="source-line-no">083</span><span id="line-83">import org.apache.hadoop.hbase.ExtendedCell;</span>
<span class="source-line-no">084</span><span id="line-84">import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;</span>
<span class="source-line-no">085</span><span id="line-85">import org.apache.hadoop.hbase.HBaseClassTestRule;</span>
<span class="source-line-no">086</span><span id="line-86">import org.apache.hadoop.hbase.HBaseConfiguration;</span>
<span class="source-line-no">087</span><span id="line-87">import org.apache.hadoop.hbase.HBaseTestingUtil;</span>
<span class="source-line-no">088</span><span id="line-88">import org.apache.hadoop.hbase.HConstants;</span>
<span class="source-line-no">089</span><span id="line-89">import org.apache.hadoop.hbase.KeyValue;</span>
<span class="source-line-no">090</span><span id="line-90">import org.apache.hadoop.hbase.MemoryCompactionPolicy;</span>
<span class="source-line-no">091</span><span id="line-91">import org.apache.hadoop.hbase.NamespaceDescriptor;</span>
<span class="source-line-no">092</span><span id="line-92">import org.apache.hadoop.hbase.PrivateCellUtil;</span>
<span class="source-line-no">093</span><span id="line-93">import org.apache.hadoop.hbase.TableName;</span>
<span class="source-line-no">094</span><span id="line-94">import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;</span>
<span class="source-line-no">095</span><span id="line-95">import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;</span>
<span class="source-line-no">096</span><span id="line-96">import org.apache.hadoop.hbase.client.Get;</span>
<span class="source-line-no">097</span><span id="line-97">import org.apache.hadoop.hbase.client.RegionInfo;</span>
<span class="source-line-no">098</span><span id="line-98">import org.apache.hadoop.hbase.client.RegionInfoBuilder;</span>
<span class="source-line-no">099</span><span id="line-99">import org.apache.hadoop.hbase.client.Scan;</span>
<span class="source-line-no">100</span><span id="line-100">import org.apache.hadoop.hbase.client.Scan.ReadType;</span>
<span class="source-line-no">101</span><span id="line-101">import org.apache.hadoop.hbase.client.TableDescriptor;</span>
<span class="source-line-no">102</span><span id="line-102">import org.apache.hadoop.hbase.client.TableDescriptorBuilder;</span>
<span class="source-line-no">103</span><span id="line-103">import org.apache.hadoop.hbase.exceptions.IllegalArgumentIOException;</span>
<span class="source-line-no">104</span><span id="line-104">import org.apache.hadoop.hbase.filter.Filter;</span>
<span class="source-line-no">105</span><span id="line-105">import org.apache.hadoop.hbase.filter.FilterBase;</span>
<span class="source-line-no">106</span><span id="line-106">import org.apache.hadoop.hbase.io.compress.Compression;</span>
<span class="source-line-no">107</span><span id="line-107">import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;</span>
<span class="source-line-no">108</span><span id="line-108">import org.apache.hadoop.hbase.io.hfile.CacheConfig;</span>
<span class="source-line-no">109</span><span id="line-109">import org.apache.hadoop.hbase.io.hfile.HFile;</span>
<span class="source-line-no">110</span><span id="line-110">import org.apache.hadoop.hbase.io.hfile.HFileContext;</span>
<span class="source-line-no">111</span><span id="line-111">import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;</span>
<span class="source-line-no">112</span><span id="line-112">import org.apache.hadoop.hbase.monitoring.MonitoredTask;</span>
<span class="source-line-no">113</span><span id="line-113">import org.apache.hadoop.hbase.nio.RefCnt;</span>
<span class="source-line-no">114</span><span id="line-114">import org.apache.hadoop.hbase.quotas.RegionSizeStoreImpl;</span>
<span class="source-line-no">115</span><span id="line-115">import org.apache.hadoop.hbase.regionserver.ChunkCreator.ChunkType;</span>
<span class="source-line-no">116</span><span id="line-116">import org.apache.hadoop.hbase.regionserver.MemStoreCompactionStrategy.Action;</span>
<span class="source-line-no">117</span><span id="line-117">import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;</span>
<span class="source-line-no">118</span><span id="line-118">import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;</span>
<span class="source-line-no">119</span><span id="line-119">import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;</span>
<span class="source-line-no">120</span><span id="line-120">import org.apache.hadoop.hbase.regionserver.compactions.EverythingPolicy;</span>
<span class="source-line-no">121</span><span id="line-121">import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;</span>
<span class="source-line-no">122</span><span id="line-122">import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;</span>
<span class="source-line-no">123</span><span id="line-123">import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;</span>
<span class="source-line-no">124</span><span id="line-124">import org.apache.hadoop.hbase.security.User;</span>
<span class="source-line-no">125</span><span id="line-125">import org.apache.hadoop.hbase.testclassification.MediumTests;</span>
<span class="source-line-no">126</span><span id="line-126">import org.apache.hadoop.hbase.testclassification.RegionServerTests;</span>
<span class="source-line-no">127</span><span id="line-127">import org.apache.hadoop.hbase.util.BloomFilterUtil;</span>
<span class="source-line-no">128</span><span id="line-128">import org.apache.hadoop.hbase.util.Bytes;</span>
<span class="source-line-no">129</span><span id="line-129">import org.apache.hadoop.hbase.util.CommonFSUtils;</span>
<span class="source-line-no">130</span><span id="line-130">import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;</span>
<span class="source-line-no">131</span><span id="line-131">import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;</span>
<span class="source-line-no">132</span><span id="line-132">import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;</span>
<span class="source-line-no">133</span><span id="line-133">import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;</span>
<span class="source-line-no">134</span><span id="line-134">import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;</span>
<span class="source-line-no">135</span><span id="line-135">import org.apache.hadoop.hbase.wal.WALFactory;</span>
<span class="source-line-no">136</span><span id="line-136">import org.apache.hadoop.util.Progressable;</span>
<span class="source-line-no">137</span><span id="line-137">import org.junit.After;</span>
<span class="source-line-no">138</span><span id="line-138">import org.junit.AfterClass;</span>
<span class="source-line-no">139</span><span id="line-139">import org.junit.Before;</span>
<span class="source-line-no">140</span><span id="line-140">import org.junit.ClassRule;</span>
<span class="source-line-no">141</span><span id="line-141">import org.junit.Rule;</span>
<span class="source-line-no">142</span><span id="line-142">import org.junit.Test;</span>
<span class="source-line-no">143</span><span id="line-143">import org.junit.experimental.categories.Category;</span>
<span class="source-line-no">144</span><span id="line-144">import org.junit.rules.TestName;</span>
<span class="source-line-no">145</span><span id="line-145">import org.mockito.Mockito;</span>
<span class="source-line-no">146</span><span id="line-146">import org.slf4j.Logger;</span>
<span class="source-line-no">147</span><span id="line-147">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">148</span><span id="line-148"></span>
<span class="source-line-no">149</span><span id="line-149">import org.apache.hbase.thirdparty.com.google.common.collect.Lists;</span>
<span class="source-line-no">150</span><span id="line-150"></span>
<span class="source-line-no">151</span><span id="line-151">/**</span>
<span class="source-line-no">152</span><span id="line-152"> * Test class for the HStore</span>
<span class="source-line-no">153</span><span id="line-153"> */</span>
<span class="source-line-no">154</span><span id="line-154">@Category({ RegionServerTests.class, MediumTests.class })</span>
<span class="source-line-no">155</span><span id="line-155">public class TestHStore {</span>
<span class="source-line-no">156</span><span id="line-156"></span>
<span class="source-line-no">157</span><span id="line-157"> @ClassRule</span>
<span class="source-line-no">158</span><span id="line-158"> public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule.forClass(TestHStore.class);</span>
<span class="source-line-no">159</span><span id="line-159"></span>
<span class="source-line-no">160</span><span id="line-160"> private static final Logger LOG = LoggerFactory.getLogger(TestHStore.class);</span>
<span class="source-line-no">161</span><span id="line-161"> @Rule</span>
<span class="source-line-no">162</span><span id="line-162"> public TestName name = new TestName();</span>
<span class="source-line-no">163</span><span id="line-163"></span>
<span class="source-line-no">164</span><span id="line-164"> HRegion region;</span>
<span class="source-line-no">165</span><span id="line-165"> HStore store;</span>
<span class="source-line-no">166</span><span id="line-166"> byte[] table = Bytes.toBytes("table");</span>
<span class="source-line-no">167</span><span id="line-167"> byte[] family = Bytes.toBytes("family");</span>
<span class="source-line-no">168</span><span id="line-168"></span>
<span class="source-line-no">169</span><span id="line-169"> byte[] row = Bytes.toBytes("row");</span>
<span class="source-line-no">170</span><span id="line-170"> byte[] row2 = Bytes.toBytes("row2");</span>
<span class="source-line-no">171</span><span id="line-171"> byte[] qf1 = Bytes.toBytes("qf1");</span>
<span class="source-line-no">172</span><span id="line-172"> byte[] qf2 = Bytes.toBytes("qf2");</span>
<span class="source-line-no">173</span><span id="line-173"> byte[] qf3 = Bytes.toBytes("qf3");</span>
<span class="source-line-no">174</span><span id="line-174"> byte[] qf4 = Bytes.toBytes("qf4");</span>
<span class="source-line-no">175</span><span id="line-175"> byte[] qf5 = Bytes.toBytes("qf5");</span>
<span class="source-line-no">176</span><span id="line-176"> byte[] qf6 = Bytes.toBytes("qf6");</span>
<span class="source-line-no">177</span><span id="line-177"></span>
<span class="source-line-no">178</span><span id="line-178"> NavigableSet&lt;byte[]&gt; qualifiers = new ConcurrentSkipListSet&lt;&gt;(Bytes.BYTES_COMPARATOR);</span>
<span class="source-line-no">179</span><span id="line-179"></span>
<span class="source-line-no">180</span><span id="line-180"> List&lt;Cell&gt; expected = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">181</span><span id="line-181"> List&lt;Cell&gt; result = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">182</span><span id="line-182"></span>
<span class="source-line-no">183</span><span id="line-183"> long id = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">184</span><span id="line-184"> Get get = new Get(row);</span>
<span class="source-line-no">185</span><span id="line-185"></span>
<span class="source-line-no">186</span><span id="line-186"> private static final HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();</span>
<span class="source-line-no">187</span><span id="line-187"> private static final String DIR = TEST_UTIL.getDataTestDir("TestStore").toString();</span>
<span class="source-line-no">188</span><span id="line-188"></span>
<span class="source-line-no">189</span><span id="line-189"> @Before</span>
<span class="source-line-no">190</span><span id="line-190"> public void setUp() throws IOException {</span>
<span class="source-line-no">191</span><span id="line-191"> qualifiers.clear();</span>
<span class="source-line-no">192</span><span id="line-192"> qualifiers.add(qf1);</span>
<span class="source-line-no">193</span><span id="line-193"> qualifiers.add(qf3);</span>
<span class="source-line-no">194</span><span id="line-194"> qualifiers.add(qf5);</span>
<span class="source-line-no">195</span><span id="line-195"></span>
<span class="source-line-no">196</span><span id="line-196"> Iterator&lt;byte[]&gt; iter = qualifiers.iterator();</span>
<span class="source-line-no">197</span><span id="line-197"> while (iter.hasNext()) {</span>
<span class="source-line-no">198</span><span id="line-198"> byte[] next = iter.next();</span>
<span class="source-line-no">199</span><span id="line-199"> expected.add(new KeyValue(row, family, next, 1, (byte[]) null));</span>
<span class="source-line-no">200</span><span id="line-200"> get.addColumn(family, next);</span>
<span class="source-line-no">201</span><span id="line-201"> }</span>
<span class="source-line-no">202</span><span id="line-202"> }</span>
<span class="source-line-no">203</span><span id="line-203"></span>
<span class="source-line-no">204</span><span id="line-204"> private void init(String methodName) throws IOException {</span>
<span class="source-line-no">205</span><span id="line-205"> init(methodName, TEST_UTIL.getConfiguration());</span>
<span class="source-line-no">206</span><span id="line-206"> }</span>
<span class="source-line-no">207</span><span id="line-207"></span>
<span class="source-line-no">208</span><span id="line-208"> private HStore init(String methodName, Configuration conf) throws IOException {</span>
<span class="source-line-no">209</span><span id="line-209"> // some of the tests write 4 versions and then flush</span>
<span class="source-line-no">210</span><span id="line-210"> // (with HBASE-4241, lower versions are collected on flush)</span>
<span class="source-line-no">211</span><span id="line-211"> return init(methodName, conf,</span>
<span class="source-line-no">212</span><span id="line-212"> ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(4).build());</span>
<span class="source-line-no">213</span><span id="line-213"> }</span>
<span class="source-line-no">214</span><span id="line-214"></span>
<span class="source-line-no">215</span><span id="line-215"> private HStore init(String methodName, Configuration conf, ColumnFamilyDescriptor hcd)</span>
<span class="source-line-no">216</span><span id="line-216"> throws IOException {</span>
<span class="source-line-no">217</span><span id="line-217"> return init(methodName, conf, TableDescriptorBuilder.newBuilder(TableName.valueOf(table)), hcd);</span>
<span class="source-line-no">218</span><span id="line-218"> }</span>
<span class="source-line-no">219</span><span id="line-219"></span>
<span class="source-line-no">220</span><span id="line-220"> private HStore init(String methodName, Configuration conf, TableDescriptorBuilder builder,</span>
<span class="source-line-no">221</span><span id="line-221"> ColumnFamilyDescriptor hcd) throws IOException {</span>
<span class="source-line-no">222</span><span id="line-222"> return init(methodName, conf, builder, hcd, null);</span>
<span class="source-line-no">223</span><span id="line-223"> }</span>
<span class="source-line-no">224</span><span id="line-224"></span>
<span class="source-line-no">225</span><span id="line-225"> private HStore init(String methodName, Configuration conf, TableDescriptorBuilder builder,</span>
<span class="source-line-no">226</span><span id="line-226"> ColumnFamilyDescriptor hcd, MyStoreHook hook) throws IOException {</span>
<span class="source-line-no">227</span><span id="line-227"> return init(methodName, conf, builder, hcd, hook, false);</span>
<span class="source-line-no">228</span><span id="line-228"> }</span>
<span class="source-line-no">229</span><span id="line-229"></span>
<span class="source-line-no">230</span><span id="line-230"> private void initHRegion(String methodName, Configuration conf, TableDescriptorBuilder builder,</span>
<span class="source-line-no">231</span><span id="line-231"> ColumnFamilyDescriptor hcd, MyStoreHook hook, boolean switchToPread) throws IOException {</span>
<span class="source-line-no">232</span><span id="line-232"> TableDescriptor htd = builder.setColumnFamily(hcd).build();</span>
<span class="source-line-no">233</span><span id="line-233"> Path basedir = new Path(DIR + methodName);</span>
<span class="source-line-no">234</span><span id="line-234"> Path tableDir = CommonFSUtils.getTableDir(basedir, htd.getTableName());</span>
<span class="source-line-no">235</span><span id="line-235"> final Path logdir = new Path(basedir, AbstractFSWALProvider.getWALDirectoryName(methodName));</span>
<span class="source-line-no">236</span><span id="line-236"></span>
<span class="source-line-no">237</span><span id="line-237"> FileSystem fs = FileSystem.get(conf);</span>
<span class="source-line-no">238</span><span id="line-238"></span>
<span class="source-line-no">239</span><span id="line-239"> fs.delete(logdir, true);</span>
<span class="source-line-no">240</span><span id="line-240"> ChunkCreator.initialize(MemStoreLAB.CHUNK_SIZE_DEFAULT, false,</span>
<span class="source-line-no">241</span><span id="line-241"> MemStoreLABImpl.CHUNK_SIZE_DEFAULT, 1, 0, null,</span>
<span class="source-line-no">242</span><span id="line-242"> MemStoreLAB.INDEX_CHUNK_SIZE_PERCENTAGE_DEFAULT);</span>
<span class="source-line-no">243</span><span id="line-243"> RegionInfo info = RegionInfoBuilder.newBuilder(htd.getTableName()).build();</span>
<span class="source-line-no">244</span><span id="line-244"> Configuration walConf = new Configuration(conf);</span>
<span class="source-line-no">245</span><span id="line-245"> CommonFSUtils.setRootDir(walConf, basedir);</span>
<span class="source-line-no">246</span><span id="line-246"> WALFactory wals = new WALFactory(walConf, methodName);</span>
<span class="source-line-no">247</span><span id="line-247"> region = new HRegion(new HRegionFileSystem(conf, fs, tableDir, info), wals.getWAL(info), conf,</span>
<span class="source-line-no">248</span><span id="line-248"> htd, null);</span>
<span class="source-line-no">249</span><span id="line-249"> region.regionServicesForStores = Mockito.spy(region.regionServicesForStores);</span>
<span class="source-line-no">250</span><span id="line-250"> ThreadPoolExecutor pool = (ThreadPoolExecutor) Executors.newFixedThreadPool(1);</span>
<span class="source-line-no">251</span><span id="line-251"> Mockito.when(region.regionServicesForStores.getInMemoryCompactionPool()).thenReturn(pool);</span>
<span class="source-line-no">252</span><span id="line-252"> }</span>
<span class="source-line-no">253</span><span id="line-253"></span>
<span class="source-line-no">254</span><span id="line-254"> private HStore init(String methodName, Configuration conf, TableDescriptorBuilder builder,</span>
<span class="source-line-no">255</span><span id="line-255"> ColumnFamilyDescriptor hcd, MyStoreHook hook, boolean switchToPread) throws IOException {</span>
<span class="source-line-no">256</span><span id="line-256"> initHRegion(methodName, conf, builder, hcd, hook, switchToPread);</span>
<span class="source-line-no">257</span><span id="line-257"> if (hook == null) {</span>
<span class="source-line-no">258</span><span id="line-258"> store = new HStore(region, hcd, conf, false);</span>
<span class="source-line-no">259</span><span id="line-259"> } else {</span>
<span class="source-line-no">260</span><span id="line-260"> store = new MyStore(region, hcd, conf, hook, switchToPread);</span>
<span class="source-line-no">261</span><span id="line-261"> }</span>
<span class="source-line-no">262</span><span id="line-262"> region.stores.put(store.getColumnFamilyDescriptor().getName(), store);</span>
<span class="source-line-no">263</span><span id="line-263"> return store;</span>
<span class="source-line-no">264</span><span id="line-264"> }</span>
<span class="source-line-no">265</span><span id="line-265"></span>
<span class="source-line-no">266</span><span id="line-266"> /**</span>
<span class="source-line-no">267</span><span id="line-267"> * Test we do not lose data if we fail a flush and then close. Part of HBase-10466</span>
<span class="source-line-no">268</span><span id="line-268"> */</span>
<span class="source-line-no">269</span><span id="line-269"> @Test</span>
<span class="source-line-no">270</span><span id="line-270"> public void testFlushSizeSizing() throws Exception {</span>
<span class="source-line-no">271</span><span id="line-271"> LOG.info("Setting up a faulty file system that cannot write in " + this.name.getMethodName());</span>
<span class="source-line-no">272</span><span id="line-272"> final Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());</span>
<span class="source-line-no">273</span><span id="line-273"> // Only retry once.</span>
<span class="source-line-no">274</span><span id="line-274"> conf.setInt("hbase.hstore.flush.retries.number", 1);</span>
<span class="source-line-no">275</span><span id="line-275"> User user = User.createUserForTesting(conf, this.name.getMethodName(), new String[] { "foo" });</span>
<span class="source-line-no">276</span><span id="line-276"> // Inject our faulty LocalFileSystem</span>
<span class="source-line-no">277</span><span id="line-277"> conf.setClass("fs.file.impl", FaultyFileSystem.class, FileSystem.class);</span>
<span class="source-line-no">278</span><span id="line-278"> user.runAs(new PrivilegedExceptionAction&lt;Object&gt;() {</span>
<span class="source-line-no">279</span><span id="line-279"> @Override</span>
<span class="source-line-no">280</span><span id="line-280"> public Object run() throws Exception {</span>
<span class="source-line-no">281</span><span id="line-281"> // Make sure it worked (above is sensitive to caching details in hadoop core)</span>
<span class="source-line-no">282</span><span id="line-282"> FileSystem fs = FileSystem.get(conf);</span>
<span class="source-line-no">283</span><span id="line-283"> assertEquals(FaultyFileSystem.class, fs.getClass());</span>
<span class="source-line-no">284</span><span id="line-284"> FaultyFileSystem ffs = (FaultyFileSystem) fs;</span>
<span class="source-line-no">285</span><span id="line-285"></span>
<span class="source-line-no">286</span><span id="line-286"> // Initialize region</span>
<span class="source-line-no">287</span><span id="line-287"> init(name.getMethodName(), conf);</span>
<span class="source-line-no">288</span><span id="line-288"></span>
<span class="source-line-no">289</span><span id="line-289"> MemStoreSize mss = store.memstore.getFlushableSize();</span>
<span class="source-line-no">290</span><span id="line-290"> assertEquals(0, mss.getDataSize());</span>
<span class="source-line-no">291</span><span id="line-291"> LOG.info("Adding some data");</span>
<span class="source-line-no">292</span><span id="line-292"> MemStoreSizing kvSize = new NonThreadSafeMemStoreSizing();</span>
<span class="source-line-no">293</span><span id="line-293"> store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), kvSize);</span>
<span class="source-line-no">294</span><span id="line-294"> // add the heap size of active (mutable) segment</span>
<span class="source-line-no">295</span><span id="line-295"> kvSize.incMemStoreSize(0, MutableSegment.DEEP_OVERHEAD, 0, 0);</span>
<span class="source-line-no">296</span><span id="line-296"> mss = store.memstore.getFlushableSize();</span>
<span class="source-line-no">297</span><span id="line-297"> assertEquals(kvSize.getMemStoreSize(), mss);</span>
<span class="source-line-no">298</span><span id="line-298"> // Flush. Bug #1 from HBASE-10466. Make sure size calculation on failed flush is right.</span>
<span class="source-line-no">299</span><span id="line-299"> try {</span>
<span class="source-line-no">300</span><span id="line-300"> LOG.info("Flushing");</span>
<span class="source-line-no">301</span><span id="line-301"> flushStore(store, id++);</span>
<span class="source-line-no">302</span><span id="line-302"> fail("Didn't bubble up IOE!");</span>
<span class="source-line-no">303</span><span id="line-303"> } catch (IOException ioe) {</span>
<span class="source-line-no">304</span><span id="line-304"> assertTrue(ioe.getMessage().contains("Fault injected"));</span>
<span class="source-line-no">305</span><span id="line-305"> }</span>
<span class="source-line-no">306</span><span id="line-306"> // due to snapshot, change mutable to immutable segment</span>
<span class="source-line-no">307</span><span id="line-307"> kvSize.incMemStoreSize(0,</span>
<span class="source-line-no">308</span><span id="line-308"> CSLMImmutableSegment.DEEP_OVERHEAD_CSLM - MutableSegment.DEEP_OVERHEAD, 0, 0);</span>
<span class="source-line-no">309</span><span id="line-309"> mss = store.memstore.getFlushableSize();</span>
<span class="source-line-no">310</span><span id="line-310"> assertEquals(kvSize.getMemStoreSize(), mss);</span>
<span class="source-line-no">311</span><span id="line-311"> MemStoreSizing kvSize2 = new NonThreadSafeMemStoreSizing();</span>
<span class="source-line-no">312</span><span id="line-312"> store.add(new KeyValue(row, family, qf2, 2, (byte[]) null), kvSize2);</span>
<span class="source-line-no">313</span><span id="line-313"> kvSize2.incMemStoreSize(0, MutableSegment.DEEP_OVERHEAD, 0, 0);</span>
<span class="source-line-no">314</span><span id="line-314"> // Even though we add a new kv, we expect the flushable size to be 'same' since we have</span>
<span class="source-line-no">315</span><span id="line-315"> // not yet cleared the snapshot -- the above flush failed.</span>
<span class="source-line-no">316</span><span id="line-316"> assertEquals(kvSize.getMemStoreSize(), mss);</span>
<span class="source-line-no">317</span><span id="line-317"> ffs.fault.set(false);</span>
<span class="source-line-no">318</span><span id="line-318"> flushStore(store, id++);</span>
<span class="source-line-no">319</span><span id="line-319"> mss = store.memstore.getFlushableSize();</span>
<span class="source-line-no">320</span><span id="line-320"> // Size should be the foreground kv size.</span>
<span class="source-line-no">321</span><span id="line-321"> assertEquals(kvSize2.getMemStoreSize(), mss);</span>
<span class="source-line-no">322</span><span id="line-322"> flushStore(store, id++);</span>
<span class="source-line-no">323</span><span id="line-323"> mss = store.memstore.getFlushableSize();</span>
<span class="source-line-no">324</span><span id="line-324"> assertEquals(0, mss.getDataSize());</span>
<span class="source-line-no">325</span><span id="line-325"> assertEquals(MutableSegment.DEEP_OVERHEAD, mss.getHeapSize());</span>
<span class="source-line-no">326</span><span id="line-326"> return null;</span>
<span class="source-line-no">327</span><span id="line-327"> }</span>
<span class="source-line-no">328</span><span id="line-328"> });</span>
<span class="source-line-no">329</span><span id="line-329"> }</span>
<span class="source-line-no">330</span><span id="line-330"></span>
<span class="source-line-no">331</span><span id="line-331"> @Test</span>
<span class="source-line-no">332</span><span id="line-332"> public void testStoreBloomFilterMetricsWithBloomRowCol() throws IOException {</span>
<span class="source-line-no">333</span><span id="line-333"> int numStoreFiles = 5;</span>
<span class="source-line-no">334</span><span id="line-334"> writeAndRead(BloomType.ROWCOL, numStoreFiles);</span>
<span class="source-line-no">335</span><span id="line-335"></span>
<span class="source-line-no">336</span><span id="line-336"> assertEquals(0, store.getBloomFilterEligibleRequestsCount());</span>
<span class="source-line-no">337</span><span id="line-337"> // hard to know exactly the numbers here, we are just trying to</span>
<span class="source-line-no">338</span><span id="line-338"> // prove that they are incrementing</span>
<span class="source-line-no">339</span><span id="line-339"> assertTrue(store.getBloomFilterRequestsCount() &gt;= numStoreFiles);</span>
<span class="source-line-no">340</span><span id="line-340"> assertTrue(store.getBloomFilterNegativeResultsCount() &gt; 0);</span>
<span class="source-line-no">341</span><span id="line-341"> }</span>
<span class="source-line-no">342</span><span id="line-342"></span>
<span class="source-line-no">343</span><span id="line-343"> @Test</span>
<span class="source-line-no">344</span><span id="line-344"> public void testStoreBloomFilterMetricsWithBloomRow() throws IOException {</span>
<span class="source-line-no">345</span><span id="line-345"> int numStoreFiles = 5;</span>
<span class="source-line-no">346</span><span id="line-346"> writeAndRead(BloomType.ROWCOL, numStoreFiles);</span>
<span class="source-line-no">347</span><span id="line-347"></span>
<span class="source-line-no">348</span><span id="line-348"> assertEquals(0, store.getBloomFilterEligibleRequestsCount());</span>
<span class="source-line-no">349</span><span id="line-349"> // hard to know exactly the numbers here, we are just trying to</span>
<span class="source-line-no">350</span><span id="line-350"> // prove that they are incrementing</span>
<span class="source-line-no">351</span><span id="line-351"> assertTrue(store.getBloomFilterRequestsCount() &gt;= numStoreFiles);</span>
<span class="source-line-no">352</span><span id="line-352"> assertTrue(store.getBloomFilterNegativeResultsCount() &gt; 0);</span>
<span class="source-line-no">353</span><span id="line-353"> }</span>
<span class="source-line-no">354</span><span id="line-354"></span>
<span class="source-line-no">355</span><span id="line-355"> @Test</span>
<span class="source-line-no">356</span><span id="line-356"> public void testStoreBloomFilterMetricsWithBloomRowPrefix() throws IOException {</span>
<span class="source-line-no">357</span><span id="line-357"> int numStoreFiles = 5;</span>
<span class="source-line-no">358</span><span id="line-358"> writeAndRead(BloomType.ROWPREFIX_FIXED_LENGTH, numStoreFiles);</span>
<span class="source-line-no">359</span><span id="line-359"></span>
<span class="source-line-no">360</span><span id="line-360"> assertEquals(0, store.getBloomFilterEligibleRequestsCount());</span>
<span class="source-line-no">361</span><span id="line-361"> // hard to know exactly the numbers here, we are just trying to</span>
<span class="source-line-no">362</span><span id="line-362"> // prove that they are incrementing</span>
<span class="source-line-no">363</span><span id="line-363"> assertTrue(store.getBloomFilterRequestsCount() &gt;= numStoreFiles);</span>
<span class="source-line-no">364</span><span id="line-364"> }</span>
<span class="source-line-no">365</span><span id="line-365"></span>
<span class="source-line-no">366</span><span id="line-366"> @Test</span>
<span class="source-line-no">367</span><span id="line-367"> public void testStoreBloomFilterMetricsWithBloomNone() throws IOException {</span>
<span class="source-line-no">368</span><span id="line-368"> int numStoreFiles = 5;</span>
<span class="source-line-no">369</span><span id="line-369"> writeAndRead(BloomType.NONE, numStoreFiles);</span>
<span class="source-line-no">370</span><span id="line-370"></span>
<span class="source-line-no">371</span><span id="line-371"> assertEquals(0, store.getBloomFilterRequestsCount());</span>
<span class="source-line-no">372</span><span id="line-372"> assertEquals(0, store.getBloomFilterNegativeResultsCount());</span>
<span class="source-line-no">373</span><span id="line-373"></span>
<span class="source-line-no">374</span><span id="line-374"> // hard to know exactly the numbers here, we are just trying to</span>
<span class="source-line-no">375</span><span id="line-375"> // prove that they are incrementing</span>
<span class="source-line-no">376</span><span id="line-376"> assertTrue(store.getBloomFilterEligibleRequestsCount() &gt;= numStoreFiles);</span>
<span class="source-line-no">377</span><span id="line-377"> }</span>
<span class="source-line-no">378</span><span id="line-378"></span>
<span class="source-line-no">379</span><span id="line-379"> private void writeAndRead(BloomType bloomType, int numStoreFiles) throws IOException {</span>
<span class="source-line-no">380</span><span id="line-380"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">381</span><span id="line-381"> FileSystem fs = FileSystem.get(conf);</span>
<span class="source-line-no">382</span><span id="line-382"></span>
<span class="source-line-no">383</span><span id="line-383"> ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.newBuilder(family)</span>
<span class="source-line-no">384</span><span id="line-384"> .setCompressionType(Compression.Algorithm.GZ).setBloomFilterType(bloomType)</span>
<span class="source-line-no">385</span><span id="line-385"> .setConfiguration(BloomFilterUtil.PREFIX_LENGTH_KEY, "3").build();</span>
<span class="source-line-no">386</span><span id="line-386"> init(name.getMethodName(), conf, hcd);</span>
<span class="source-line-no">387</span><span id="line-387"></span>
<span class="source-line-no">388</span><span id="line-388"> for (int i = 1; i &lt;= numStoreFiles; i++) {</span>
<span class="source-line-no">389</span><span id="line-389"> byte[] row = Bytes.toBytes("row" + i);</span>
<span class="source-line-no">390</span><span id="line-390"> LOG.info("Adding some data for the store file #" + i);</span>
<span class="source-line-no">391</span><span id="line-391"> long timeStamp = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">392</span><span id="line-392"> this.store.add(new KeyValue(row, family, qf1, timeStamp, (byte[]) null), null);</span>
<span class="source-line-no">393</span><span id="line-393"> this.store.add(new KeyValue(row, family, qf2, timeStamp, (byte[]) null), null);</span>
<span class="source-line-no">394</span><span id="line-394"> this.store.add(new KeyValue(row, family, qf3, timeStamp, (byte[]) null), null);</span>
<span class="source-line-no">395</span><span id="line-395"> flush(i);</span>
<span class="source-line-no">396</span><span id="line-396"> }</span>
<span class="source-line-no">397</span><span id="line-397"></span>
<span class="source-line-no">398</span><span id="line-398"> // Verify the total number of store files</span>
<span class="source-line-no">399</span><span id="line-399"> assertEquals(numStoreFiles, this.store.getStorefiles().size());</span>
<span class="source-line-no">400</span><span id="line-400"></span>
<span class="source-line-no">401</span><span id="line-401"> TreeSet&lt;byte[]&gt; columns = new TreeSet&lt;&gt;(Bytes.BYTES_COMPARATOR);</span>
<span class="source-line-no">402</span><span id="line-402"> columns.add(qf1);</span>
<span class="source-line-no">403</span><span id="line-403"></span>
<span class="source-line-no">404</span><span id="line-404"> for (int i = 1; i &lt;= numStoreFiles; i++) {</span>
<span class="source-line-no">405</span><span id="line-405"> KeyValueScanner scanner =</span>
<span class="source-line-no">406</span><span id="line-406"> store.getScanner(new Scan(new Get(Bytes.toBytes("row" + i))), columns, 0);</span>
<span class="source-line-no">407</span><span id="line-407"> scanner.peek();</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"></span>
<span class="source-line-no">411</span><span id="line-411"> /**</span>
<span class="source-line-no">412</span><span id="line-412"> * Verify that compression and data block encoding are respected by the createWriter method, used</span>
<span class="source-line-no">413</span><span id="line-413"> * on store flush.</span>
<span class="source-line-no">414</span><span id="line-414"> */</span>
<span class="source-line-no">415</span><span id="line-415"> @Test</span>
<span class="source-line-no">416</span><span id="line-416"> public void testCreateWriter() throws Exception {</span>
<span class="source-line-no">417</span><span id="line-417"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">418</span><span id="line-418"> FileSystem fs = FileSystem.get(conf);</span>
<span class="source-line-no">419</span><span id="line-419"></span>
<span class="source-line-no">420</span><span id="line-420"> ColumnFamilyDescriptor hcd =</span>
<span class="source-line-no">421</span><span id="line-421"> ColumnFamilyDescriptorBuilder.newBuilder(family).setCompressionType(Compression.Algorithm.GZ)</span>
<span class="source-line-no">422</span><span id="line-422"> .setDataBlockEncoding(DataBlockEncoding.DIFF).build();</span>
<span class="source-line-no">423</span><span id="line-423"> init(name.getMethodName(), conf, hcd);</span>
<span class="source-line-no">424</span><span id="line-424"></span>
<span class="source-line-no">425</span><span id="line-425"> // Test createWriter</span>
<span class="source-line-no">426</span><span id="line-426"> StoreFileWriter writer = store.getStoreEngine()</span>
<span class="source-line-no">427</span><span id="line-427"> .createWriter(CreateStoreFileWriterParams.create().maxKeyCount(4)</span>
<span class="source-line-no">428</span><span id="line-428"> .compression(hcd.getCompressionType()).isCompaction(false).includeMVCCReadpoint(true)</span>
<span class="source-line-no">429</span><span id="line-429"> .includesTag(false).shouldDropBehind(false));</span>
<span class="source-line-no">430</span><span id="line-430"> Path path = writer.getPath();</span>
<span class="source-line-no">431</span><span id="line-431"> writer.append(new KeyValue(row, family, qf1, Bytes.toBytes(1)));</span>
<span class="source-line-no">432</span><span id="line-432"> writer.append(new KeyValue(row, family, qf2, Bytes.toBytes(2)));</span>
<span class="source-line-no">433</span><span id="line-433"> writer.append(new KeyValue(row2, family, qf1, Bytes.toBytes(3)));</span>
<span class="source-line-no">434</span><span id="line-434"> writer.append(new KeyValue(row2, family, qf2, Bytes.toBytes(4)));</span>
<span class="source-line-no">435</span><span id="line-435"> writer.close();</span>
<span class="source-line-no">436</span><span id="line-436"></span>
<span class="source-line-no">437</span><span id="line-437"> // Verify that compression and encoding settings are respected</span>
<span class="source-line-no">438</span><span id="line-438"> HFile.Reader reader = HFile.createReader(fs, path, new CacheConfig(conf), true, conf);</span>
<span class="source-line-no">439</span><span id="line-439"> assertEquals(hcd.getCompressionType(), reader.getTrailer().getCompressionCodec());</span>
<span class="source-line-no">440</span><span id="line-440"> assertEquals(hcd.getDataBlockEncoding(), reader.getDataBlockEncoding());</span>
<span class="source-line-no">441</span><span id="line-441"> reader.close();</span>
<span class="source-line-no">442</span><span id="line-442"> }</span>
<span class="source-line-no">443</span><span id="line-443"></span>
<span class="source-line-no">444</span><span id="line-444"> @Test</span>
<span class="source-line-no">445</span><span id="line-445"> public void testDeleteExpiredStoreFiles() throws Exception {</span>
<span class="source-line-no">446</span><span id="line-446"> testDeleteExpiredStoreFiles(0);</span>
<span class="source-line-no">447</span><span id="line-447"> testDeleteExpiredStoreFiles(1);</span>
<span class="source-line-no">448</span><span id="line-448"> }</span>
<span class="source-line-no">449</span><span id="line-449"></span>
<span class="source-line-no">450</span><span id="line-450"> /**</span>
<span class="source-line-no">451</span><span id="line-451"> * @param minVersions the MIN_VERSIONS for the column family</span>
<span class="source-line-no">452</span><span id="line-452"> */</span>
<span class="source-line-no">453</span><span id="line-453"> public void testDeleteExpiredStoreFiles(int minVersions) throws Exception {</span>
<span class="source-line-no">454</span><span id="line-454"> int storeFileNum = 4;</span>
<span class="source-line-no">455</span><span id="line-455"> int ttl = 4;</span>
<span class="source-line-no">456</span><span id="line-456"> IncrementingEnvironmentEdge edge = new IncrementingEnvironmentEdge();</span>
<span class="source-line-no">457</span><span id="line-457"> EnvironmentEdgeManagerTestHelper.injectEdge(edge);</span>
<span class="source-line-no">458</span><span id="line-458"></span>
<span class="source-line-no">459</span><span id="line-459"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">460</span><span id="line-460"> // Enable the expired store file deletion</span>
<span class="source-line-no">461</span><span id="line-461"> conf.setBoolean("hbase.store.delete.expired.storefile", true);</span>
<span class="source-line-no">462</span><span id="line-462"> // Set the compaction threshold higher to avoid normal compactions.</span>
<span class="source-line-no">463</span><span id="line-463"> conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MIN_KEY, 5);</span>
<span class="source-line-no">464</span><span id="line-464"></span>
<span class="source-line-no">465</span><span id="line-465"> init(name.getMethodName() + "-" + minVersions, conf, ColumnFamilyDescriptorBuilder</span>
<span class="source-line-no">466</span><span id="line-466"> .newBuilder(family).setMinVersions(minVersions).setTimeToLive(ttl).build());</span>
<span class="source-line-no">467</span><span id="line-467"></span>
<span class="source-line-no">468</span><span id="line-468"> long storeTtl = this.store.getScanInfo().getTtl();</span>
<span class="source-line-no">469</span><span id="line-469"> long sleepTime = storeTtl / storeFileNum;</span>
<span class="source-line-no">470</span><span id="line-470"> long timeStamp;</span>
<span class="source-line-no">471</span><span id="line-471"> // There are 4 store files and the max time stamp difference among these</span>
<span class="source-line-no">472</span><span id="line-472"> // store files will be (this.store.ttl / storeFileNum)</span>
<span class="source-line-no">473</span><span id="line-473"> for (int i = 1; i &lt;= storeFileNum; i++) {</span>
<span class="source-line-no">474</span><span id="line-474"> LOG.info("Adding some data for the store file #" + i);</span>
<span class="source-line-no">475</span><span id="line-475"> timeStamp = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">476</span><span id="line-476"> this.store.add(new KeyValue(row, family, qf1, timeStamp, (byte[]) null), null);</span>
<span class="source-line-no">477</span><span id="line-477"> this.store.add(new KeyValue(row, family, qf2, timeStamp, (byte[]) null), null);</span>
<span class="source-line-no">478</span><span id="line-478"> this.store.add(new KeyValue(row, family, qf3, timeStamp, (byte[]) null), null);</span>
<span class="source-line-no">479</span><span id="line-479"> flush(i);</span>
<span class="source-line-no">480</span><span id="line-480"> edge.incrementTime(sleepTime);</span>
<span class="source-line-no">481</span><span id="line-481"> }</span>
<span class="source-line-no">482</span><span id="line-482"></span>
<span class="source-line-no">483</span><span id="line-483"> // Verify the total number of store files</span>
<span class="source-line-no">484</span><span id="line-484"> assertEquals(storeFileNum, this.store.getStorefiles().size());</span>
<span class="source-line-no">485</span><span id="line-485"></span>
<span class="source-line-no">486</span><span id="line-486"> // Each call will find one expired store file and delete it before compaction happens.</span>
<span class="source-line-no">487</span><span id="line-487"> // There will be no compaction due to threshold above. Last file will not be replaced.</span>
<span class="source-line-no">488</span><span id="line-488"> for (int i = 1; i &lt;= storeFileNum - 1; i++) {</span>
<span class="source-line-no">489</span><span id="line-489"> // verify the expired store file.</span>
<span class="source-line-no">490</span><span id="line-490"> assertFalse(this.store.requestCompaction().isPresent());</span>
<span class="source-line-no">491</span><span id="line-491"> Collection&lt;HStoreFile&gt; sfs = this.store.getStorefiles();</span>
<span class="source-line-no">492</span><span id="line-492"> // Ensure i files are gone.</span>
<span class="source-line-no">493</span><span id="line-493"> if (minVersions == 0) {</span>
<span class="source-line-no">494</span><span id="line-494"> assertEquals(storeFileNum - i, sfs.size());</span>
<span class="source-line-no">495</span><span id="line-495"> // Ensure only non-expired files remain.</span>
<span class="source-line-no">496</span><span id="line-496"> for (HStoreFile sf : sfs) {</span>
<span class="source-line-no">497</span><span id="line-497"> assertTrue(sf.getReader().getMaxTimestamp() &gt;= (edge.currentTime() - storeTtl));</span>
<span class="source-line-no">498</span><span id="line-498"> }</span>
<span class="source-line-no">499</span><span id="line-499"> } else {</span>
<span class="source-line-no">500</span><span id="line-500"> assertEquals(storeFileNum, sfs.size());</span>
<span class="source-line-no">501</span><span id="line-501"> }</span>
<span class="source-line-no">502</span><span id="line-502"> // Let the next store file expired.</span>
<span class="source-line-no">503</span><span id="line-503"> edge.incrementTime(sleepTime);</span>
<span class="source-line-no">504</span><span id="line-504"> }</span>
<span class="source-line-no">505</span><span id="line-505"> assertFalse(this.store.requestCompaction().isPresent());</span>
<span class="source-line-no">506</span><span id="line-506"></span>
<span class="source-line-no">507</span><span id="line-507"> Collection&lt;HStoreFile&gt; sfs = this.store.getStorefiles();</span>
<span class="source-line-no">508</span><span id="line-508"> // Assert the last expired file is not removed.</span>
<span class="source-line-no">509</span><span id="line-509"> if (minVersions == 0) {</span>
<span class="source-line-no">510</span><span id="line-510"> assertEquals(1, sfs.size());</span>
<span class="source-line-no">511</span><span id="line-511"> }</span>
<span class="source-line-no">512</span><span id="line-512"> long ts = sfs.iterator().next().getReader().getMaxTimestamp();</span>
<span class="source-line-no">513</span><span id="line-513"> assertTrue(ts &lt; (edge.currentTime() - storeTtl));</span>
<span class="source-line-no">514</span><span id="line-514"></span>
<span class="source-line-no">515</span><span id="line-515"> for (HStoreFile sf : sfs) {</span>
<span class="source-line-no">516</span><span id="line-516"> sf.closeStoreFile(true);</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"></span>
<span class="source-line-no">520</span><span id="line-520"> @Test</span>
<span class="source-line-no">521</span><span id="line-521"> public void testLowestModificationTime() throws Exception {</span>
<span class="source-line-no">522</span><span id="line-522"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">523</span><span id="line-523"> FileSystem fs = FileSystem.get(conf);</span>
<span class="source-line-no">524</span><span id="line-524"> // Initialize region</span>
<span class="source-line-no">525</span><span id="line-525"> init(name.getMethodName(), conf);</span>
<span class="source-line-no">526</span><span id="line-526"></span>
<span class="source-line-no">527</span><span id="line-527"> int storeFileNum = 4;</span>
<span class="source-line-no">528</span><span id="line-528"> for (int i = 1; i &lt;= storeFileNum; i++) {</span>
<span class="source-line-no">529</span><span id="line-529"> LOG.info("Adding some data for the store file #" + i);</span>
<span class="source-line-no">530</span><span id="line-530"> this.store.add(new KeyValue(row, family, qf1, i, (byte[]) null), null);</span>
<span class="source-line-no">531</span><span id="line-531"> this.store.add(new KeyValue(row, family, qf2, i, (byte[]) null), null);</span>
<span class="source-line-no">532</span><span id="line-532"> this.store.add(new KeyValue(row, family, qf3, i, (byte[]) null), null);</span>
<span class="source-line-no">533</span><span id="line-533"> flush(i);</span>
<span class="source-line-no">534</span><span id="line-534"> }</span>
<span class="source-line-no">535</span><span id="line-535"> // after flush; check the lowest time stamp</span>
<span class="source-line-no">536</span><span id="line-536"> long lowestTimeStampFromManager = StoreUtils.getLowestTimestamp(store.getStorefiles());</span>
<span class="source-line-no">537</span><span id="line-537"> long lowestTimeStampFromFS = getLowestTimeStampFromFS(fs, store.getStorefiles());</span>
<span class="source-line-no">538</span><span id="line-538"> assertEquals(lowestTimeStampFromManager, lowestTimeStampFromFS);</span>
<span class="source-line-no">539</span><span id="line-539"></span>
<span class="source-line-no">540</span><span id="line-540"> // after compact; check the lowest time stamp</span>
<span class="source-line-no">541</span><span id="line-541"> store.compact(store.requestCompaction().get(), NoLimitThroughputController.INSTANCE, null);</span>
<span class="source-line-no">542</span><span id="line-542"> lowestTimeStampFromManager = StoreUtils.getLowestTimestamp(store.getStorefiles());</span>
<span class="source-line-no">543</span><span id="line-543"> lowestTimeStampFromFS = getLowestTimeStampFromFS(fs, store.getStorefiles());</span>
<span class="source-line-no">544</span><span id="line-544"> assertEquals(lowestTimeStampFromManager, lowestTimeStampFromFS);</span>
<span class="source-line-no">545</span><span id="line-545"> }</span>
<span class="source-line-no">546</span><span id="line-546"></span>
<span class="source-line-no">547</span><span id="line-547"> private static long getLowestTimeStampFromFS(FileSystem fs,</span>
<span class="source-line-no">548</span><span id="line-548"> final Collection&lt;HStoreFile&gt; candidates) throws IOException {</span>
<span class="source-line-no">549</span><span id="line-549"> long minTs = Long.MAX_VALUE;</span>
<span class="source-line-no">550</span><span id="line-550"> if (candidates.isEmpty()) {</span>
<span class="source-line-no">551</span><span id="line-551"> return minTs;</span>
<span class="source-line-no">552</span><span id="line-552"> }</span>
<span class="source-line-no">553</span><span id="line-553"> Path[] p = new Path[candidates.size()];</span>
<span class="source-line-no">554</span><span id="line-554"> int i = 0;</span>
<span class="source-line-no">555</span><span id="line-555"> for (HStoreFile sf : candidates) {</span>
<span class="source-line-no">556</span><span id="line-556"> p[i] = sf.getPath();</span>
<span class="source-line-no">557</span><span id="line-557"> ++i;</span>
<span class="source-line-no">558</span><span id="line-558"> }</span>
<span class="source-line-no">559</span><span id="line-559"></span>
<span class="source-line-no">560</span><span id="line-560"> FileStatus[] stats = fs.listStatus(p);</span>
<span class="source-line-no">561</span><span id="line-561"> if (stats == null || stats.length == 0) {</span>
<span class="source-line-no">562</span><span id="line-562"> return minTs;</span>
<span class="source-line-no">563</span><span id="line-563"> }</span>
<span class="source-line-no">564</span><span id="line-564"> for (FileStatus s : stats) {</span>
<span class="source-line-no">565</span><span id="line-565"> minTs = Math.min(minTs, s.getModificationTime());</span>
<span class="source-line-no">566</span><span id="line-566"> }</span>
<span class="source-line-no">567</span><span id="line-567"> return minTs;</span>
<span class="source-line-no">568</span><span id="line-568"> }</span>
<span class="source-line-no">569</span><span id="line-569"></span>
<span class="source-line-no">570</span><span id="line-570"> //////////////////////////////////////////////////////////////////////////////</span>
<span class="source-line-no">571</span><span id="line-571"> // Get tests</span>
<span class="source-line-no">572</span><span id="line-572"> //////////////////////////////////////////////////////////////////////////////</span>
<span class="source-line-no">573</span><span id="line-573"></span>
<span class="source-line-no">574</span><span id="line-574"> private static final int BLOCKSIZE_SMALL = 8192;</span>
<span class="source-line-no">575</span><span id="line-575"></span>
<span class="source-line-no">576</span><span id="line-576"> /**</span>
<span class="source-line-no">577</span><span id="line-577"> * Test for hbase-1686.</span>
<span class="source-line-no">578</span><span id="line-578"> */</span>
<span class="source-line-no">579</span><span id="line-579"> @Test</span>
<span class="source-line-no">580</span><span id="line-580"> public void testEmptyStoreFile() throws IOException {</span>
<span class="source-line-no">581</span><span id="line-581"> init(this.name.getMethodName());</span>
<span class="source-line-no">582</span><span id="line-582"> // Write a store file.</span>
<span class="source-line-no">583</span><span id="line-583"> this.store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), null);</span>
<span class="source-line-no">584</span><span id="line-584"> this.store.add(new KeyValue(row, family, qf2, 1, (byte[]) null), null);</span>
<span class="source-line-no">585</span><span id="line-585"> flush(1);</span>
<span class="source-line-no">586</span><span id="line-586"> // Now put in place an empty store file. Its a little tricky. Have to</span>
<span class="source-line-no">587</span><span id="line-587"> // do manually with hacked in sequence id.</span>
<span class="source-line-no">588</span><span id="line-588"> HStoreFile f = this.store.getStorefiles().iterator().next();</span>
<span class="source-line-no">589</span><span id="line-589"> Path storedir = f.getPath().getParent();</span>
<span class="source-line-no">590</span><span id="line-590"> long seqid = f.getMaxSequenceId();</span>
<span class="source-line-no">591</span><span id="line-591"> Configuration c = HBaseConfiguration.create();</span>
<span class="source-line-no">592</span><span id="line-592"> FileSystem fs = FileSystem.get(c);</span>
<span class="source-line-no">593</span><span id="line-593"> HFileContext meta = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL).build();</span>
<span class="source-line-no">594</span><span id="line-594"> StoreFileWriter w = new StoreFileWriter.Builder(c, new CacheConfig(c), fs)</span>
<span class="source-line-no">595</span><span id="line-595"> .withOutputDir(storedir).withFileContext(meta).build();</span>
<span class="source-line-no">596</span><span id="line-596"> w.appendMetadata(seqid + 1, false);</span>
<span class="source-line-no">597</span><span id="line-597"> w.close();</span>
<span class="source-line-no">598</span><span id="line-598"> this.store.close();</span>
<span class="source-line-no">599</span><span id="line-599"> // Reopen it... should pick up two files</span>
<span class="source-line-no">600</span><span id="line-600"> this.store =</span>
<span class="source-line-no">601</span><span id="line-601"> new HStore(this.store.getHRegion(), this.store.getColumnFamilyDescriptor(), c, false);</span>
<span class="source-line-no">602</span><span id="line-602"> assertEquals(2, this.store.getStorefilesCount());</span>
<span class="source-line-no">603</span><span id="line-603"></span>
<span class="source-line-no">604</span><span id="line-604"> result = HBaseTestingUtil.getFromStoreFile(store, get.getRow(), qualifiers);</span>
<span class="source-line-no">605</span><span id="line-605"> assertEquals(1, result.size());</span>
<span class="source-line-no">606</span><span id="line-606"> }</span>
<span class="source-line-no">607</span><span id="line-607"></span>
<span class="source-line-no">608</span><span id="line-608"> /**</span>
<span class="source-line-no">609</span><span id="line-609"> * Getting data from memstore only</span>
<span class="source-line-no">610</span><span id="line-610"> */</span>
<span class="source-line-no">611</span><span id="line-611"> @Test</span>
<span class="source-line-no">612</span><span id="line-612"> public void testGet_FromMemStoreOnly() throws IOException {</span>
<span class="source-line-no">613</span><span id="line-613"> init(this.name.getMethodName());</span>
<span class="source-line-no">614</span><span id="line-614"></span>
<span class="source-line-no">615</span><span id="line-615"> // Put data in memstore</span>
<span class="source-line-no">616</span><span id="line-616"> this.store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), null);</span>
<span class="source-line-no">617</span><span id="line-617"> this.store.add(new KeyValue(row, family, qf2, 1, (byte[]) null), null);</span>
<span class="source-line-no">618</span><span id="line-618"> this.store.add(new KeyValue(row, family, qf3, 1, (byte[]) null), null);</span>
<span class="source-line-no">619</span><span id="line-619"> this.store.add(new KeyValue(row, family, qf4, 1, (byte[]) null), null);</span>
<span class="source-line-no">620</span><span id="line-620"> this.store.add(new KeyValue(row, family, qf5, 1, (byte[]) null), null);</span>
<span class="source-line-no">621</span><span id="line-621"> this.store.add(new KeyValue(row, family, qf6, 1, (byte[]) null), null);</span>
<span class="source-line-no">622</span><span id="line-622"></span>
<span class="source-line-no">623</span><span id="line-623"> // Get</span>
<span class="source-line-no">624</span><span id="line-624"> result = HBaseTestingUtil.getFromStoreFile(store, get.getRow(), qualifiers);</span>
<span class="source-line-no">625</span><span id="line-625"></span>
<span class="source-line-no">626</span><span id="line-626"> // Compare</span>
<span class="source-line-no">627</span><span id="line-627"> assertCheck();</span>
<span class="source-line-no">628</span><span id="line-628"> }</span>
<span class="source-line-no">629</span><span id="line-629"></span>
<span class="source-line-no">630</span><span id="line-630"> @Test</span>
<span class="source-line-no">631</span><span id="line-631"> public void testTimeRangeIfSomeCellsAreDroppedInFlush() throws IOException {</span>
<span class="source-line-no">632</span><span id="line-632"> testTimeRangeIfSomeCellsAreDroppedInFlush(1);</span>
<span class="source-line-no">633</span><span id="line-633"> testTimeRangeIfSomeCellsAreDroppedInFlush(3);</span>
<span class="source-line-no">634</span><span id="line-634"> testTimeRangeIfSomeCellsAreDroppedInFlush(5);</span>
<span class="source-line-no">635</span><span id="line-635"> }</span>
<span class="source-line-no">636</span><span id="line-636"></span>
<span class="source-line-no">637</span><span id="line-637"> private void testTimeRangeIfSomeCellsAreDroppedInFlush(int maxVersion) throws IOException {</span>
<span class="source-line-no">638</span><span id="line-638"> init(this.name.getMethodName(), TEST_UTIL.getConfiguration(),</span>
<span class="source-line-no">639</span><span id="line-639"> ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(maxVersion).build());</span>
<span class="source-line-no">640</span><span id="line-640"> long currentTs = 100;</span>
<span class="source-line-no">641</span><span id="line-641"> long minTs = currentTs;</span>
<span class="source-line-no">642</span><span id="line-642"> // the extra cell won't be flushed to disk,</span>
<span class="source-line-no">643</span><span id="line-643"> // so the min of timerange will be different between memStore and hfile.</span>
<span class="source-line-no">644</span><span id="line-644"> for (int i = 0; i != (maxVersion + 1); ++i) {</span>
<span class="source-line-no">645</span><span id="line-645"> this.store.add(new KeyValue(row, family, qf1, ++currentTs, (byte[]) null), null);</span>
<span class="source-line-no">646</span><span id="line-646"> if (i == 1) {</span>
<span class="source-line-no">647</span><span id="line-647"> minTs = currentTs;</span>
<span class="source-line-no">648</span><span id="line-648"> }</span>
<span class="source-line-no">649</span><span id="line-649"> }</span>
<span class="source-line-no">650</span><span id="line-650"> flushStore(store, id++);</span>
<span class="source-line-no">651</span><span id="line-651"></span>
<span class="source-line-no">652</span><span id="line-652"> Collection&lt;HStoreFile&gt; files = store.getStorefiles();</span>
<span class="source-line-no">653</span><span id="line-653"> assertEquals(1, files.size());</span>
<span class="source-line-no">654</span><span id="line-654"> HStoreFile f = files.iterator().next();</span>
<span class="source-line-no">655</span><span id="line-655"> f.initReader();</span>
<span class="source-line-no">656</span><span id="line-656"> StoreFileReader reader = f.getReader();</span>
<span class="source-line-no">657</span><span id="line-657"> assertEquals(minTs, reader.timeRange.getMin());</span>
<span class="source-line-no">658</span><span id="line-658"> assertEquals(currentTs, reader.timeRange.getMax());</span>
<span class="source-line-no">659</span><span id="line-659"> }</span>
<span class="source-line-no">660</span><span id="line-660"></span>
<span class="source-line-no">661</span><span id="line-661"> /**</span>
<span class="source-line-no">662</span><span id="line-662"> * Getting data from files only</span>
<span class="source-line-no">663</span><span id="line-663"> */</span>
<span class="source-line-no">664</span><span id="line-664"> @Test</span>
<span class="source-line-no">665</span><span id="line-665"> public void testGet_FromFilesOnly() throws IOException {</span>
<span class="source-line-no">666</span><span id="line-666"> init(this.name.getMethodName());</span>
<span class="source-line-no">667</span><span id="line-667"></span>
<span class="source-line-no">668</span><span id="line-668"> // Put data in memstore</span>
<span class="source-line-no">669</span><span id="line-669"> this.store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), null);</span>
<span class="source-line-no">670</span><span id="line-670"> this.store.add(new KeyValue(row, family, qf2, 1, (byte[]) null), null);</span>
<span class="source-line-no">671</span><span id="line-671"> // flush</span>
<span class="source-line-no">672</span><span id="line-672"> flush(1);</span>
<span class="source-line-no">673</span><span id="line-673"></span>
<span class="source-line-no">674</span><span id="line-674"> // Add more data</span>
<span class="source-line-no">675</span><span id="line-675"> this.store.add(new KeyValue(row, family, qf3, 1, (byte[]) null), null);</span>
<span class="source-line-no">676</span><span id="line-676"> this.store.add(new KeyValue(row, family, qf4, 1, (byte[]) null), null);</span>
<span class="source-line-no">677</span><span id="line-677"> // flush</span>
<span class="source-line-no">678</span><span id="line-678"> flush(2);</span>
<span class="source-line-no">679</span><span id="line-679"></span>
<span class="source-line-no">680</span><span id="line-680"> // Add more data</span>
<span class="source-line-no">681</span><span id="line-681"> this.store.add(new KeyValue(row, family, qf5, 1, (byte[]) null), null);</span>
<span class="source-line-no">682</span><span id="line-682"> this.store.add(new KeyValue(row, family, qf6, 1, (byte[]) null), null);</span>
<span class="source-line-no">683</span><span id="line-683"> // flush</span>
<span class="source-line-no">684</span><span id="line-684"> flush(3);</span>
<span class="source-line-no">685</span><span id="line-685"></span>
<span class="source-line-no">686</span><span id="line-686"> // Get</span>
<span class="source-line-no">687</span><span id="line-687"> result = HBaseTestingUtil.getFromStoreFile(store, get.getRow(), qualifiers);</span>
<span class="source-line-no">688</span><span id="line-688"> // this.store.get(get, qualifiers, result);</span>
<span class="source-line-no">689</span><span id="line-689"></span>
<span class="source-line-no">690</span><span id="line-690"> // Need to sort the result since multiple files</span>
<span class="source-line-no">691</span><span id="line-691"> Collections.sort(result, CellComparatorImpl.COMPARATOR);</span>
<span class="source-line-no">692</span><span id="line-692"></span>
<span class="source-line-no">693</span><span id="line-693"> // Compare</span>
<span class="source-line-no">694</span><span id="line-694"> assertCheck();</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"> /**</span>
<span class="source-line-no">698</span><span id="line-698"> * Getting data from memstore and files</span>
<span class="source-line-no">699</span><span id="line-699"> */</span>
<span class="source-line-no">700</span><span id="line-700"> @Test</span>
<span class="source-line-no">701</span><span id="line-701"> public void testGet_FromMemStoreAndFiles() throws IOException {</span>
<span class="source-line-no">702</span><span id="line-702"> init(this.name.getMethodName());</span>
<span class="source-line-no">703</span><span id="line-703"></span>
<span class="source-line-no">704</span><span id="line-704"> // Put data in memstore</span>
<span class="source-line-no">705</span><span id="line-705"> this.store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), null);</span>
<span class="source-line-no">706</span><span id="line-706"> this.store.add(new KeyValue(row, family, qf2, 1, (byte[]) null), null);</span>
<span class="source-line-no">707</span><span id="line-707"> // flush</span>
<span class="source-line-no">708</span><span id="line-708"> flush(1);</span>
<span class="source-line-no">709</span><span id="line-709"></span>
<span class="source-line-no">710</span><span id="line-710"> // Add more data</span>
<span class="source-line-no">711</span><span id="line-711"> this.store.add(new KeyValue(row, family, qf3, 1, (byte[]) null), null);</span>
<span class="source-line-no">712</span><span id="line-712"> this.store.add(new KeyValue(row, family, qf4, 1, (byte[]) null), null);</span>
<span class="source-line-no">713</span><span id="line-713"> // flush</span>
<span class="source-line-no">714</span><span id="line-714"> flush(2);</span>
<span class="source-line-no">715</span><span id="line-715"></span>
<span class="source-line-no">716</span><span id="line-716"> // Add more data</span>
<span class="source-line-no">717</span><span id="line-717"> this.store.add(new KeyValue(row, family, qf5, 1, (byte[]) null), null);</span>
<span class="source-line-no">718</span><span id="line-718"> this.store.add(new KeyValue(row, family, qf6, 1, (byte[]) null), null);</span>
<span class="source-line-no">719</span><span id="line-719"></span>
<span class="source-line-no">720</span><span id="line-720"> // Get</span>
<span class="source-line-no">721</span><span id="line-721"> result = HBaseTestingUtil.getFromStoreFile(store, get.getRow(), qualifiers);</span>
<span class="source-line-no">722</span><span id="line-722"></span>
<span class="source-line-no">723</span><span id="line-723"> // Need to sort the result since multiple files</span>
<span class="source-line-no">724</span><span id="line-724"> Collections.sort(result, CellComparatorImpl.COMPARATOR);</span>
<span class="source-line-no">725</span><span id="line-725"></span>
<span class="source-line-no">726</span><span id="line-726"> // Compare</span>
<span class="source-line-no">727</span><span id="line-727"> assertCheck();</span>
<span class="source-line-no">728</span><span id="line-728"> }</span>
<span class="source-line-no">729</span><span id="line-729"></span>
<span class="source-line-no">730</span><span id="line-730"> private void flush(int storeFilessize) throws IOException {</span>
<span class="source-line-no">731</span><span id="line-731"> flushStore(store, id++);</span>
<span class="source-line-no">732</span><span id="line-732"> assertEquals(storeFilessize, this.store.getStorefiles().size());</span>
<span class="source-line-no">733</span><span id="line-733"> assertEquals(0, ((AbstractMemStore) this.store.memstore).getActive().getCellsCount());</span>
<span class="source-line-no">734</span><span id="line-734"> }</span>
<span class="source-line-no">735</span><span id="line-735"></span>
<span class="source-line-no">736</span><span id="line-736"> private void assertCheck() {</span>
<span class="source-line-no">737</span><span id="line-737"> assertEquals(expected.size(), result.size());</span>
<span class="source-line-no">738</span><span id="line-738"> for (int i = 0; i &lt; expected.size(); i++) {</span>
<span class="source-line-no">739</span><span id="line-739"> assertEquals(expected.get(i), result.get(i));</span>
<span class="source-line-no">740</span><span id="line-740"> }</span>
<span class="source-line-no">741</span><span id="line-741"> }</span>
<span class="source-line-no">742</span><span id="line-742"></span>
<span class="source-line-no">743</span><span id="line-743"> @After</span>
<span class="source-line-no">744</span><span id="line-744"> public void tearDown() throws Exception {</span>
<span class="source-line-no">745</span><span id="line-745"> EnvironmentEdgeManagerTestHelper.reset();</span>
<span class="source-line-no">746</span><span id="line-746"> if (store != null) {</span>
<span class="source-line-no">747</span><span id="line-747"> try {</span>
<span class="source-line-no">748</span><span id="line-748"> store.close();</span>
<span class="source-line-no">749</span><span id="line-749"> } catch (IOException e) {</span>
<span class="source-line-no">750</span><span id="line-750"> }</span>
<span class="source-line-no">751</span><span id="line-751"> store = null;</span>
<span class="source-line-no">752</span><span id="line-752"> }</span>
<span class="source-line-no">753</span><span id="line-753"> if (region != null) {</span>
<span class="source-line-no">754</span><span id="line-754"> region.close();</span>
<span class="source-line-no">755</span><span id="line-755"> region = null;</span>
<span class="source-line-no">756</span><span id="line-756"> }</span>
<span class="source-line-no">757</span><span id="line-757"> }</span>
<span class="source-line-no">758</span><span id="line-758"></span>
<span class="source-line-no">759</span><span id="line-759"> @AfterClass</span>
<span class="source-line-no">760</span><span id="line-760"> public static void tearDownAfterClass() throws IOException {</span>
<span class="source-line-no">761</span><span id="line-761"> TEST_UTIL.cleanupTestDir();</span>
<span class="source-line-no">762</span><span id="line-762"> }</span>
<span class="source-line-no">763</span><span id="line-763"></span>
<span class="source-line-no">764</span><span id="line-764"> @Test</span>
<span class="source-line-no">765</span><span id="line-765"> public void testHandleErrorsInFlush() throws Exception {</span>
<span class="source-line-no">766</span><span id="line-766"> LOG.info("Setting up a faulty file system that cannot write");</span>
<span class="source-line-no">767</span><span id="line-767"></span>
<span class="source-line-no">768</span><span id="line-768"> final Configuration conf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());</span>
<span class="source-line-no">769</span><span id="line-769"> User user = User.createUserForTesting(conf, "testhandleerrorsinflush", new String[] { "foo" });</span>
<span class="source-line-no">770</span><span id="line-770"> // Inject our faulty LocalFileSystem</span>
<span class="source-line-no">771</span><span id="line-771"> conf.setClass("fs.file.impl", FaultyFileSystem.class, FileSystem.class);</span>
<span class="source-line-no">772</span><span id="line-772"> user.runAs(new PrivilegedExceptionAction&lt;Object&gt;() {</span>
<span class="source-line-no">773</span><span id="line-773"> @Override</span>
<span class="source-line-no">774</span><span id="line-774"> public Object run() throws Exception {</span>
<span class="source-line-no">775</span><span id="line-775"> // Make sure it worked (above is sensitive to caching details in hadoop core)</span>
<span class="source-line-no">776</span><span id="line-776"> FileSystem fs = FileSystem.get(conf);</span>
<span class="source-line-no">777</span><span id="line-777"> assertEquals(FaultyFileSystem.class, fs.getClass());</span>
<span class="source-line-no">778</span><span id="line-778"></span>
<span class="source-line-no">779</span><span id="line-779"> // Initialize region</span>
<span class="source-line-no">780</span><span id="line-780"> init(name.getMethodName(), conf);</span>
<span class="source-line-no">781</span><span id="line-781"></span>
<span class="source-line-no">782</span><span id="line-782"> LOG.info("Adding some data");</span>
<span class="source-line-no">783</span><span id="line-783"> store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), null);</span>
<span class="source-line-no">784</span><span id="line-784"> store.add(new KeyValue(row, family, qf2, 1, (byte[]) null), null);</span>
<span class="source-line-no">785</span><span id="line-785"> store.add(new KeyValue(row, family, qf3, 1, (byte[]) null), null);</span>
<span class="source-line-no">786</span><span id="line-786"></span>
<span class="source-line-no">787</span><span id="line-787"> LOG.info("Before flush, we should have no files");</span>
<span class="source-line-no">788</span><span id="line-788"></span>
<span class="source-line-no">789</span><span id="line-789"> Collection&lt;StoreFileInfo&gt; files =</span>
<span class="source-line-no">790</span><span id="line-790"> store.getRegionFileSystem().getStoreFiles(store.getColumnFamilyName());</span>
<span class="source-line-no">791</span><span id="line-791"> assertEquals(0, files != null ? files.size() : 0);</span>
<span class="source-line-no">792</span><span id="line-792"></span>
<span class="source-line-no">793</span><span id="line-793"> // flush</span>
<span class="source-line-no">794</span><span id="line-794"> try {</span>
<span class="source-line-no">795</span><span id="line-795"> LOG.info("Flushing");</span>
<span class="source-line-no">796</span><span id="line-796"> flush(1);</span>
<span class="source-line-no">797</span><span id="line-797"> fail("Didn't bubble up IOE!");</span>
<span class="source-line-no">798</span><span id="line-798"> } catch (IOException ioe) {</span>
<span class="source-line-no">799</span><span id="line-799"> assertTrue(ioe.getMessage().contains("Fault injected"));</span>
<span class="source-line-no">800</span><span id="line-800"> }</span>
<span class="source-line-no">801</span><span id="line-801"></span>
<span class="source-line-no">802</span><span id="line-802"> LOG.info("After failed flush, we should still have no files!");</span>
<span class="source-line-no">803</span><span id="line-803"> files = store.getRegionFileSystem().getStoreFiles(store.getColumnFamilyName());</span>
<span class="source-line-no">804</span><span id="line-804"> assertEquals(0, files != null ? files.size() : 0);</span>
<span class="source-line-no">805</span><span id="line-805"> store.getHRegion().getWAL().close();</span>
<span class="source-line-no">806</span><span id="line-806"> return null;</span>
<span class="source-line-no">807</span><span id="line-807"> }</span>
<span class="source-line-no">808</span><span id="line-808"> });</span>
<span class="source-line-no">809</span><span id="line-809"> FileSystem.closeAllForUGI(user.getUGI());</span>
<span class="source-line-no">810</span><span id="line-810"> }</span>
<span class="source-line-no">811</span><span id="line-811"></span>
<span class="source-line-no">812</span><span id="line-812"> /**</span>
<span class="source-line-no">813</span><span id="line-813"> * Faulty file system that will fail if you write past its fault position the FIRST TIME only;</span>
<span class="source-line-no">814</span><span id="line-814"> * thereafter it will succeed. Used by {@link TestHRegion} too.</span>
<span class="source-line-no">815</span><span id="line-815"> */</span>
<span class="source-line-no">816</span><span id="line-816"> static class FaultyFileSystem extends FilterFileSystem {</span>
<span class="source-line-no">817</span><span id="line-817"> List&lt;SoftReference&lt;FaultyOutputStream&gt;&gt; outStreams = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">818</span><span id="line-818"> private long faultPos = 200;</span>
<span class="source-line-no">819</span><span id="line-819"> AtomicBoolean fault = new AtomicBoolean(true);</span>
<span class="source-line-no">820</span><span id="line-820"></span>
<span class="source-line-no">821</span><span id="line-821"> public FaultyFileSystem() {</span>
<span class="source-line-no">822</span><span id="line-822"> super(new LocalFileSystem());</span>
<span class="source-line-no">823</span><span id="line-823"> LOG.info("Creating faulty!");</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"> @Override</span>
<span class="source-line-no">827</span><span id="line-827"> public FSDataOutputStream create(Path p) throws IOException {</span>
<span class="source-line-no">828</span><span id="line-828"> return new FaultyOutputStream(super.create(p), faultPos, fault);</span>
<span class="source-line-no">829</span><span id="line-829"> }</span>
<span class="source-line-no">830</span><span id="line-830"></span>
<span class="source-line-no">831</span><span id="line-831"> @Override</span>
<span class="source-line-no">832</span><span id="line-832"> public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite,</span>
<span class="source-line-no">833</span><span id="line-833"> int bufferSize, short replication, long blockSize, Progressable progress) throws IOException {</span>
<span class="source-line-no">834</span><span id="line-834"> return new FaultyOutputStream(</span>
<span class="source-line-no">835</span><span id="line-835"> super.create(f, permission, overwrite, bufferSize, replication, blockSize, progress),</span>
<span class="source-line-no">836</span><span id="line-836"> faultPos, fault);</span>
<span class="source-line-no">837</span><span id="line-837"> }</span>
<span class="source-line-no">838</span><span id="line-838"></span>
<span class="source-line-no">839</span><span id="line-839"> @Override</span>
<span class="source-line-no">840</span><span id="line-840"> public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, int bufferSize,</span>
<span class="source-line-no">841</span><span id="line-841"> short replication, long blockSize, Progressable progress) throws IOException {</span>
<span class="source-line-no">842</span><span id="line-842"> // Fake it. Call create instead. The default implementation throws an IOE</span>
<span class="source-line-no">843</span><span id="line-843"> // that this is not supported.</span>
<span class="source-line-no">844</span><span id="line-844"> return create(f, overwrite, bufferSize, replication, blockSize, progress);</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"> static class FaultyOutputStream extends FSDataOutputStream {</span>
<span class="source-line-no">849</span><span id="line-849"> volatile long faultPos = Long.MAX_VALUE;</span>
<span class="source-line-no">850</span><span id="line-850"> private final AtomicBoolean fault;</span>
<span class="source-line-no">851</span><span id="line-851"></span>
<span class="source-line-no">852</span><span id="line-852"> public FaultyOutputStream(FSDataOutputStream out, long faultPos, final AtomicBoolean fault)</span>
<span class="source-line-no">853</span><span id="line-853"> throws IOException {</span>
<span class="source-line-no">854</span><span id="line-854"> super(out, null);</span>
<span class="source-line-no">855</span><span id="line-855"> this.faultPos = faultPos;</span>
<span class="source-line-no">856</span><span id="line-856"> this.fault = fault;</span>
<span class="source-line-no">857</span><span id="line-857"> }</span>
<span class="source-line-no">858</span><span id="line-858"></span>
<span class="source-line-no">859</span><span id="line-859"> @Override</span>
<span class="source-line-no">860</span><span id="line-860"> public synchronized void write(byte[] buf, int offset, int length) throws IOException {</span>
<span class="source-line-no">861</span><span id="line-861"> LOG.info("faulty stream write at pos " + getPos());</span>
<span class="source-line-no">862</span><span id="line-862"> injectFault();</span>
<span class="source-line-no">863</span><span id="line-863"> super.write(buf, offset, length);</span>
<span class="source-line-no">864</span><span id="line-864"> }</span>
<span class="source-line-no">865</span><span id="line-865"></span>
<span class="source-line-no">866</span><span id="line-866"> private void injectFault() throws IOException {</span>
<span class="source-line-no">867</span><span id="line-867"> if (this.fault.get() &amp;&amp; getPos() &gt;= faultPos) {</span>
<span class="source-line-no">868</span><span id="line-868"> throw new IOException("Fault injected");</span>
<span class="source-line-no">869</span><span id="line-869"> }</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"></span>
<span class="source-line-no">873</span><span id="line-873"> private static StoreFlushContext flushStore(HStore store, long id) throws IOException {</span>
<span class="source-line-no">874</span><span id="line-874"> StoreFlushContext storeFlushCtx = store.createFlushContext(id, FlushLifeCycleTracker.DUMMY);</span>
<span class="source-line-no">875</span><span id="line-875"> storeFlushCtx.prepare();</span>
<span class="source-line-no">876</span><span id="line-876"> storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));</span>
<span class="source-line-no">877</span><span id="line-877"> storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));</span>
<span class="source-line-no">878</span><span id="line-878"> return storeFlushCtx;</span>
<span class="source-line-no">879</span><span id="line-879"> }</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"> * Generate a list of KeyValues for testing based on given parameters</span>
<span class="source-line-no">883</span><span id="line-883"> * @return the rows key-value list</span>
<span class="source-line-no">884</span><span id="line-884"> */</span>
<span class="source-line-no">885</span><span id="line-885"> private List&lt;ExtendedCell&gt; getKeyValueSet(long[] timestamps, int numRows, byte[] qualifier,</span>
<span class="source-line-no">886</span><span id="line-886"> byte[] family) {</span>
<span class="source-line-no">887</span><span id="line-887"> List&lt;ExtendedCell&gt; kvList = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">888</span><span id="line-888"> for (int i = 1; i &lt;= numRows; i++) {</span>
<span class="source-line-no">889</span><span id="line-889"> byte[] b = Bytes.toBytes(i);</span>
<span class="source-line-no">890</span><span id="line-890"> for (long timestamp : timestamps) {</span>
<span class="source-line-no">891</span><span id="line-891"> kvList.add(new KeyValue(b, family, qualifier, timestamp, b));</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"> return kvList;</span>
<span class="source-line-no">895</span><span id="line-895"> }</span>
<span class="source-line-no">896</span><span id="line-896"></span>
<span class="source-line-no">897</span><span id="line-897"> /**</span>
<span class="source-line-no">898</span><span id="line-898"> * Test to ensure correctness when using Stores with multiple timestamps</span>
<span class="source-line-no">899</span><span id="line-899"> */</span>
<span class="source-line-no">900</span><span id="line-900"> @Test</span>
<span class="source-line-no">901</span><span id="line-901"> public void testMultipleTimestamps() throws IOException {</span>
<span class="source-line-no">902</span><span id="line-902"> int numRows = 1;</span>
<span class="source-line-no">903</span><span id="line-903"> long[] timestamps1 = new long[] { 1, 5, 10, 20 };</span>
<span class="source-line-no">904</span><span id="line-904"> long[] timestamps2 = new long[] { 30, 80 };</span>
<span class="source-line-no">905</span><span id="line-905"></span>
<span class="source-line-no">906</span><span id="line-906"> init(this.name.getMethodName());</span>
<span class="source-line-no">907</span><span id="line-907"></span>
<span class="source-line-no">908</span><span id="line-908"> List&lt;ExtendedCell&gt; kvList1 = getKeyValueSet(timestamps1, numRows, qf1, family);</span>
<span class="source-line-no">909</span><span id="line-909"> for (ExtendedCell kv : kvList1) {</span>
<span class="source-line-no">910</span><span id="line-910"> this.store.add(kv, null);</span>
<span class="source-line-no">911</span><span id="line-911"> }</span>
<span class="source-line-no">912</span><span id="line-912"></span>
<span class="source-line-no">913</span><span id="line-913"> flushStore(store, id++);</span>
<span class="source-line-no">914</span><span id="line-914"></span>
<span class="source-line-no">915</span><span id="line-915"> List&lt;ExtendedCell&gt; kvList2 = getKeyValueSet(timestamps2, numRows, qf1, family);</span>
<span class="source-line-no">916</span><span id="line-916"> for (ExtendedCell kv : kvList2) {</span>
<span class="source-line-no">917</span><span id="line-917"> this.store.add(kv, null);</span>
<span class="source-line-no">918</span><span id="line-918"> }</span>
<span class="source-line-no">919</span><span id="line-919"></span>
<span class="source-line-no">920</span><span id="line-920"> List&lt;Cell&gt; result;</span>
<span class="source-line-no">921</span><span id="line-921"> Get get = new Get(Bytes.toBytes(1));</span>
<span class="source-line-no">922</span><span id="line-922"> get.addColumn(family, qf1);</span>
<span class="source-line-no">923</span><span id="line-923"></span>
<span class="source-line-no">924</span><span id="line-924"> get.setTimeRange(0, 15);</span>
<span class="source-line-no">925</span><span id="line-925"> result = HBaseTestingUtil.getFromStoreFile(store, get);</span>
<span class="source-line-no">926</span><span id="line-926"> assertTrue(result.size() &gt; 0);</span>
<span class="source-line-no">927</span><span id="line-927"></span>
<span class="source-line-no">928</span><span id="line-928"> get.setTimeRange(40, 90);</span>
<span class="source-line-no">929</span><span id="line-929"> result = HBaseTestingUtil.getFromStoreFile(store, get);</span>
<span class="source-line-no">930</span><span id="line-930"> assertTrue(result.size() &gt; 0);</span>
<span class="source-line-no">931</span><span id="line-931"></span>
<span class="source-line-no">932</span><span id="line-932"> get.setTimeRange(10, 45);</span>
<span class="source-line-no">933</span><span id="line-933"> result = HBaseTestingUtil.getFromStoreFile(store, get);</span>
<span class="source-line-no">934</span><span id="line-934"> assertTrue(result.size() &gt; 0);</span>
<span class="source-line-no">935</span><span id="line-935"></span>
<span class="source-line-no">936</span><span id="line-936"> get.setTimeRange(80, 145);</span>
<span class="source-line-no">937</span><span id="line-937"> result = HBaseTestingUtil.getFromStoreFile(store, get);</span>
<span class="source-line-no">938</span><span id="line-938"> assertTrue(result.size() &gt; 0);</span>
<span class="source-line-no">939</span><span id="line-939"></span>
<span class="source-line-no">940</span><span id="line-940"> get.setTimeRange(1, 2);</span>
<span class="source-line-no">941</span><span id="line-941"> result = HBaseTestingUtil.getFromStoreFile(store, get);</span>
<span class="source-line-no">942</span><span id="line-942"> assertTrue(result.size() &gt; 0);</span>
<span class="source-line-no">943</span><span id="line-943"></span>
<span class="source-line-no">944</span><span id="line-944"> get.setTimeRange(90, 200);</span>
<span class="source-line-no">945</span><span id="line-945"> result = HBaseTestingUtil.getFromStoreFile(store, get);</span>
<span class="source-line-no">946</span><span id="line-946"> assertTrue(result.size() == 0);</span>
<span class="source-line-no">947</span><span id="line-947"> }</span>
<span class="source-line-no">948</span><span id="line-948"></span>
<span class="source-line-no">949</span><span id="line-949"> /**</span>
<span class="source-line-no">950</span><span id="line-950"> * Test for HBASE-3492 - Test split on empty colfam (no store files).</span>
<span class="source-line-no">951</span><span id="line-951"> * @throws IOException When the IO operations fail.</span>
<span class="source-line-no">952</span><span id="line-952"> */</span>
<span class="source-line-no">953</span><span id="line-953"> @Test</span>
<span class="source-line-no">954</span><span id="line-954"> public void testSplitWithEmptyColFam() throws IOException {</span>
<span class="source-line-no">955</span><span id="line-955"> init(this.name.getMethodName());</span>
<span class="source-line-no">956</span><span id="line-956"> assertFalse(store.getSplitPoint().isPresent());</span>
<span class="source-line-no">957</span><span id="line-957"> }</span>
<span class="source-line-no">958</span><span id="line-958"></span>
<span class="source-line-no">959</span><span id="line-959"> @Test</span>
<span class="source-line-no">960</span><span id="line-960"> public void testStoreUsesConfigurationFromHcdAndHtd() throws Exception {</span>
<span class="source-line-no">961</span><span id="line-961"> final String CONFIG_KEY = "hbase.regionserver.thread.compaction.throttle";</span>
<span class="source-line-no">962</span><span id="line-962"> long anyValue = 10;</span>
<span class="source-line-no">963</span><span id="line-963"></span>
<span class="source-line-no">964</span><span id="line-964"> // We'll check that it uses correct config and propagates it appropriately by going thru</span>
<span class="source-line-no">965</span><span id="line-965"> // the simplest "real" path I can find - "throttleCompaction", which just checks whether</span>
<span class="source-line-no">966</span><span id="line-966"> // a number we pass in is higher than some config value, inside compactionPolicy.</span>
<span class="source-line-no">967</span><span id="line-967"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">968</span><span id="line-968"> conf.setLong(CONFIG_KEY, anyValue);</span>
<span class="source-line-no">969</span><span id="line-969"> init(name.getMethodName() + "-xml", conf);</span>
<span class="source-line-no">970</span><span id="line-970"> assertTrue(store.throttleCompaction(anyValue + 1));</span>
<span class="source-line-no">971</span><span id="line-971"> assertFalse(store.throttleCompaction(anyValue));</span>
<span class="source-line-no">972</span><span id="line-972"></span>
<span class="source-line-no">973</span><span id="line-973"> // HTD overrides XML.</span>
<span class="source-line-no">974</span><span id="line-974"> --anyValue;</span>
<span class="source-line-no">975</span><span id="line-975"> init(</span>
<span class="source-line-no">976</span><span id="line-976"> name.getMethodName() + "-htd", conf, TableDescriptorBuilder</span>
<span class="source-line-no">977</span><span id="line-977"> .newBuilder(TableName.valueOf(table)).setValue(CONFIG_KEY, Long.toString(anyValue)),</span>
<span class="source-line-no">978</span><span id="line-978"> ColumnFamilyDescriptorBuilder.of(family));</span>
<span class="source-line-no">979</span><span id="line-979"> assertTrue(store.throttleCompaction(anyValue + 1));</span>
<span class="source-line-no">980</span><span id="line-980"> assertFalse(store.throttleCompaction(anyValue));</span>
<span class="source-line-no">981</span><span id="line-981"></span>
<span class="source-line-no">982</span><span id="line-982"> // HCD overrides them both.</span>
<span class="source-line-no">983</span><span id="line-983"> --anyValue;</span>
<span class="source-line-no">984</span><span id="line-984"> init(name.getMethodName() + "-hcd", conf,</span>
<span class="source-line-no">985</span><span id="line-985"> TableDescriptorBuilder.newBuilder(TableName.valueOf(table)).setValue(CONFIG_KEY,</span>
<span class="source-line-no">986</span><span id="line-986"> Long.toString(anyValue)),</span>
<span class="source-line-no">987</span><span id="line-987"> ColumnFamilyDescriptorBuilder.newBuilder(family).setValue(CONFIG_KEY, Long.toString(anyValue))</span>
<span class="source-line-no">988</span><span id="line-988"> .build());</span>
<span class="source-line-no">989</span><span id="line-989"> assertTrue(store.throttleCompaction(anyValue + 1));</span>
<span class="source-line-no">990</span><span id="line-990"> assertFalse(store.throttleCompaction(anyValue));</span>
<span class="source-line-no">991</span><span id="line-991"> }</span>
<span class="source-line-no">992</span><span id="line-992"></span>
<span class="source-line-no">993</span><span id="line-993"> public static class DummyStoreEngine extends DefaultStoreEngine {</span>
<span class="source-line-no">994</span><span id="line-994"> public static DefaultCompactor lastCreatedCompactor = null;</span>
<span class="source-line-no">995</span><span id="line-995"></span>
<span class="source-line-no">996</span><span id="line-996"> @Override</span>
<span class="source-line-no">997</span><span id="line-997"> protected void createComponents(Configuration conf, HStore store, CellComparator comparator)</span>
<span class="source-line-no">998</span><span id="line-998"> throws IOException {</span>
<span class="source-line-no">999</span><span id="line-999"> super.createComponents(conf, store, comparator);</span>
<span class="source-line-no">1000</span><span id="line-1000"> lastCreatedCompactor = this.compactor;</span>
<span class="source-line-no">1001</span><span id="line-1001"> }</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"> @Test</span>
<span class="source-line-no">1005</span><span id="line-1005"> public void testStoreUsesSearchEngineOverride() throws Exception {</span>
<span class="source-line-no">1006</span><span id="line-1006"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">1007</span><span id="line-1007"> conf.set(StoreEngine.STORE_ENGINE_CLASS_KEY, DummyStoreEngine.class.getName());</span>
<span class="source-line-no">1008</span><span id="line-1008"> init(this.name.getMethodName(), conf);</span>
<span class="source-line-no">1009</span><span id="line-1009"> assertEquals(DummyStoreEngine.lastCreatedCompactor, this.store.storeEngine.getCompactor());</span>
<span class="source-line-no">1010</span><span id="line-1010"> }</span>
<span class="source-line-no">1011</span><span id="line-1011"></span>
<span class="source-line-no">1012</span><span id="line-1012"> private void addStoreFile() throws IOException {</span>
<span class="source-line-no">1013</span><span id="line-1013"> HStoreFile f = this.store.getStorefiles().iterator().next();</span>
<span class="source-line-no">1014</span><span id="line-1014"> Path storedir = f.getPath().getParent();</span>
<span class="source-line-no">1015</span><span id="line-1015"> long seqid = this.store.getMaxSequenceId().orElse(0L);</span>
<span class="source-line-no">1016</span><span id="line-1016"> Configuration c = TEST_UTIL.getConfiguration();</span>
<span class="source-line-no">1017</span><span id="line-1017"> FileSystem fs = FileSystem.get(c);</span>
<span class="source-line-no">1018</span><span id="line-1018"> HFileContext fileContext = new HFileContextBuilder().withBlockSize(BLOCKSIZE_SMALL).build();</span>
<span class="source-line-no">1019</span><span id="line-1019"> StoreFileWriter w = new StoreFileWriter.Builder(c, new CacheConfig(c), fs)</span>
<span class="source-line-no">1020</span><span id="line-1020"> .withOutputDir(storedir).withFileContext(fileContext).build();</span>
<span class="source-line-no">1021</span><span id="line-1021"> w.appendMetadata(seqid + 1, false);</span>
<span class="source-line-no">1022</span><span id="line-1022"> w.close();</span>
<span class="source-line-no">1023</span><span id="line-1023"> LOG.info("Added store file:" + w.getPath());</span>
<span class="source-line-no">1024</span><span id="line-1024"> }</span>
<span class="source-line-no">1025</span><span id="line-1025"></span>
<span class="source-line-no">1026</span><span id="line-1026"> private void archiveStoreFile(int index) throws IOException {</span>
<span class="source-line-no">1027</span><span id="line-1027"> Collection&lt;HStoreFile&gt; files = this.store.getStorefiles();</span>
<span class="source-line-no">1028</span><span id="line-1028"> HStoreFile sf = null;</span>
<span class="source-line-no">1029</span><span id="line-1029"> Iterator&lt;HStoreFile&gt; it = files.iterator();</span>
<span class="source-line-no">1030</span><span id="line-1030"> for (int i = 0; i &lt;= index; i++) {</span>
<span class="source-line-no">1031</span><span id="line-1031"> sf = it.next();</span>
<span class="source-line-no">1032</span><span id="line-1032"> }</span>
<span class="source-line-no">1033</span><span id="line-1033"> store.getRegionFileSystem().removeStoreFiles(store.getColumnFamilyName(),</span>
<span class="source-line-no">1034</span><span id="line-1034"> Lists.newArrayList(sf));</span>
<span class="source-line-no">1035</span><span id="line-1035"> }</span>
<span class="source-line-no">1036</span><span id="line-1036"></span>
<span class="source-line-no">1037</span><span id="line-1037"> private void closeCompactedFile(int index) throws IOException {</span>
<span class="source-line-no">1038</span><span id="line-1038"> Collection&lt;HStoreFile&gt; files =</span>
<span class="source-line-no">1039</span><span id="line-1039"> this.store.getStoreEngine().getStoreFileManager().getCompactedfiles();</span>
<span class="source-line-no">1040</span><span id="line-1040"> if (files.size() &gt; 0) {</span>
<span class="source-line-no">1041</span><span id="line-1041"> HStoreFile sf = null;</span>
<span class="source-line-no">1042</span><span id="line-1042"> Iterator&lt;HStoreFile&gt; it = files.iterator();</span>
<span class="source-line-no">1043</span><span id="line-1043"> for (int i = 0; i &lt;= index; i++) {</span>
<span class="source-line-no">1044</span><span id="line-1044"> sf = it.next();</span>
<span class="source-line-no">1045</span><span id="line-1045"> }</span>
<span class="source-line-no">1046</span><span id="line-1046"> sf.closeStoreFile(true);</span>
<span class="source-line-no">1047</span><span id="line-1047"> store.getStoreEngine().getStoreFileManager()</span>
<span class="source-line-no">1048</span><span id="line-1048"> .removeCompactedFiles(Collections.singletonList(sf));</span>
<span class="source-line-no">1049</span><span id="line-1049"> }</span>
<span class="source-line-no">1050</span><span id="line-1050"> }</span>
<span class="source-line-no">1051</span><span id="line-1051"></span>
<span class="source-line-no">1052</span><span id="line-1052"> @Test</span>
<span class="source-line-no">1053</span><span id="line-1053"> public void testRefreshStoreFiles() throws Exception {</span>
<span class="source-line-no">1054</span><span id="line-1054"> init(name.getMethodName());</span>
<span class="source-line-no">1055</span><span id="line-1055"></span>
<span class="source-line-no">1056</span><span id="line-1056"> assertEquals(0, this.store.getStorefilesCount());</span>
<span class="source-line-no">1057</span><span id="line-1057"></span>
<span class="source-line-no">1058</span><span id="line-1058"> // Test refreshing store files when no store files are there</span>
<span class="source-line-no">1059</span><span id="line-1059"> store.refreshStoreFiles();</span>
<span class="source-line-no">1060</span><span id="line-1060"> assertEquals(0, this.store.getStorefilesCount());</span>
<span class="source-line-no">1061</span><span id="line-1061"></span>
<span class="source-line-no">1062</span><span id="line-1062"> // add some data, flush</span>
<span class="source-line-no">1063</span><span id="line-1063"> this.store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), null);</span>
<span class="source-line-no">1064</span><span id="line-1064"> flush(1);</span>
<span class="source-line-no">1065</span><span id="line-1065"> assertEquals(1, this.store.getStorefilesCount());</span>
<span class="source-line-no">1066</span><span id="line-1066"></span>
<span class="source-line-no">1067</span><span id="line-1067"> // add one more file</span>
<span class="source-line-no">1068</span><span id="line-1068"> addStoreFile();</span>
<span class="source-line-no">1069</span><span id="line-1069"></span>
<span class="source-line-no">1070</span><span id="line-1070"> assertEquals(1, this.store.getStorefilesCount());</span>
<span class="source-line-no">1071</span><span id="line-1071"> store.refreshStoreFiles();</span>
<span class="source-line-no">1072</span><span id="line-1072"> assertEquals(2, this.store.getStorefilesCount());</span>
<span class="source-line-no">1073</span><span id="line-1073"></span>
<span class="source-line-no">1074</span><span id="line-1074"> // add three more files</span>
<span class="source-line-no">1075</span><span id="line-1075"> addStoreFile();</span>
<span class="source-line-no">1076</span><span id="line-1076"> addStoreFile();</span>
<span class="source-line-no">1077</span><span id="line-1077"> addStoreFile();</span>
<span class="source-line-no">1078</span><span id="line-1078"></span>
<span class="source-line-no">1079</span><span id="line-1079"> assertEquals(2, this.store.getStorefilesCount());</span>
<span class="source-line-no">1080</span><span id="line-1080"> store.refreshStoreFiles();</span>
<span class="source-line-no">1081</span><span id="line-1081"> assertEquals(5, this.store.getStorefilesCount());</span>
<span class="source-line-no">1082</span><span id="line-1082"></span>
<span class="source-line-no">1083</span><span id="line-1083"> closeCompactedFile(0);</span>
<span class="source-line-no">1084</span><span id="line-1084"> archiveStoreFile(0);</span>
<span class="source-line-no">1085</span><span id="line-1085"></span>
<span class="source-line-no">1086</span><span id="line-1086"> assertEquals(5, this.store.getStorefilesCount());</span>
<span class="source-line-no">1087</span><span id="line-1087"> store.refreshStoreFiles();</span>
<span class="source-line-no">1088</span><span id="line-1088"> assertEquals(4, this.store.getStorefilesCount());</span>
<span class="source-line-no">1089</span><span id="line-1089"></span>
<span class="source-line-no">1090</span><span id="line-1090"> archiveStoreFile(0);</span>
<span class="source-line-no">1091</span><span id="line-1091"> archiveStoreFile(1);</span>
<span class="source-line-no">1092</span><span id="line-1092"> archiveStoreFile(2);</span>
<span class="source-line-no">1093</span><span id="line-1093"></span>
<span class="source-line-no">1094</span><span id="line-1094"> assertEquals(4, this.store.getStorefilesCount());</span>
<span class="source-line-no">1095</span><span id="line-1095"> store.refreshStoreFiles();</span>
<span class="source-line-no">1096</span><span id="line-1096"> assertEquals(1, this.store.getStorefilesCount());</span>
<span class="source-line-no">1097</span><span id="line-1097"></span>
<span class="source-line-no">1098</span><span id="line-1098"> archiveStoreFile(0);</span>
<span class="source-line-no">1099</span><span id="line-1099"> store.refreshStoreFiles();</span>
<span class="source-line-no">1100</span><span id="line-1100"> assertEquals(0, this.store.getStorefilesCount());</span>
<span class="source-line-no">1101</span><span id="line-1101"> }</span>
<span class="source-line-no">1102</span><span id="line-1102"></span>
<span class="source-line-no">1103</span><span id="line-1103"> @Test</span>
<span class="source-line-no">1104</span><span id="line-1104"> public void testRefreshStoreFilesNotChanged() throws IOException {</span>
<span class="source-line-no">1105</span><span id="line-1105"> init(name.getMethodName());</span>
<span class="source-line-no">1106</span><span id="line-1106"></span>
<span class="source-line-no">1107</span><span id="line-1107"> assertEquals(0, this.store.getStorefilesCount());</span>
<span class="source-line-no">1108</span><span id="line-1108"></span>
<span class="source-line-no">1109</span><span id="line-1109"> // add some data, flush</span>
<span class="source-line-no">1110</span><span id="line-1110"> this.store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), null);</span>
<span class="source-line-no">1111</span><span id="line-1111"> flush(1);</span>
<span class="source-line-no">1112</span><span id="line-1112"> // add one more file</span>
<span class="source-line-no">1113</span><span id="line-1113"> addStoreFile();</span>
<span class="source-line-no">1114</span><span id="line-1114"></span>
<span class="source-line-no">1115</span><span id="line-1115"> StoreEngine&lt;?, ?, ?, ?&gt; spiedStoreEngine = spy(store.getStoreEngine());</span>
<span class="source-line-no">1116</span><span id="line-1116"></span>
<span class="source-line-no">1117</span><span id="line-1117"> // call first time after files changed</span>
<span class="source-line-no">1118</span><span id="line-1118"> spiedStoreEngine.refreshStoreFiles();</span>
<span class="source-line-no">1119</span><span id="line-1119"> assertEquals(2, this.store.getStorefilesCount());</span>
<span class="source-line-no">1120</span><span id="line-1120"> verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any(), any(), any());</span>
<span class="source-line-no">1121</span><span id="line-1121"></span>
<span class="source-line-no">1122</span><span id="line-1122"> // call second time</span>
<span class="source-line-no">1123</span><span id="line-1123"> spiedStoreEngine.refreshStoreFiles();</span>
<span class="source-line-no">1124</span><span id="line-1124"></span>
<span class="source-line-no">1125</span><span id="line-1125"> // ensure that replaceStoreFiles is not called, i.e, the times does not change, if files are not</span>
<span class="source-line-no">1126</span><span id="line-1126"> // refreshed,</span>
<span class="source-line-no">1127</span><span id="line-1127"> verify(spiedStoreEngine, times(1)).replaceStoreFiles(any(), any(), any(), any());</span>
<span class="source-line-no">1128</span><span id="line-1128"> }</span>
<span class="source-line-no">1129</span><span id="line-1129"></span>
<span class="source-line-no">1130</span><span id="line-1130"> @Test</span>
<span class="source-line-no">1131</span><span id="line-1131"> public void testScanWithCompactionAfterFlush() throws Exception {</span>
<span class="source-line-no">1132</span><span id="line-1132"> TEST_UTIL.getConfiguration().set(DEFAULT_COMPACTION_POLICY_CLASS_KEY,</span>
<span class="source-line-no">1133</span><span id="line-1133"> EverythingPolicy.class.getName());</span>
<span class="source-line-no">1134</span><span id="line-1134"> init(name.getMethodName());</span>
<span class="source-line-no">1135</span><span id="line-1135"></span>
<span class="source-line-no">1136</span><span id="line-1136"> assertEquals(0, this.store.getStorefilesCount());</span>
<span class="source-line-no">1137</span><span id="line-1137"></span>
<span class="source-line-no">1138</span><span id="line-1138"> KeyValue kv = new KeyValue(row, family, qf1, 1, (byte[]) null);</span>
<span class="source-line-no">1139</span><span id="line-1139"> // add some data, flush</span>
<span class="source-line-no">1140</span><span id="line-1140"> this.store.add(kv, null);</span>
<span class="source-line-no">1141</span><span id="line-1141"> flush(1);</span>
<span class="source-line-no">1142</span><span id="line-1142"> kv = new KeyValue(row, family, qf2, 1, (byte[]) null);</span>
<span class="source-line-no">1143</span><span id="line-1143"> // add some data, flush</span>
<span class="source-line-no">1144</span><span id="line-1144"> this.store.add(kv, null);</span>
<span class="source-line-no">1145</span><span id="line-1145"> flush(2);</span>
<span class="source-line-no">1146</span><span id="line-1146"> kv = new KeyValue(row, family, qf3, 1, (byte[]) null);</span>
<span class="source-line-no">1147</span><span id="line-1147"> // add some data, flush</span>
<span class="source-line-no">1148</span><span id="line-1148"> this.store.add(kv, null);</span>
<span class="source-line-no">1149</span><span id="line-1149"> flush(3);</span>
<span class="source-line-no">1150</span><span id="line-1150"></span>
<span class="source-line-no">1151</span><span id="line-1151"> ExecutorService service = Executors.newFixedThreadPool(2);</span>
<span class="source-line-no">1152</span><span id="line-1152"></span>
<span class="source-line-no">1153</span><span id="line-1153"> Scan scan = new Scan(new Get(row));</span>
<span class="source-line-no">1154</span><span id="line-1154"> Future&lt;KeyValueScanner&gt; scanFuture = service.submit(() -&gt; {</span>
<span class="source-line-no">1155</span><span id="line-1155"> try {</span>
<span class="source-line-no">1156</span><span id="line-1156"> LOG.info("&gt;&gt;&gt;&gt; creating scanner");</span>
<span class="source-line-no">1157</span><span id="line-1157"> return this.store.createScanner(scan,</span>
<span class="source-line-no">1158</span><span id="line-1158"> new ScanInfo(HBaseConfiguration.create(),</span>
<span class="source-line-no">1159</span><span id="line-1159"> ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(4).build(),</span>
<span class="source-line-no">1160</span><span id="line-1160"> Long.MAX_VALUE, 0, CellComparator.getInstance()),</span>
<span class="source-line-no">1161</span><span id="line-1161"> scan.getFamilyMap().get(store.getColumnFamilyDescriptor().getName()), 0);</span>
<span class="source-line-no">1162</span><span id="line-1162"> } catch (IOException e) {</span>
<span class="source-line-no">1163</span><span id="line-1163"> e.printStackTrace();</span>
<span class="source-line-no">1164</span><span id="line-1164"> return null;</span>
<span class="source-line-no">1165</span><span id="line-1165"> }</span>
<span class="source-line-no">1166</span><span id="line-1166"> });</span>
<span class="source-line-no">1167</span><span id="line-1167"> Future compactFuture = service.submit(() -&gt; {</span>
<span class="source-line-no">1168</span><span id="line-1168"> try {</span>
<span class="source-line-no">1169</span><span id="line-1169"> LOG.info("&gt;&gt;&gt;&gt;&gt;&gt; starting compaction");</span>
<span class="source-line-no">1170</span><span id="line-1170"> Optional&lt;CompactionContext&gt; opCompaction = this.store.requestCompaction();</span>
<span class="source-line-no">1171</span><span id="line-1171"> assertTrue(opCompaction.isPresent());</span>
<span class="source-line-no">1172</span><span id="line-1172"> store.compact(opCompaction.get(), new NoLimitThroughputController(), User.getCurrent());</span>
<span class="source-line-no">1173</span><span id="line-1173"> LOG.info("&gt;&gt;&gt;&gt;&gt;&gt; Compaction is finished");</span>
<span class="source-line-no">1174</span><span id="line-1174"> this.store.closeAndArchiveCompactedFiles();</span>
<span class="source-line-no">1175</span><span id="line-1175"> LOG.info("&gt;&gt;&gt;&gt;&gt;&gt; Compacted files deleted");</span>
<span class="source-line-no">1176</span><span id="line-1176"> } catch (IOException e) {</span>
<span class="source-line-no">1177</span><span id="line-1177"> e.printStackTrace();</span>
<span class="source-line-no">1178</span><span id="line-1178"> }</span>
<span class="source-line-no">1179</span><span id="line-1179"> });</span>
<span class="source-line-no">1180</span><span id="line-1180"></span>
<span class="source-line-no">1181</span><span id="line-1181"> KeyValueScanner kvs = scanFuture.get();</span>
<span class="source-line-no">1182</span><span id="line-1182"> compactFuture.get();</span>
<span class="source-line-no">1183</span><span id="line-1183"> ((StoreScanner) kvs).currentScanners.forEach(s -&gt; {</span>
<span class="source-line-no">1184</span><span id="line-1184"> if (s instanceof StoreFileScanner) {</span>
<span class="source-line-no">1185</span><span id="line-1185"> assertEquals(1, ((StoreFileScanner) s).getReader().getRefCount());</span>
<span class="source-line-no">1186</span><span id="line-1186"> }</span>
<span class="source-line-no">1187</span><span id="line-1187"> });</span>
<span class="source-line-no">1188</span><span id="line-1188"> kvs.seek(kv);</span>
<span class="source-line-no">1189</span><span id="line-1189"> service.shutdownNow();</span>
<span class="source-line-no">1190</span><span id="line-1190"> }</span>
<span class="source-line-no">1191</span><span id="line-1191"></span>
<span class="source-line-no">1192</span><span id="line-1192"> private long countMemStoreScanner(StoreScanner scanner) {</span>
<span class="source-line-no">1193</span><span id="line-1193"> if (scanner.currentScanners == null) {</span>
<span class="source-line-no">1194</span><span id="line-1194"> return 0;</span>
<span class="source-line-no">1195</span><span id="line-1195"> }</span>
<span class="source-line-no">1196</span><span id="line-1196"> return scanner.currentScanners.stream().filter(s -&gt; !s.isFileScanner()).count();</span>
<span class="source-line-no">1197</span><span id="line-1197"> }</span>
<span class="source-line-no">1198</span><span id="line-1198"></span>
<span class="source-line-no">1199</span><span id="line-1199"> @Test</span>
<span class="source-line-no">1200</span><span id="line-1200"> public void testNumberOfMemStoreScannersAfterFlush() throws IOException {</span>
<span class="source-line-no">1201</span><span id="line-1201"> long seqId = 100;</span>
<span class="source-line-no">1202</span><span id="line-1202"> long timestamp = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">1203</span><span id="line-1203"> ExtendedCell cell0 =</span>
<span class="source-line-no">1204</span><span id="line-1204"> ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row).setFamily(family)</span>
<span class="source-line-no">1205</span><span id="line-1205"> .setQualifier(qf1).setTimestamp(timestamp).setType(Cell.Type.Put).setValue(qf1).build();</span>
<span class="source-line-no">1206</span><span id="line-1206"> PrivateCellUtil.setSequenceId(cell0, seqId);</span>
<span class="source-line-no">1207</span><span id="line-1207"> testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Collections.emptyList());</span>
<span class="source-line-no">1208</span><span id="line-1208"></span>
<span class="source-line-no">1209</span><span id="line-1209"> ExtendedCell cell1 =</span>
<span class="source-line-no">1210</span><span id="line-1210"> ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row).setFamily(family)</span>
<span class="source-line-no">1211</span><span id="line-1211"> .setQualifier(qf2).setTimestamp(timestamp).setType(Cell.Type.Put).setValue(qf1).build();</span>
<span class="source-line-no">1212</span><span id="line-1212"> PrivateCellUtil.setSequenceId(cell1, seqId);</span>
<span class="source-line-no">1213</span><span id="line-1213"> testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1));</span>
<span class="source-line-no">1214</span><span id="line-1214"></span>
<span class="source-line-no">1215</span><span id="line-1215"> seqId = 101;</span>
<span class="source-line-no">1216</span><span id="line-1216"> timestamp = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">1217</span><span id="line-1217"> ExtendedCell cell2 =</span>
<span class="source-line-no">1218</span><span id="line-1218"> ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row2).setFamily(family)</span>
<span class="source-line-no">1219</span><span id="line-1219"> .setQualifier(qf2).setTimestamp(timestamp).setType(Cell.Type.Put).setValue(qf1).build();</span>
<span class="source-line-no">1220</span><span id="line-1220"> PrivateCellUtil.setSequenceId(cell2, seqId);</span>
<span class="source-line-no">1221</span><span id="line-1221"> testNumberOfMemStoreScannersAfterFlush(Arrays.asList(cell0), Arrays.asList(cell1, cell2));</span>
<span class="source-line-no">1222</span><span id="line-1222"> }</span>
<span class="source-line-no">1223</span><span id="line-1223"></span>
<span class="source-line-no">1224</span><span id="line-1224"> private void testNumberOfMemStoreScannersAfterFlush(List&lt;ExtendedCell&gt; inputCellsBeforeSnapshot,</span>
<span class="source-line-no">1225</span><span id="line-1225"> List&lt;ExtendedCell&gt; inputCellsAfterSnapshot) throws IOException {</span>
<span class="source-line-no">1226</span><span id="line-1226"> init(this.name.getMethodName() + "-" + inputCellsBeforeSnapshot.size());</span>
<span class="source-line-no">1227</span><span id="line-1227"> TreeSet&lt;byte[]&gt; quals = new TreeSet&lt;&gt;(Bytes.BYTES_COMPARATOR);</span>
<span class="source-line-no">1228</span><span id="line-1228"> long seqId = Long.MIN_VALUE;</span>
<span class="source-line-no">1229</span><span id="line-1229"> for (ExtendedCell c : inputCellsBeforeSnapshot) {</span>
<span class="source-line-no">1230</span><span id="line-1230"> quals.add(CellUtil.cloneQualifier(c));</span>
<span class="source-line-no">1231</span><span id="line-1231"> seqId = Math.max(seqId, c.getSequenceId());</span>
<span class="source-line-no">1232</span><span id="line-1232"> }</span>
<span class="source-line-no">1233</span><span id="line-1233"> for (ExtendedCell c : inputCellsAfterSnapshot) {</span>
<span class="source-line-no">1234</span><span id="line-1234"> quals.add(CellUtil.cloneQualifier(c));</span>
<span class="source-line-no">1235</span><span id="line-1235"> seqId = Math.max(seqId, c.getSequenceId());</span>
<span class="source-line-no">1236</span><span id="line-1236"> }</span>
<span class="source-line-no">1237</span><span id="line-1237"> inputCellsBeforeSnapshot.forEach(c -&gt; store.add(c, null));</span>
<span class="source-line-no">1238</span><span id="line-1238"> StoreFlushContext storeFlushCtx = store.createFlushContext(id++, FlushLifeCycleTracker.DUMMY);</span>
<span class="source-line-no">1239</span><span id="line-1239"> storeFlushCtx.prepare();</span>
<span class="source-line-no">1240</span><span id="line-1240"> inputCellsAfterSnapshot.forEach(c -&gt; store.add(c, null));</span>
<span class="source-line-no">1241</span><span id="line-1241"> int numberOfMemScannersBeforeFlush = inputCellsAfterSnapshot.isEmpty() ? 1 : 2;</span>
<span class="source-line-no">1242</span><span id="line-1242"> try (StoreScanner s = (StoreScanner) store.getScanner(new Scan(), quals, seqId)) {</span>
<span class="source-line-no">1243</span><span id="line-1243"> // snapshot + active (if inputCellsAfterSnapshot isn't empty)</span>
<span class="source-line-no">1244</span><span id="line-1244"> assertEquals(numberOfMemScannersBeforeFlush, countMemStoreScanner(s));</span>
<span class="source-line-no">1245</span><span id="line-1245"> storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));</span>
<span class="source-line-no">1246</span><span id="line-1246"> storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));</span>
<span class="source-line-no">1247</span><span id="line-1247"> // snapshot has no data after flush</span>
<span class="source-line-no">1248</span><span id="line-1248"> int numberOfMemScannersAfterFlush = inputCellsAfterSnapshot.isEmpty() ? 0 : 1;</span>
<span class="source-line-no">1249</span><span id="line-1249"> boolean more;</span>
<span class="source-line-no">1250</span><span id="line-1250"> int cellCount = 0;</span>
<span class="source-line-no">1251</span><span id="line-1251"> do {</span>
<span class="source-line-no">1252</span><span id="line-1252"> List&lt;Cell&gt; cells = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">1253</span><span id="line-1253"> more = s.next(cells);</span>
<span class="source-line-no">1254</span><span id="line-1254"> cellCount += cells.size();</span>
<span class="source-line-no">1255</span><span id="line-1255"> assertEquals(more ? numberOfMemScannersAfterFlush : 0, countMemStoreScanner(s));</span>
<span class="source-line-no">1256</span><span id="line-1256"> } while (more);</span>
<span class="source-line-no">1257</span><span id="line-1257"> assertEquals(</span>
<span class="source-line-no">1258</span><span id="line-1258"> "The number of cells added before snapshot is " + inputCellsBeforeSnapshot.size()</span>
<span class="source-line-no">1259</span><span id="line-1259"> + ", The number of cells added after snapshot is " + inputCellsAfterSnapshot.size(),</span>
<span class="source-line-no">1260</span><span id="line-1260"> inputCellsBeforeSnapshot.size() + inputCellsAfterSnapshot.size(), cellCount);</span>
<span class="source-line-no">1261</span><span id="line-1261"> // the current scanners is cleared</span>
<span class="source-line-no">1262</span><span id="line-1262"> assertEquals(0, countMemStoreScanner(s));</span>
<span class="source-line-no">1263</span><span id="line-1263"> }</span>
<span class="source-line-no">1264</span><span id="line-1264"> }</span>
<span class="source-line-no">1265</span><span id="line-1265"></span>
<span class="source-line-no">1266</span><span id="line-1266"> private ExtendedCell createCell(byte[] qualifier, long ts, long sequenceId, byte[] value)</span>
<span class="source-line-no">1267</span><span id="line-1267"> throws IOException {</span>
<span class="source-line-no">1268</span><span id="line-1268"> return createCell(row, qualifier, ts, sequenceId, value);</span>
<span class="source-line-no">1269</span><span id="line-1269"> }</span>
<span class="source-line-no">1270</span><span id="line-1270"></span>
<span class="source-line-no">1271</span><span id="line-1271"> private ExtendedCell createCell(byte[] row, byte[] qualifier, long ts, long sequenceId,</span>
<span class="source-line-no">1272</span><span id="line-1272"> byte[] value) throws IOException {</span>
<span class="source-line-no">1273</span><span id="line-1273"> return ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY).setRow(row)</span>
<span class="source-line-no">1274</span><span id="line-1274"> .setFamily(family).setQualifier(qualifier).setTimestamp(ts).setType(Cell.Type.Put)</span>
<span class="source-line-no">1275</span><span id="line-1275"> .setValue(value).setSequenceId(sequenceId).build();</span>
<span class="source-line-no">1276</span><span id="line-1276"> }</span>
<span class="source-line-no">1277</span><span id="line-1277"></span>
<span class="source-line-no">1278</span><span id="line-1278"> @Test</span>
<span class="source-line-no">1279</span><span id="line-1279"> public void testFlushBeforeCompletingScanWoFilter() throws IOException, InterruptedException {</span>
<span class="source-line-no">1280</span><span id="line-1280"> final AtomicBoolean timeToGoNextRow = new AtomicBoolean(false);</span>
<span class="source-line-no">1281</span><span id="line-1281"> final int expectedSize = 3;</span>
<span class="source-line-no">1282</span><span id="line-1282"> testFlushBeforeCompletingScan(new MyListHook() {</span>
<span class="source-line-no">1283</span><span id="line-1283"> @Override</span>
<span class="source-line-no">1284</span><span id="line-1284"> public void hook(int currentSize) {</span>
<span class="source-line-no">1285</span><span id="line-1285"> if (currentSize == expectedSize - 1) {</span>
<span class="source-line-no">1286</span><span id="line-1286"> try {</span>
<span class="source-line-no">1287</span><span id="line-1287"> flushStore(store, id++);</span>
<span class="source-line-no">1288</span><span id="line-1288"> timeToGoNextRow.set(true);</span>
<span class="source-line-no">1289</span><span id="line-1289"> } catch (IOException e) {</span>
<span class="source-line-no">1290</span><span id="line-1290"> throw new RuntimeException(e);</span>
<span class="source-line-no">1291</span><span id="line-1291"> }</span>
<span class="source-line-no">1292</span><span id="line-1292"> }</span>
<span class="source-line-no">1293</span><span id="line-1293"> }</span>
<span class="source-line-no">1294</span><span id="line-1294"> }, new FilterBase() {</span>
<span class="source-line-no">1295</span><span id="line-1295"> @Override</span>
<span class="source-line-no">1296</span><span id="line-1296"> public Filter.ReturnCode filterCell(final Cell c) throws IOException {</span>
<span class="source-line-no">1297</span><span id="line-1297"> return ReturnCode.INCLUDE;</span>
<span class="source-line-no">1298</span><span id="line-1298"> }</span>
<span class="source-line-no">1299</span><span id="line-1299"> }, expectedSize);</span>
<span class="source-line-no">1300</span><span id="line-1300"> }</span>
<span class="source-line-no">1301</span><span id="line-1301"></span>
<span class="source-line-no">1302</span><span id="line-1302"> @Test</span>
<span class="source-line-no">1303</span><span id="line-1303"> public void testFlushBeforeCompletingScanWithFilter() throws IOException, InterruptedException {</span>
<span class="source-line-no">1304</span><span id="line-1304"> final AtomicBoolean timeToGoNextRow = new AtomicBoolean(false);</span>
<span class="source-line-no">1305</span><span id="line-1305"> final int expectedSize = 2;</span>
<span class="source-line-no">1306</span><span id="line-1306"> testFlushBeforeCompletingScan(new MyListHook() {</span>
<span class="source-line-no">1307</span><span id="line-1307"> @Override</span>
<span class="source-line-no">1308</span><span id="line-1308"> public void hook(int currentSize) {</span>
<span class="source-line-no">1309</span><span id="line-1309"> if (currentSize == expectedSize - 1) {</span>
<span class="source-line-no">1310</span><span id="line-1310"> try {</span>
<span class="source-line-no">1311</span><span id="line-1311"> flushStore(store, id++);</span>
<span class="source-line-no">1312</span><span id="line-1312"> timeToGoNextRow.set(true);</span>
<span class="source-line-no">1313</span><span id="line-1313"> } catch (IOException e) {</span>
<span class="source-line-no">1314</span><span id="line-1314"> throw new RuntimeException(e);</span>
<span class="source-line-no">1315</span><span id="line-1315"> }</span>
<span class="source-line-no">1316</span><span id="line-1316"> }</span>
<span class="source-line-no">1317</span><span id="line-1317"> }</span>
<span class="source-line-no">1318</span><span id="line-1318"> }, new FilterBase() {</span>
<span class="source-line-no">1319</span><span id="line-1319"> @Override</span>
<span class="source-line-no">1320</span><span id="line-1320"> public Filter.ReturnCode filterCell(final Cell c) throws IOException {</span>
<span class="source-line-no">1321</span><span id="line-1321"> if (timeToGoNextRow.get()) {</span>
<span class="source-line-no">1322</span><span id="line-1322"> timeToGoNextRow.set(false);</span>
<span class="source-line-no">1323</span><span id="line-1323"> return ReturnCode.NEXT_ROW;</span>
<span class="source-line-no">1324</span><span id="line-1324"> } else {</span>
<span class="source-line-no">1325</span><span id="line-1325"> return ReturnCode.INCLUDE;</span>
<span class="source-line-no">1326</span><span id="line-1326"> }</span>
<span class="source-line-no">1327</span><span id="line-1327"> }</span>
<span class="source-line-no">1328</span><span id="line-1328"> }, expectedSize);</span>
<span class="source-line-no">1329</span><span id="line-1329"> }</span>
<span class="source-line-no">1330</span><span id="line-1330"></span>
<span class="source-line-no">1331</span><span id="line-1331"> @Test</span>
<span class="source-line-no">1332</span><span id="line-1332"> public void testFlushBeforeCompletingScanWithFilterHint()</span>
<span class="source-line-no">1333</span><span id="line-1333"> throws IOException, InterruptedException {</span>
<span class="source-line-no">1334</span><span id="line-1334"> final AtomicBoolean timeToGetHint = new AtomicBoolean(false);</span>
<span class="source-line-no">1335</span><span id="line-1335"> final int expectedSize = 2;</span>
<span class="source-line-no">1336</span><span id="line-1336"> testFlushBeforeCompletingScan(new MyListHook() {</span>
<span class="source-line-no">1337</span><span id="line-1337"> @Override</span>
<span class="source-line-no">1338</span><span id="line-1338"> public void hook(int currentSize) {</span>
<span class="source-line-no">1339</span><span id="line-1339"> if (currentSize == expectedSize - 1) {</span>
<span class="source-line-no">1340</span><span id="line-1340"> try {</span>
<span class="source-line-no">1341</span><span id="line-1341"> flushStore(store, id++);</span>
<span class="source-line-no">1342</span><span id="line-1342"> timeToGetHint.set(true);</span>
<span class="source-line-no">1343</span><span id="line-1343"> } catch (IOException e) {</span>
<span class="source-line-no">1344</span><span id="line-1344"> throw new RuntimeException(e);</span>
<span class="source-line-no">1345</span><span id="line-1345"> }</span>
<span class="source-line-no">1346</span><span id="line-1346"> }</span>
<span class="source-line-no">1347</span><span id="line-1347"> }</span>
<span class="source-line-no">1348</span><span id="line-1348"> }, new FilterBase() {</span>
<span class="source-line-no">1349</span><span id="line-1349"> @Override</span>
<span class="source-line-no">1350</span><span id="line-1350"> public Filter.ReturnCode filterCell(final Cell c) throws IOException {</span>
<span class="source-line-no">1351</span><span id="line-1351"> if (timeToGetHint.get()) {</span>
<span class="source-line-no">1352</span><span id="line-1352"> timeToGetHint.set(false);</span>
<span class="source-line-no">1353</span><span id="line-1353"> return Filter.ReturnCode.SEEK_NEXT_USING_HINT;</span>
<span class="source-line-no">1354</span><span id="line-1354"> } else {</span>
<span class="source-line-no">1355</span><span id="line-1355"> return Filter.ReturnCode.INCLUDE;</span>
<span class="source-line-no">1356</span><span id="line-1356"> }</span>
<span class="source-line-no">1357</span><span id="line-1357"> }</span>
<span class="source-line-no">1358</span><span id="line-1358"></span>
<span class="source-line-no">1359</span><span id="line-1359"> @Override</span>
<span class="source-line-no">1360</span><span id="line-1360"> public Cell getNextCellHint(Cell currentCell) throws IOException {</span>
<span class="source-line-no">1361</span><span id="line-1361"> return currentCell;</span>
<span class="source-line-no">1362</span><span id="line-1362"> }</span>
<span class="source-line-no">1363</span><span id="line-1363"> }, expectedSize);</span>
<span class="source-line-no">1364</span><span id="line-1364"> }</span>
<span class="source-line-no">1365</span><span id="line-1365"></span>
<span class="source-line-no">1366</span><span id="line-1366"> private void testFlushBeforeCompletingScan(MyListHook hook, Filter filter, int expectedSize)</span>
<span class="source-line-no">1367</span><span id="line-1367"> throws IOException, InterruptedException {</span>
<span class="source-line-no">1368</span><span id="line-1368"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">1369</span><span id="line-1369"> byte[] r0 = Bytes.toBytes("row0");</span>
<span class="source-line-no">1370</span><span id="line-1370"> byte[] r1 = Bytes.toBytes("row1");</span>
<span class="source-line-no">1371</span><span id="line-1371"> byte[] r2 = Bytes.toBytes("row2");</span>
<span class="source-line-no">1372</span><span id="line-1372"> byte[] value0 = Bytes.toBytes("value0");</span>
<span class="source-line-no">1373</span><span id="line-1373"> byte[] value1 = Bytes.toBytes("value1");</span>
<span class="source-line-no">1374</span><span id="line-1374"> byte[] value2 = Bytes.toBytes("value2");</span>
<span class="source-line-no">1375</span><span id="line-1375"> MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();</span>
<span class="source-line-no">1376</span><span id="line-1376"> long ts = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">1377</span><span id="line-1377"> long seqId = 100;</span>
<span class="source-line-no">1378</span><span id="line-1378"> init(name.getMethodName(), conf, TableDescriptorBuilder.newBuilder(TableName.valueOf(table)),</span>
<span class="source-line-no">1379</span><span id="line-1379"> ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(1).build(),</span>
<span class="source-line-no">1380</span><span id="line-1380"> new MyStoreHook() {</span>
<span class="source-line-no">1381</span><span id="line-1381"> @Override</span>
<span class="source-line-no">1382</span><span id="line-1382"> public long getSmallestReadPoint(HStore store) {</span>
<span class="source-line-no">1383</span><span id="line-1383"> return seqId + 3;</span>
<span class="source-line-no">1384</span><span id="line-1384"> }</span>
<span class="source-line-no">1385</span><span id="line-1385"> });</span>
<span class="source-line-no">1386</span><span id="line-1386"> // The cells having the value0 won't be flushed to disk because the value of max version is 1</span>
<span class="source-line-no">1387</span><span id="line-1387"> store.add(createCell(r0, qf1, ts, seqId, value0), memStoreSizing);</span>
<span class="source-line-no">1388</span><span id="line-1388"> store.add(createCell(r0, qf2, ts, seqId, value0), memStoreSizing);</span>
<span class="source-line-no">1389</span><span id="line-1389"> store.add(createCell(r0, qf3, ts, seqId, value0), memStoreSizing);</span>
<span class="source-line-no">1390</span><span id="line-1390"> store.add(createCell(r1, qf1, ts + 1, seqId + 1, value1), memStoreSizing);</span>
<span class="source-line-no">1391</span><span id="line-1391"> store.add(createCell(r1, qf2, ts + 1, seqId + 1, value1), memStoreSizing);</span>
<span class="source-line-no">1392</span><span id="line-1392"> store.add(createCell(r1, qf3, ts + 1, seqId + 1, value1), memStoreSizing);</span>
<span class="source-line-no">1393</span><span id="line-1393"> store.add(createCell(r2, qf1, ts + 2, seqId + 2, value2), memStoreSizing);</span>
<span class="source-line-no">1394</span><span id="line-1394"> store.add(createCell(r2, qf2, ts + 2, seqId + 2, value2), memStoreSizing);</span>
<span class="source-line-no">1395</span><span id="line-1395"> store.add(createCell(r2, qf3, ts + 2, seqId + 2, value2), memStoreSizing);</span>
<span class="source-line-no">1396</span><span id="line-1396"> store.add(createCell(r1, qf1, ts + 3, seqId + 3, value1), memStoreSizing);</span>
<span class="source-line-no">1397</span><span id="line-1397"> store.add(createCell(r1, qf2, ts + 3, seqId + 3, value1), memStoreSizing);</span>
<span class="source-line-no">1398</span><span id="line-1398"> store.add(createCell(r1, qf3, ts + 3, seqId + 3, value1), memStoreSizing);</span>
<span class="source-line-no">1399</span><span id="line-1399"> List&lt;Cell&gt; myList = new MyList&lt;&gt;(hook);</span>
<span class="source-line-no">1400</span><span id="line-1400"> Scan scan = new Scan().withStartRow(r1).setFilter(filter);</span>
<span class="source-line-no">1401</span><span id="line-1401"> try (InternalScanner scanner = (InternalScanner) store.getScanner(scan, null, seqId + 3)) {</span>
<span class="source-line-no">1402</span><span id="line-1402"> // r1</span>
<span class="source-line-no">1403</span><span id="line-1403"> scanner.next(myList);</span>
<span class="source-line-no">1404</span><span id="line-1404"> assertEquals(expectedSize, myList.size());</span>
<span class="source-line-no">1405</span><span id="line-1405"> for (Cell c : myList) {</span>
<span class="source-line-no">1406</span><span id="line-1406"> byte[] actualValue = CellUtil.cloneValue(c);</span>
<span class="source-line-no">1407</span><span id="line-1407"> assertTrue("expected:" + Bytes.toStringBinary(value1) + ", actual:"</span>
<span class="source-line-no">1408</span><span id="line-1408"> + Bytes.toStringBinary(actualValue), Bytes.equals(actualValue, value1));</span>
<span class="source-line-no">1409</span><span id="line-1409"> }</span>
<span class="source-line-no">1410</span><span id="line-1410"> List&lt;Cell&gt; normalList = new ArrayList&lt;&gt;(3);</span>
<span class="source-line-no">1411</span><span id="line-1411"> // r2</span>
<span class="source-line-no">1412</span><span id="line-1412"> scanner.next(normalList);</span>
<span class="source-line-no">1413</span><span id="line-1413"> assertEquals(3, normalList.size());</span>
<span class="source-line-no">1414</span><span id="line-1414"> for (Cell c : normalList) {</span>
<span class="source-line-no">1415</span><span id="line-1415"> byte[] actualValue = CellUtil.cloneValue(c);</span>
<span class="source-line-no">1416</span><span id="line-1416"> assertTrue("expected:" + Bytes.toStringBinary(value2) + ", actual:"</span>
<span class="source-line-no">1417</span><span id="line-1417"> + Bytes.toStringBinary(actualValue), Bytes.equals(actualValue, value2));</span>
<span class="source-line-no">1418</span><span id="line-1418"> }</span>
<span class="source-line-no">1419</span><span id="line-1419"> }</span>
<span class="source-line-no">1420</span><span id="line-1420"> }</span>
<span class="source-line-no">1421</span><span id="line-1421"></span>
<span class="source-line-no">1422</span><span id="line-1422"> @Test</span>
<span class="source-line-no">1423</span><span id="line-1423"> public void testCreateScannerAndSnapshotConcurrently() throws IOException, InterruptedException {</span>
<span class="source-line-no">1424</span><span id="line-1424"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">1425</span><span id="line-1425"> conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore.class.getName());</span>
<span class="source-line-no">1426</span><span id="line-1426"> init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)</span>
<span class="source-line-no">1427</span><span id="line-1427"> .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());</span>
<span class="source-line-no">1428</span><span id="line-1428"> byte[] value = Bytes.toBytes("value");</span>
<span class="source-line-no">1429</span><span id="line-1429"> MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();</span>
<span class="source-line-no">1430</span><span id="line-1430"> long ts = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">1431</span><span id="line-1431"> long seqId = 100;</span>
<span class="source-line-no">1432</span><span id="line-1432"> // older data whihc shouldn't be "seen" by client</span>
<span class="source-line-no">1433</span><span id="line-1433"> store.add(createCell(qf1, ts, seqId, value), memStoreSizing);</span>
<span class="source-line-no">1434</span><span id="line-1434"> store.add(createCell(qf2, ts, seqId, value), memStoreSizing);</span>
<span class="source-line-no">1435</span><span id="line-1435"> store.add(createCell(qf3, ts, seqId, value), memStoreSizing);</span>
<span class="source-line-no">1436</span><span id="line-1436"> TreeSet&lt;byte[]&gt; quals = new TreeSet&lt;&gt;(Bytes.BYTES_COMPARATOR);</span>
<span class="source-line-no">1437</span><span id="line-1437"> quals.add(qf1);</span>
<span class="source-line-no">1438</span><span id="line-1438"> quals.add(qf2);</span>
<span class="source-line-no">1439</span><span id="line-1439"> quals.add(qf3);</span>
<span class="source-line-no">1440</span><span id="line-1440"> StoreFlushContext storeFlushCtx = store.createFlushContext(id++, FlushLifeCycleTracker.DUMMY);</span>
<span class="source-line-no">1441</span><span id="line-1441"> MyCompactingMemStore.START_TEST.set(true);</span>
<span class="source-line-no">1442</span><span id="line-1442"> Runnable flush = () -&gt; {</span>
<span class="source-line-no">1443</span><span id="line-1443"> // this is blocked until we create first scanner from pipeline and snapshot -- phase (1/5)</span>
<span class="source-line-no">1444</span><span id="line-1444"> // recreate the active memstore -- phase (4/5)</span>
<span class="source-line-no">1445</span><span id="line-1445"> storeFlushCtx.prepare();</span>
<span class="source-line-no">1446</span><span id="line-1446"> };</span>
<span class="source-line-no">1447</span><span id="line-1447"> ExecutorService service = Executors.newSingleThreadExecutor();</span>
<span class="source-line-no">1448</span><span id="line-1448"> service.execute(flush);</span>
<span class="source-line-no">1449</span><span id="line-1449"> // we get scanner from pipeline and snapshot but they are empty. -- phase (2/5)</span>
<span class="source-line-no">1450</span><span id="line-1450"> // this is blocked until we recreate the active memstore -- phase (3/5)</span>
<span class="source-line-no">1451</span><span id="line-1451"> // we get scanner from active memstore but it is empty -- phase (5/5)</span>
<span class="source-line-no">1452</span><span id="line-1452"> InternalScanner scanner =</span>
<span class="source-line-no">1453</span><span id="line-1453"> (InternalScanner) store.getScanner(new Scan(new Get(row)), quals, seqId + 1);</span>
<span class="source-line-no">1454</span><span id="line-1454"> service.shutdown();</span>
<span class="source-line-no">1455</span><span id="line-1455"> service.awaitTermination(20, TimeUnit.SECONDS);</span>
<span class="source-line-no">1456</span><span id="line-1456"> try {</span>
<span class="source-line-no">1457</span><span id="line-1457"> try {</span>
<span class="source-line-no">1458</span><span id="line-1458"> List&lt;Cell&gt; results = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">1459</span><span id="line-1459"> scanner.next(results);</span>
<span class="source-line-no">1460</span><span id="line-1460"> assertEquals(3, results.size());</span>
<span class="source-line-no">1461</span><span id="line-1461"> for (Cell c : results) {</span>
<span class="source-line-no">1462</span><span id="line-1462"> byte[] actualValue = CellUtil.cloneValue(c);</span>
<span class="source-line-no">1463</span><span id="line-1463"> assertTrue("expected:" + Bytes.toStringBinary(value) + ", actual:"</span>
<span class="source-line-no">1464</span><span id="line-1464"> + Bytes.toStringBinary(actualValue), Bytes.equals(actualValue, value));</span>
<span class="source-line-no">1465</span><span id="line-1465"> }</span>
<span class="source-line-no">1466</span><span id="line-1466"> } finally {</span>
<span class="source-line-no">1467</span><span id="line-1467"> scanner.close();</span>
<span class="source-line-no">1468</span><span id="line-1468"> }</span>
<span class="source-line-no">1469</span><span id="line-1469"> } finally {</span>
<span class="source-line-no">1470</span><span id="line-1470"> MyCompactingMemStore.START_TEST.set(false);</span>
<span class="source-line-no">1471</span><span id="line-1471"> storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));</span>
<span class="source-line-no">1472</span><span id="line-1472"> storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));</span>
<span class="source-line-no">1473</span><span id="line-1473"> }</span>
<span class="source-line-no">1474</span><span id="line-1474"> }</span>
<span class="source-line-no">1475</span><span id="line-1475"></span>
<span class="source-line-no">1476</span><span id="line-1476"> @Test</span>
<span class="source-line-no">1477</span><span id="line-1477"> public void testScanWithDoubleFlush() throws IOException {</span>
<span class="source-line-no">1478</span><span id="line-1478"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">1479</span><span id="line-1479"> // Initialize region</span>
<span class="source-line-no">1480</span><span id="line-1480"> MyStore myStore = initMyStore(name.getMethodName(), conf, new MyStoreHook() {</span>
<span class="source-line-no">1481</span><span id="line-1481"> @Override</span>
<span class="source-line-no">1482</span><span id="line-1482"> public void getScanners(MyStore store) throws IOException {</span>
<span class="source-line-no">1483</span><span id="line-1483"> final long tmpId = id++;</span>
<span class="source-line-no">1484</span><span id="line-1484"> ExecutorService s = Executors.newSingleThreadExecutor();</span>
<span class="source-line-no">1485</span><span id="line-1485"> s.execute(() -&gt; {</span>
<span class="source-line-no">1486</span><span id="line-1486"> try {</span>
<span class="source-line-no">1487</span><span id="line-1487"> // flush the store before storescanner updates the scanners from store.</span>
<span class="source-line-no">1488</span><span id="line-1488"> // The current data will be flushed into files, and the memstore will</span>
<span class="source-line-no">1489</span><span id="line-1489"> // be clear.</span>
<span class="source-line-no">1490</span><span id="line-1490"> // -- phase (4/4)</span>
<span class="source-line-no">1491</span><span id="line-1491"> flushStore(store, tmpId);</span>
<span class="source-line-no">1492</span><span id="line-1492"> } catch (IOException ex) {</span>
<span class="source-line-no">1493</span><span id="line-1493"> throw new RuntimeException(ex);</span>
<span class="source-line-no">1494</span><span id="line-1494"> }</span>
<span class="source-line-no">1495</span><span id="line-1495"> });</span>
<span class="source-line-no">1496</span><span id="line-1496"> s.shutdown();</span>
<span class="source-line-no">1497</span><span id="line-1497"> try {</span>
<span class="source-line-no">1498</span><span id="line-1498"> // wait for the flush, the thread will be blocked in HStore#notifyChangedReadersObservers.</span>
<span class="source-line-no">1499</span><span id="line-1499"> s.awaitTermination(3, TimeUnit.SECONDS);</span>
<span class="source-line-no">1500</span><span id="line-1500"> } catch (InterruptedException ex) {</span>
<span class="source-line-no">1501</span><span id="line-1501"> }</span>
<span class="source-line-no">1502</span><span id="line-1502"> }</span>
<span class="source-line-no">1503</span><span id="line-1503"> });</span>
<span class="source-line-no">1504</span><span id="line-1504"> byte[] oldValue = Bytes.toBytes("oldValue");</span>
<span class="source-line-no">1505</span><span id="line-1505"> byte[] currentValue = Bytes.toBytes("currentValue");</span>
<span class="source-line-no">1506</span><span id="line-1506"> MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();</span>
<span class="source-line-no">1507</span><span id="line-1507"> long ts = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">1508</span><span id="line-1508"> long seqId = 100;</span>
<span class="source-line-no">1509</span><span id="line-1509"> // older data whihc shouldn't be "seen" by client</span>
<span class="source-line-no">1510</span><span id="line-1510"> myStore.add(createCell(qf1, ts, seqId, oldValue), memStoreSizing);</span>
<span class="source-line-no">1511</span><span id="line-1511"> myStore.add(createCell(qf2, ts, seqId, oldValue), memStoreSizing);</span>
<span class="source-line-no">1512</span><span id="line-1512"> myStore.add(createCell(qf3, ts, seqId, oldValue), memStoreSizing);</span>
<span class="source-line-no">1513</span><span id="line-1513"> long snapshotId = id++;</span>
<span class="source-line-no">1514</span><span id="line-1514"> // push older data into snapshot -- phase (1/4)</span>
<span class="source-line-no">1515</span><span id="line-1515"> StoreFlushContext storeFlushCtx =</span>
<span class="source-line-no">1516</span><span id="line-1516"> store.createFlushContext(snapshotId, FlushLifeCycleTracker.DUMMY);</span>
<span class="source-line-no">1517</span><span id="line-1517"> storeFlushCtx.prepare();</span>
<span class="source-line-no">1518</span><span id="line-1518"></span>
<span class="source-line-no">1519</span><span id="line-1519"> // insert current data into active -- phase (2/4)</span>
<span class="source-line-no">1520</span><span id="line-1520"> myStore.add(createCell(qf1, ts + 1, seqId + 1, currentValue), memStoreSizing);</span>
<span class="source-line-no">1521</span><span id="line-1521"> myStore.add(createCell(qf2, ts + 1, seqId + 1, currentValue), memStoreSizing);</span>
<span class="source-line-no">1522</span><span id="line-1522"> myStore.add(createCell(qf3, ts + 1, seqId + 1, currentValue), memStoreSizing);</span>
<span class="source-line-no">1523</span><span id="line-1523"> TreeSet&lt;byte[]&gt; quals = new TreeSet&lt;&gt;(Bytes.BYTES_COMPARATOR);</span>
<span class="source-line-no">1524</span><span id="line-1524"> quals.add(qf1);</span>
<span class="source-line-no">1525</span><span id="line-1525"> quals.add(qf2);</span>
<span class="source-line-no">1526</span><span id="line-1526"> quals.add(qf3);</span>
<span class="source-line-no">1527</span><span id="line-1527"> try (InternalScanner scanner =</span>
<span class="source-line-no">1528</span><span id="line-1528"> (InternalScanner) myStore.getScanner(new Scan(new Get(row)), quals, seqId + 1)) {</span>
<span class="source-line-no">1529</span><span id="line-1529"> // complete the flush -- phase (3/4)</span>
<span class="source-line-no">1530</span><span id="line-1530"> storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));</span>
<span class="source-line-no">1531</span><span id="line-1531"> storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));</span>
<span class="source-line-no">1532</span><span id="line-1532"></span>
<span class="source-line-no">1533</span><span id="line-1533"> List&lt;Cell&gt; results = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">1534</span><span id="line-1534"> scanner.next(results);</span>
<span class="source-line-no">1535</span><span id="line-1535"> assertEquals(3, results.size());</span>
<span class="source-line-no">1536</span><span id="line-1536"> for (Cell c : results) {</span>
<span class="source-line-no">1537</span><span id="line-1537"> byte[] actualValue = CellUtil.cloneValue(c);</span>
<span class="source-line-no">1538</span><span id="line-1538"> assertTrue("expected:" + Bytes.toStringBinary(currentValue) + ", actual:"</span>
<span class="source-line-no">1539</span><span id="line-1539"> + Bytes.toStringBinary(actualValue), Bytes.equals(actualValue, currentValue));</span>
<span class="source-line-no">1540</span><span id="line-1540"> }</span>
<span class="source-line-no">1541</span><span id="line-1541"> }</span>
<span class="source-line-no">1542</span><span id="line-1542"> }</span>
<span class="source-line-no">1543</span><span id="line-1543"></span>
<span class="source-line-no">1544</span><span id="line-1544"> /**</span>
<span class="source-line-no">1545</span><span id="line-1545"> * This test is for HBASE-27519, when the {@link StoreScanner} is scanning,the Flush and the</span>
<span class="source-line-no">1546</span><span id="line-1546"> * Compaction execute concurrently and theCcompaction compact and archive the flushed</span>
<span class="source-line-no">1547</span><span id="line-1547"> * {@link HStoreFile} which is used by {@link StoreScanner#updateReaders}.Before</span>
<span class="source-line-no">1548</span><span id="line-1548"> * HBASE-27519,{@link StoreScanner.updateReaders} would throw {@link FileNotFoundException}.</span>
<span class="source-line-no">1549</span><span id="line-1549"> */</span>
<span class="source-line-no">1550</span><span id="line-1550"> @Test</span>
<span class="source-line-no">1551</span><span id="line-1551"> public void testStoreScannerUpdateReadersWhenFlushAndCompactConcurrently() throws IOException {</span>
<span class="source-line-no">1552</span><span id="line-1552"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">1553</span><span id="line-1553"> conf.setBoolean(WALFactory.WAL_ENABLED, false);</span>
<span class="source-line-no">1554</span><span id="line-1554"> conf.set(DEFAULT_COMPACTION_POLICY_CLASS_KEY, EverythingPolicy.class.getName());</span>
<span class="source-line-no">1555</span><span id="line-1555"> byte[] r0 = Bytes.toBytes("row0");</span>
<span class="source-line-no">1556</span><span id="line-1556"> byte[] r1 = Bytes.toBytes("row1");</span>
<span class="source-line-no">1557</span><span id="line-1557"> final CyclicBarrier cyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">1558</span><span id="line-1558"> final AtomicBoolean shouldWaitRef = new AtomicBoolean(false);</span>
<span class="source-line-no">1559</span><span id="line-1559"> // Initialize region</span>
<span class="source-line-no">1560</span><span id="line-1560"> final MyStore myStore = initMyStore(name.getMethodName(), conf, new MyStoreHook() {</span>
<span class="source-line-no">1561</span><span id="line-1561"> @Override</span>
<span class="source-line-no">1562</span><span id="line-1562"> public void getScanners(MyStore store) throws IOException {</span>
<span class="source-line-no">1563</span><span id="line-1563"> try {</span>
<span class="source-line-no">1564</span><span id="line-1564"> // Here this method is called by StoreScanner.updateReaders which is invoked by the</span>
<span class="source-line-no">1565</span><span id="line-1565"> // following TestHStore.flushStore</span>
<span class="source-line-no">1566</span><span id="line-1566"> if (shouldWaitRef.get()) {</span>
<span class="source-line-no">1567</span><span id="line-1567"> // wait the following compaction Task start</span>
<span class="source-line-no">1568</span><span id="line-1568"> cyclicBarrier.await();</span>
<span class="source-line-no">1569</span><span id="line-1569"> // wait the following HStore.closeAndArchiveCompactedFiles end.</span>
<span class="source-line-no">1570</span><span id="line-1570"> cyclicBarrier.await();</span>
<span class="source-line-no">1571</span><span id="line-1571"> }</span>
<span class="source-line-no">1572</span><span id="line-1572"> } catch (BrokenBarrierException | InterruptedException e) {</span>
<span class="source-line-no">1573</span><span id="line-1573"> throw new RuntimeException(e);</span>
<span class="source-line-no">1574</span><span id="line-1574"> }</span>
<span class="source-line-no">1575</span><span id="line-1575"> }</span>
<span class="source-line-no">1576</span><span id="line-1576"> });</span>
<span class="source-line-no">1577</span><span id="line-1577"></span>
<span class="source-line-no">1578</span><span id="line-1578"> final AtomicReference&lt;Throwable&gt; compactionExceptionRef = new AtomicReference&lt;Throwable&gt;(null);</span>
<span class="source-line-no">1579</span><span id="line-1579"> Runnable compactionTask = () -&gt; {</span>
<span class="source-line-no">1580</span><span id="line-1580"> try {</span>
<span class="source-line-no">1581</span><span id="line-1581"> // Only when the StoreScanner.updateReaders invoked by TestHStore.flushStore prepares for</span>
<span class="source-line-no">1582</span><span id="line-1582"> // entering the MyStore.getScanners, compactionTask could start.</span>
<span class="source-line-no">1583</span><span id="line-1583"> cyclicBarrier.await();</span>
<span class="source-line-no">1584</span><span id="line-1584"> region.compactStore(family, new NoLimitThroughputController());</span>
<span class="source-line-no">1585</span><span id="line-1585"> myStore.closeAndArchiveCompactedFiles();</span>
<span class="source-line-no">1586</span><span id="line-1586"> // Notify StoreScanner.updateReaders could enter MyStore.getScanners.</span>
<span class="source-line-no">1587</span><span id="line-1587"> cyclicBarrier.await();</span>
<span class="source-line-no">1588</span><span id="line-1588"> } catch (Throwable e) {</span>
<span class="source-line-no">1589</span><span id="line-1589"> compactionExceptionRef.set(e);</span>
<span class="source-line-no">1590</span><span id="line-1590"> }</span>
<span class="source-line-no">1591</span><span id="line-1591"> };</span>
<span class="source-line-no">1592</span><span id="line-1592"></span>
<span class="source-line-no">1593</span><span id="line-1593"> long ts = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">1594</span><span id="line-1594"> long seqId = 100;</span>
<span class="source-line-no">1595</span><span id="line-1595"> byte[] value = Bytes.toBytes("value");</span>
<span class="source-line-no">1596</span><span id="line-1596"> // older data whihc shouldn't be "seen" by client</span>
<span class="source-line-no">1597</span><span id="line-1597"> myStore.add(createCell(r0, qf1, ts, seqId, value), null);</span>
<span class="source-line-no">1598</span><span id="line-1598"> flushStore(myStore, id++);</span>
<span class="source-line-no">1599</span><span id="line-1599"> myStore.add(createCell(r0, qf2, ts, seqId, value), null);</span>
<span class="source-line-no">1600</span><span id="line-1600"> flushStore(myStore, id++);</span>
<span class="source-line-no">1601</span><span id="line-1601"> myStore.add(createCell(r0, qf3, ts, seqId, value), null);</span>
<span class="source-line-no">1602</span><span id="line-1602"> TreeSet&lt;byte[]&gt; quals = new TreeSet&lt;&gt;(Bytes.BYTES_COMPARATOR);</span>
<span class="source-line-no">1603</span><span id="line-1603"> quals.add(qf1);</span>
<span class="source-line-no">1604</span><span id="line-1604"> quals.add(qf2);</span>
<span class="source-line-no">1605</span><span id="line-1605"> quals.add(qf3);</span>
<span class="source-line-no">1606</span><span id="line-1606"></span>
<span class="source-line-no">1607</span><span id="line-1607"> myStore.add(createCell(r1, qf1, ts, seqId, value), null);</span>
<span class="source-line-no">1608</span><span id="line-1608"> myStore.add(createCell(r1, qf2, ts, seqId, value), null);</span>
<span class="source-line-no">1609</span><span id="line-1609"> myStore.add(createCell(r1, qf3, ts, seqId, value), null);</span>
<span class="source-line-no">1610</span><span id="line-1610"></span>
<span class="source-line-no">1611</span><span id="line-1611"> Thread.currentThread()</span>
<span class="source-line-no">1612</span><span id="line-1612"> .setName("testStoreScannerUpdateReadersWhenFlushAndCompactConcurrently thread");</span>
<span class="source-line-no">1613</span><span id="line-1613"> Scan scan = new Scan();</span>
<span class="source-line-no">1614</span><span id="line-1614"> scan.withStartRow(r0, true);</span>
<span class="source-line-no">1615</span><span id="line-1615"> try (InternalScanner scanner = (InternalScanner) myStore.getScanner(scan, quals, seqId)) {</span>
<span class="source-line-no">1616</span><span id="line-1616"> List&lt;Cell&gt; results = new MyList&lt;&gt;(size -&gt; {</span>
<span class="source-line-no">1617</span><span id="line-1617"> switch (size) {</span>
<span class="source-line-no">1618</span><span id="line-1618"> case 1:</span>
<span class="source-line-no">1619</span><span id="line-1619"> shouldWaitRef.set(true);</span>
<span class="source-line-no">1620</span><span id="line-1620"> Thread thread = new Thread(compactionTask);</span>
<span class="source-line-no">1621</span><span id="line-1621"> thread.setName("MyCompacting Thread.");</span>
<span class="source-line-no">1622</span><span id="line-1622"> thread.start();</span>
<span class="source-line-no">1623</span><span id="line-1623"> try {</span>
<span class="source-line-no">1624</span><span id="line-1624"> flushStore(myStore, id++);</span>
<span class="source-line-no">1625</span><span id="line-1625"> thread.join();</span>
<span class="source-line-no">1626</span><span id="line-1626"> } catch (IOException | InterruptedException e) {</span>
<span class="source-line-no">1627</span><span id="line-1627"> throw new RuntimeException(e);</span>
<span class="source-line-no">1628</span><span id="line-1628"> }</span>
<span class="source-line-no">1629</span><span id="line-1629"> shouldWaitRef.set(false);</span>
<span class="source-line-no">1630</span><span id="line-1630"> break;</span>
<span class="source-line-no">1631</span><span id="line-1631"> default:</span>
<span class="source-line-no">1632</span><span id="line-1632"> break;</span>
<span class="source-line-no">1633</span><span id="line-1633"> }</span>
<span class="source-line-no">1634</span><span id="line-1634"> });</span>
<span class="source-line-no">1635</span><span id="line-1635"> // Before HBASE-27519, here would throw java.io.FileNotFoundException because the storeFile</span>
<span class="source-line-no">1636</span><span id="line-1636"> // which used by StoreScanner.updateReaders is deleted by compactionTask.</span>
<span class="source-line-no">1637</span><span id="line-1637"> scanner.next(results);</span>
<span class="source-line-no">1638</span><span id="line-1638"> // The results is r0 row cells.</span>
<span class="source-line-no">1639</span><span id="line-1639"> assertEquals(3, results.size());</span>
<span class="source-line-no">1640</span><span id="line-1640"> assertTrue(compactionExceptionRef.get() == null);</span>
<span class="source-line-no">1641</span><span id="line-1641"> }</span>
<span class="source-line-no">1642</span><span id="line-1642"> }</span>
<span class="source-line-no">1643</span><span id="line-1643"></span>
<span class="source-line-no">1644</span><span id="line-1644"> @Test</span>
<span class="source-line-no">1645</span><span id="line-1645"> public void testReclaimChunkWhenScaning() throws IOException {</span>
<span class="source-line-no">1646</span><span id="line-1646"> init("testReclaimChunkWhenScaning");</span>
<span class="source-line-no">1647</span><span id="line-1647"> long ts = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">1648</span><span id="line-1648"> long seqId = 100;</span>
<span class="source-line-no">1649</span><span id="line-1649"> byte[] value = Bytes.toBytes("value");</span>
<span class="source-line-no">1650</span><span id="line-1650"> // older data whihc shouldn't be "seen" by client</span>
<span class="source-line-no">1651</span><span id="line-1651"> store.add(createCell(qf1, ts, seqId, value), null);</span>
<span class="source-line-no">1652</span><span id="line-1652"> store.add(createCell(qf2, ts, seqId, value), null);</span>
<span class="source-line-no">1653</span><span id="line-1653"> store.add(createCell(qf3, ts, seqId, value), null);</span>
<span class="source-line-no">1654</span><span id="line-1654"> TreeSet&lt;byte[]&gt; quals = new TreeSet&lt;&gt;(Bytes.BYTES_COMPARATOR);</span>
<span class="source-line-no">1655</span><span id="line-1655"> quals.add(qf1);</span>
<span class="source-line-no">1656</span><span id="line-1656"> quals.add(qf2);</span>
<span class="source-line-no">1657</span><span id="line-1657"> quals.add(qf3);</span>
<span class="source-line-no">1658</span><span id="line-1658"> try (InternalScanner scanner =</span>
<span class="source-line-no">1659</span><span id="line-1659"> (InternalScanner) store.getScanner(new Scan(new Get(row)), quals, seqId)) {</span>
<span class="source-line-no">1660</span><span id="line-1660"> List&lt;Cell&gt; results = new MyList&lt;&gt;(size -&gt; {</span>
<span class="source-line-no">1661</span><span id="line-1661"> switch (size) {</span>
<span class="source-line-no">1662</span><span id="line-1662"> // 1) we get the first cell (qf1)</span>
<span class="source-line-no">1663</span><span id="line-1663"> // 2) flush the data to have StoreScanner update inner scanners</span>
<span class="source-line-no">1664</span><span id="line-1664"> // 3) the chunk will be reclaimed after updaing</span>
<span class="source-line-no">1665</span><span id="line-1665"> case 1:</span>
<span class="source-line-no">1666</span><span id="line-1666"> try {</span>
<span class="source-line-no">1667</span><span id="line-1667"> flushStore(store, id++);</span>
<span class="source-line-no">1668</span><span id="line-1668"> } catch (IOException e) {</span>
<span class="source-line-no">1669</span><span id="line-1669"> throw new RuntimeException(e);</span>
<span class="source-line-no">1670</span><span id="line-1670"> }</span>
<span class="source-line-no">1671</span><span id="line-1671"> break;</span>
<span class="source-line-no">1672</span><span id="line-1672"> // 1) we get the second cell (qf2)</span>
<span class="source-line-no">1673</span><span id="line-1673"> // 2) add some cell to fill some byte into the chunk (we have only one chunk)</span>
<span class="source-line-no">1674</span><span id="line-1674"> case 2:</span>
<span class="source-line-no">1675</span><span id="line-1675"> try {</span>
<span class="source-line-no">1676</span><span id="line-1676"> byte[] newValue = Bytes.toBytes("newValue");</span>
<span class="source-line-no">1677</span><span id="line-1677"> // older data whihc shouldn't be "seen" by client</span>
<span class="source-line-no">1678</span><span id="line-1678"> store.add(createCell(qf1, ts + 1, seqId + 1, newValue), null);</span>
<span class="source-line-no">1679</span><span id="line-1679"> store.add(createCell(qf2, ts + 1, seqId + 1, newValue), null);</span>
<span class="source-line-no">1680</span><span id="line-1680"> store.add(createCell(qf3, ts + 1, seqId + 1, newValue), null);</span>
<span class="source-line-no">1681</span><span id="line-1681"> } catch (IOException e) {</span>
<span class="source-line-no">1682</span><span id="line-1682"> throw new RuntimeException(e);</span>
<span class="source-line-no">1683</span><span id="line-1683"> }</span>
<span class="source-line-no">1684</span><span id="line-1684"> break;</span>
<span class="source-line-no">1685</span><span id="line-1685"> default:</span>
<span class="source-line-no">1686</span><span id="line-1686"> break;</span>
<span class="source-line-no">1687</span><span id="line-1687"> }</span>
<span class="source-line-no">1688</span><span id="line-1688"> });</span>
<span class="source-line-no">1689</span><span id="line-1689"> scanner.next(results);</span>
<span class="source-line-no">1690</span><span id="line-1690"> assertEquals(3, results.size());</span>
<span class="source-line-no">1691</span><span id="line-1691"> for (Cell c : results) {</span>
<span class="source-line-no">1692</span><span id="line-1692"> byte[] actualValue = CellUtil.cloneValue(c);</span>
<span class="source-line-no">1693</span><span id="line-1693"> assertTrue("expected:" + Bytes.toStringBinary(value) + ", actual:"</span>
<span class="source-line-no">1694</span><span id="line-1694"> + Bytes.toStringBinary(actualValue), Bytes.equals(actualValue, value));</span>
<span class="source-line-no">1695</span><span id="line-1695"> }</span>
<span class="source-line-no">1696</span><span id="line-1696"> }</span>
<span class="source-line-no">1697</span><span id="line-1697"> }</span>
<span class="source-line-no">1698</span><span id="line-1698"></span>
<span class="source-line-no">1699</span><span id="line-1699"> /**</span>
<span class="source-line-no">1700</span><span id="line-1700"> * If there are two running InMemoryFlushRunnable, the later InMemoryFlushRunnable may change the</span>
<span class="source-line-no">1701</span><span id="line-1701"> * versionedList. And the first InMemoryFlushRunnable will use the chagned versionedList to remove</span>
<span class="source-line-no">1702</span><span id="line-1702"> * the corresponding segments. In short, there will be some segements which isn't in merge are</span>
<span class="source-line-no">1703</span><span id="line-1703"> * removed.</span>
<span class="source-line-no">1704</span><span id="line-1704"> */</span>
<span class="source-line-no">1705</span><span id="line-1705"> @Test</span>
<span class="source-line-no">1706</span><span id="line-1706"> public void testRunDoubleMemStoreCompactors() throws IOException, InterruptedException {</span>
<span class="source-line-no">1707</span><span id="line-1707"> int flushSize = 500;</span>
<span class="source-line-no">1708</span><span id="line-1708"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">1709</span><span id="line-1709"> conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStoreWithCustomCompactor.class.getName());</span>
<span class="source-line-no">1710</span><span id="line-1710"> conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.25);</span>
<span class="source-line-no">1711</span><span id="line-1711"> MyCompactingMemStoreWithCustomCompactor.RUNNER_COUNT.set(0);</span>
<span class="source-line-no">1712</span><span id="line-1712"> conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushSize));</span>
<span class="source-line-no">1713</span><span id="line-1713"> // Set the lower threshold to invoke the "MERGE" policy</span>
<span class="source-line-no">1714</span><span id="line-1714"> conf.set(MemStoreCompactionStrategy.COMPACTING_MEMSTORE_THRESHOLD_KEY, String.valueOf(0));</span>
<span class="source-line-no">1715</span><span id="line-1715"> init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)</span>
<span class="source-line-no">1716</span><span id="line-1716"> .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());</span>
<span class="source-line-no">1717</span><span id="line-1717"> byte[] value = Bytes.toBytes("thisisavarylargevalue");</span>
<span class="source-line-no">1718</span><span id="line-1718"> MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();</span>
<span class="source-line-no">1719</span><span id="line-1719"> long ts = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">1720</span><span id="line-1720"> long seqId = 100;</span>
<span class="source-line-no">1721</span><span id="line-1721"> // older data whihc shouldn't be "seen" by client</span>
<span class="source-line-no">1722</span><span id="line-1722"> store.add(createCell(qf1, ts, seqId, value), memStoreSizing);</span>
<span class="source-line-no">1723</span><span id="line-1723"> store.add(createCell(qf2, ts, seqId, value), memStoreSizing);</span>
<span class="source-line-no">1724</span><span id="line-1724"> store.add(createCell(qf3, ts, seqId, value), memStoreSizing);</span>
<span class="source-line-no">1725</span><span id="line-1725"> assertEquals(1, MyCompactingMemStoreWithCustomCompactor.RUNNER_COUNT.get());</span>
<span class="source-line-no">1726</span><span id="line-1726"> StoreFlushContext storeFlushCtx = store.createFlushContext(id++, FlushLifeCycleTracker.DUMMY);</span>
<span class="source-line-no">1727</span><span id="line-1727"> storeFlushCtx.prepare();</span>
<span class="source-line-no">1728</span><span id="line-1728"> // This shouldn't invoke another in-memory flush because the first compactor thread</span>
<span class="source-line-no">1729</span><span id="line-1729"> // hasn't accomplished the in-memory compaction.</span>
<span class="source-line-no">1730</span><span id="line-1730"> store.add(createCell(qf1, ts + 1, seqId + 1, value), memStoreSizing);</span>
<span class="source-line-no">1731</span><span id="line-1731"> store.add(createCell(qf1, ts + 1, seqId + 1, value), memStoreSizing);</span>
<span class="source-line-no">1732</span><span id="line-1732"> store.add(createCell(qf1, ts + 1, seqId + 1, value), memStoreSizing);</span>
<span class="source-line-no">1733</span><span id="line-1733"> assertEquals(1, MyCompactingMemStoreWithCustomCompactor.RUNNER_COUNT.get());</span>
<span class="source-line-no">1734</span><span id="line-1734"> // okay. Let the compaction be completed</span>
<span class="source-line-no">1735</span><span id="line-1735"> MyMemStoreCompactor.START_COMPACTOR_LATCH.countDown();</span>
<span class="source-line-no">1736</span><span id="line-1736"> CompactingMemStore mem = (CompactingMemStore) ((HStore) store).memstore;</span>
<span class="source-line-no">1737</span><span id="line-1737"> while (mem.isMemStoreFlushingInMemory()) {</span>
<span class="source-line-no">1738</span><span id="line-1738"> TimeUnit.SECONDS.sleep(1);</span>
<span class="source-line-no">1739</span><span id="line-1739"> }</span>
<span class="source-line-no">1740</span><span id="line-1740"> // This should invoke another in-memory flush.</span>
<span class="source-line-no">1741</span><span id="line-1741"> store.add(createCell(qf1, ts + 2, seqId + 2, value), memStoreSizing);</span>
<span class="source-line-no">1742</span><span id="line-1742"> store.add(createCell(qf1, ts + 2, seqId + 2, value), memStoreSizing);</span>
<span class="source-line-no">1743</span><span id="line-1743"> store.add(createCell(qf1, ts + 2, seqId + 2, value), memStoreSizing);</span>
<span class="source-line-no">1744</span><span id="line-1744"> assertEquals(2, MyCompactingMemStoreWithCustomCompactor.RUNNER_COUNT.get());</span>
<span class="source-line-no">1745</span><span id="line-1745"> conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,</span>
<span class="source-line-no">1746</span><span id="line-1746"> String.valueOf(TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE));</span>
<span class="source-line-no">1747</span><span id="line-1747"> storeFlushCtx.flushCache(Mockito.mock(MonitoredTask.class));</span>
<span class="source-line-no">1748</span><span id="line-1748"> storeFlushCtx.commit(Mockito.mock(MonitoredTask.class));</span>
<span class="source-line-no">1749</span><span id="line-1749"> }</span>
<span class="source-line-no">1750</span><span id="line-1750"></span>
<span class="source-line-no">1751</span><span id="line-1751"> @Test</span>
<span class="source-line-no">1752</span><span id="line-1752"> public void testAge() throws IOException {</span>
<span class="source-line-no">1753</span><span id="line-1753"> long currentTime = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">1754</span><span id="line-1754"> ManualEnvironmentEdge edge = new ManualEnvironmentEdge();</span>
<span class="source-line-no">1755</span><span id="line-1755"> edge.setValue(currentTime);</span>
<span class="source-line-no">1756</span><span id="line-1756"> EnvironmentEdgeManager.injectEdge(edge);</span>
<span class="source-line-no">1757</span><span id="line-1757"> Configuration conf = TEST_UTIL.getConfiguration();</span>
<span class="source-line-no">1758</span><span id="line-1758"> ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.of(family);</span>
<span class="source-line-no">1759</span><span id="line-1759"> initHRegion(name.getMethodName(), conf,</span>
<span class="source-line-no">1760</span><span id="line-1760"> TableDescriptorBuilder.newBuilder(TableName.valueOf(table)), hcd, null, false);</span>
<span class="source-line-no">1761</span><span id="line-1761"> HStore store = new HStore(region, hcd, conf, false) {</span>
<span class="source-line-no">1762</span><span id="line-1762"></span>
<span class="source-line-no">1763</span><span id="line-1763"> @Override</span>
<span class="source-line-no">1764</span><span id="line-1764"> protected StoreEngine&lt;?, ?, ?, ?&gt; createStoreEngine(HStore store, Configuration conf,</span>
<span class="source-line-no">1765</span><span id="line-1765"> CellComparator kvComparator) throws IOException {</span>
<span class="source-line-no">1766</span><span id="line-1766"> List&lt;HStoreFile&gt; storefiles =</span>
<span class="source-line-no">1767</span><span id="line-1767"> Arrays.asList(mockStoreFile(currentTime - 10), mockStoreFile(currentTime - 100),</span>
<span class="source-line-no">1768</span><span id="line-1768"> mockStoreFile(currentTime - 1000), mockStoreFile(currentTime - 10000));</span>
<span class="source-line-no">1769</span><span id="line-1769"> StoreFileManager sfm = mock(StoreFileManager.class);</span>
<span class="source-line-no">1770</span><span id="line-1770"> when(sfm.getStoreFiles()).thenReturn(storefiles);</span>
<span class="source-line-no">1771</span><span id="line-1771"> StoreEngine&lt;?, ?, ?, ?&gt; storeEngine = mock(StoreEngine.class);</span>
<span class="source-line-no">1772</span><span id="line-1772"> when(storeEngine.getStoreFileManager()).thenReturn(sfm);</span>
<span class="source-line-no">1773</span><span id="line-1773"> return storeEngine;</span>
<span class="source-line-no">1774</span><span id="line-1774"> }</span>
<span class="source-line-no">1775</span><span id="line-1775"> };</span>
<span class="source-line-no">1776</span><span id="line-1776"> assertEquals(10L, store.getMinStoreFileAge().getAsLong());</span>
<span class="source-line-no">1777</span><span id="line-1777"> assertEquals(10000L, store.getMaxStoreFileAge().getAsLong());</span>
<span class="source-line-no">1778</span><span id="line-1778"> assertEquals((10 + 100 + 1000 + 10000) / 4.0, store.getAvgStoreFileAge().getAsDouble(), 1E-4);</span>
<span class="source-line-no">1779</span><span id="line-1779"> }</span>
<span class="source-line-no">1780</span><span id="line-1780"></span>
<span class="source-line-no">1781</span><span id="line-1781"> private HStoreFile mockStoreFile(long createdTime) {</span>
<span class="source-line-no">1782</span><span id="line-1782"> StoreFileInfo info = mock(StoreFileInfo.class);</span>
<span class="source-line-no">1783</span><span id="line-1783"> when(info.getCreatedTimestamp()).thenReturn(createdTime);</span>
<span class="source-line-no">1784</span><span id="line-1784"> HStoreFile sf = mock(HStoreFile.class);</span>
<span class="source-line-no">1785</span><span id="line-1785"> when(sf.getReader()).thenReturn(mock(StoreFileReader.class));</span>
<span class="source-line-no">1786</span><span id="line-1786"> when(sf.isHFile()).thenReturn(true);</span>
<span class="source-line-no">1787</span><span id="line-1787"> when(sf.getFileInfo()).thenReturn(info);</span>
<span class="source-line-no">1788</span><span id="line-1788"> return sf;</span>
<span class="source-line-no">1789</span><span id="line-1789"> }</span>
<span class="source-line-no">1790</span><span id="line-1790"></span>
<span class="source-line-no">1791</span><span id="line-1791"> private MyStore initMyStore(String methodName, Configuration conf, MyStoreHook hook)</span>
<span class="source-line-no">1792</span><span id="line-1792"> throws IOException {</span>
<span class="source-line-no">1793</span><span id="line-1793"> return (MyStore) init(methodName, conf,</span>
<span class="source-line-no">1794</span><span id="line-1794"> TableDescriptorBuilder.newBuilder(TableName.valueOf(table)),</span>
<span class="source-line-no">1795</span><span id="line-1795"> ColumnFamilyDescriptorBuilder.newBuilder(family).setMaxVersions(5).build(), hook);</span>
<span class="source-line-no">1796</span><span id="line-1796"> }</span>
<span class="source-line-no">1797</span><span id="line-1797"></span>
<span class="source-line-no">1798</span><span id="line-1798"> private static class MyStore extends HStore {</span>
<span class="source-line-no">1799</span><span id="line-1799"> private final MyStoreHook hook;</span>
<span class="source-line-no">1800</span><span id="line-1800"></span>
<span class="source-line-no">1801</span><span id="line-1801"> MyStore(final HRegion region, final ColumnFamilyDescriptor family,</span>
<span class="source-line-no">1802</span><span id="line-1802"> final Configuration confParam, MyStoreHook hook, boolean switchToPread) throws IOException {</span>
<span class="source-line-no">1803</span><span id="line-1803"> super(region, family, confParam, false);</span>
<span class="source-line-no">1804</span><span id="line-1804"> this.hook = hook;</span>
<span class="source-line-no">1805</span><span id="line-1805"> }</span>
<span class="source-line-no">1806</span><span id="line-1806"></span>
<span class="source-line-no">1807</span><span id="line-1807"> @Override</span>
<span class="source-line-no">1808</span><span id="line-1808"> public List&lt;KeyValueScanner&gt; getScanners(List&lt;HStoreFile&gt; files, boolean cacheBlocks,</span>
<span class="source-line-no">1809</span><span id="line-1809"> boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow,</span>
<span class="source-line-no">1810</span><span id="line-1810"> boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt,</span>
<span class="source-line-no">1811</span><span id="line-1811"> boolean includeMemstoreScanner, boolean onlyLatestVersion) throws IOException {</span>
<span class="source-line-no">1812</span><span id="line-1812"> hook.getScanners(this);</span>
<span class="source-line-no">1813</span><span id="line-1813"> return super.getScanners(files, cacheBlocks, usePread, isCompaction, matcher, startRow, true,</span>
<span class="source-line-no">1814</span><span id="line-1814"> stopRow, false, readPt, includeMemstoreScanner, onlyLatestVersion);</span>
<span class="source-line-no">1815</span><span id="line-1815"> }</span>
<span class="source-line-no">1816</span><span id="line-1816"></span>
<span class="source-line-no">1817</span><span id="line-1817"> @Override</span>
<span class="source-line-no">1818</span><span id="line-1818"> public long getSmallestReadPoint() {</span>
<span class="source-line-no">1819</span><span id="line-1819"> return hook.getSmallestReadPoint(this);</span>
<span class="source-line-no">1820</span><span id="line-1820"> }</span>
<span class="source-line-no">1821</span><span id="line-1821"> }</span>
<span class="source-line-no">1822</span><span id="line-1822"></span>
<span class="source-line-no">1823</span><span id="line-1823"> private abstract static class MyStoreHook {</span>
<span class="source-line-no">1824</span><span id="line-1824"></span>
<span class="source-line-no">1825</span><span id="line-1825"> void getScanners(MyStore store) throws IOException {</span>
<span class="source-line-no">1826</span><span id="line-1826"> }</span>
<span class="source-line-no">1827</span><span id="line-1827"></span>
<span class="source-line-no">1828</span><span id="line-1828"> long getSmallestReadPoint(HStore store) {</span>
<span class="source-line-no">1829</span><span id="line-1829"> return store.getHRegion().getSmallestReadPoint();</span>
<span class="source-line-no">1830</span><span id="line-1830"> }</span>
<span class="source-line-no">1831</span><span id="line-1831"> }</span>
<span class="source-line-no">1832</span><span id="line-1832"></span>
<span class="source-line-no">1833</span><span id="line-1833"> @Test</span>
<span class="source-line-no">1834</span><span id="line-1834"> public void testSwitchingPreadtoStreamParallelyWithCompactionDischarger() throws Exception {</span>
<span class="source-line-no">1835</span><span id="line-1835"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">1836</span><span id="line-1836"> conf.set("hbase.hstore.engine.class", DummyStoreEngine.class.getName());</span>
<span class="source-line-no">1837</span><span id="line-1837"> conf.setLong(StoreScanner.STORESCANNER_PREAD_MAX_BYTES, 0);</span>
<span class="source-line-no">1838</span><span id="line-1838"> // Set the lower threshold to invoke the "MERGE" policy</span>
<span class="source-line-no">1839</span><span id="line-1839"> MyStore store = initMyStore(name.getMethodName(), conf, new MyStoreHook() {</span>
<span class="source-line-no">1840</span><span id="line-1840"> });</span>
<span class="source-line-no">1841</span><span id="line-1841"> MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();</span>
<span class="source-line-no">1842</span><span id="line-1842"> long ts = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">1843</span><span id="line-1843"> long seqID = 1L;</span>
<span class="source-line-no">1844</span><span id="line-1844"> // Add some data to the region and do some flushes</span>
<span class="source-line-no">1845</span><span id="line-1845"> for (int i = 1; i &lt; 10; i++) {</span>
<span class="source-line-no">1846</span><span id="line-1846"> store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),</span>
<span class="source-line-no">1847</span><span id="line-1847"> memStoreSizing);</span>
<span class="source-line-no">1848</span><span id="line-1848"> }</span>
<span class="source-line-no">1849</span><span id="line-1849"> // flush them</span>
<span class="source-line-no">1850</span><span id="line-1850"> flushStore(store, seqID);</span>
<span class="source-line-no">1851</span><span id="line-1851"> for (int i = 11; i &lt; 20; i++) {</span>
<span class="source-line-no">1852</span><span id="line-1852"> store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),</span>
<span class="source-line-no">1853</span><span id="line-1853"> memStoreSizing);</span>
<span class="source-line-no">1854</span><span id="line-1854"> }</span>
<span class="source-line-no">1855</span><span id="line-1855"> // flush them</span>
<span class="source-line-no">1856</span><span id="line-1856"> flushStore(store, seqID);</span>
<span class="source-line-no">1857</span><span id="line-1857"> for (int i = 21; i &lt; 30; i++) {</span>
<span class="source-line-no">1858</span><span id="line-1858"> store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),</span>
<span class="source-line-no">1859</span><span id="line-1859"> memStoreSizing);</span>
<span class="source-line-no">1860</span><span id="line-1860"> }</span>
<span class="source-line-no">1861</span><span id="line-1861"> // flush them</span>
<span class="source-line-no">1862</span><span id="line-1862"> flushStore(store, seqID);</span>
<span class="source-line-no">1863</span><span id="line-1863"></span>
<span class="source-line-no">1864</span><span id="line-1864"> assertEquals(3, store.getStorefilesCount());</span>
<span class="source-line-no">1865</span><span id="line-1865"> Scan scan = new Scan();</span>
<span class="source-line-no">1866</span><span id="line-1866"> scan.addFamily(family);</span>
<span class="source-line-no">1867</span><span id="line-1867"> Collection&lt;HStoreFile&gt; storefiles2 = store.getStorefiles();</span>
<span class="source-line-no">1868</span><span id="line-1868"> ArrayList&lt;HStoreFile&gt; actualStorefiles = Lists.newArrayList(storefiles2);</span>
<span class="source-line-no">1869</span><span id="line-1869"> StoreScanner storeScanner =</span>
<span class="source-line-no">1870</span><span id="line-1870"> (StoreScanner) store.getScanner(scan, scan.getFamilyMap().get(family), Long.MAX_VALUE);</span>
<span class="source-line-no">1871</span><span id="line-1871"> // get the current heap</span>
<span class="source-line-no">1872</span><span id="line-1872"> KeyValueHeap heap = storeScanner.heap;</span>
<span class="source-line-no">1873</span><span id="line-1873"> // create more store files</span>
<span class="source-line-no">1874</span><span id="line-1874"> for (int i = 31; i &lt; 40; i++) {</span>
<span class="source-line-no">1875</span><span id="line-1875"> store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),</span>
<span class="source-line-no">1876</span><span id="line-1876"> memStoreSizing);</span>
<span class="source-line-no">1877</span><span id="line-1877"> }</span>
<span class="source-line-no">1878</span><span id="line-1878"> // flush them</span>
<span class="source-line-no">1879</span><span id="line-1879"> flushStore(store, seqID);</span>
<span class="source-line-no">1880</span><span id="line-1880"></span>
<span class="source-line-no">1881</span><span id="line-1881"> for (int i = 41; i &lt; 50; i++) {</span>
<span class="source-line-no">1882</span><span id="line-1882"> store.add(createCell(Bytes.toBytes("row" + i), qf1, ts, seqID++, Bytes.toBytes("")),</span>
<span class="source-line-no">1883</span><span id="line-1883"> memStoreSizing);</span>
<span class="source-line-no">1884</span><span id="line-1884"> }</span>
<span class="source-line-no">1885</span><span id="line-1885"> // flush them</span>
<span class="source-line-no">1886</span><span id="line-1886"> flushStore(store, seqID);</span>
<span class="source-line-no">1887</span><span id="line-1887"> storefiles2 = store.getStorefiles();</span>
<span class="source-line-no">1888</span><span id="line-1888"> ArrayList&lt;HStoreFile&gt; actualStorefiles1 = Lists.newArrayList(storefiles2);</span>
<span class="source-line-no">1889</span><span id="line-1889"> actualStorefiles1.removeAll(actualStorefiles);</span>
<span class="source-line-no">1890</span><span id="line-1890"> // Do compaction</span>
<span class="source-line-no">1891</span><span id="line-1891"> MyThread thread = new MyThread(storeScanner);</span>
<span class="source-line-no">1892</span><span id="line-1892"> thread.start();</span>
<span class="source-line-no">1893</span><span id="line-1893"> store.replaceStoreFiles(actualStorefiles, actualStorefiles1, false);</span>
<span class="source-line-no">1894</span><span id="line-1894"> thread.join();</span>
<span class="source-line-no">1895</span><span id="line-1895"> KeyValueHeap heap2 = thread.getHeap();</span>
<span class="source-line-no">1896</span><span id="line-1896"> assertFalse(heap.equals(heap2));</span>
<span class="source-line-no">1897</span><span id="line-1897"> }</span>
<span class="source-line-no">1898</span><span id="line-1898"></span>
<span class="source-line-no">1899</span><span id="line-1899"> @Test</span>
<span class="source-line-no">1900</span><span id="line-1900"> public void testMaxPreadBytesConfiguredToBeLessThanZero() throws Exception {</span>
<span class="source-line-no">1901</span><span id="line-1901"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">1902</span><span id="line-1902"> conf.set("hbase.hstore.engine.class", DummyStoreEngine.class.getName());</span>
<span class="source-line-no">1903</span><span id="line-1903"> // Set 'hbase.storescanner.pread.max.bytes' &lt; 0, so that StoreScanner will be a STREAM type.</span>
<span class="source-line-no">1904</span><span id="line-1904"> conf.setLong(StoreScanner.STORESCANNER_PREAD_MAX_BYTES, -1);</span>
<span class="source-line-no">1905</span><span id="line-1905"> MyStore store = initMyStore(name.getMethodName(), conf, new MyStoreHook() {</span>
<span class="source-line-no">1906</span><span id="line-1906"> });</span>
<span class="source-line-no">1907</span><span id="line-1907"> Scan scan = new Scan();</span>
<span class="source-line-no">1908</span><span id="line-1908"> scan.addFamily(family);</span>
<span class="source-line-no">1909</span><span id="line-1909"> // ReadType on Scan is still DEFAULT only.</span>
<span class="source-line-no">1910</span><span id="line-1910"> assertEquals(ReadType.DEFAULT, scan.getReadType());</span>
<span class="source-line-no">1911</span><span id="line-1911"> StoreScanner storeScanner =</span>
<span class="source-line-no">1912</span><span id="line-1912"> (StoreScanner) store.getScanner(scan, scan.getFamilyMap().get(family), Long.MAX_VALUE);</span>
<span class="source-line-no">1913</span><span id="line-1913"> assertFalse(storeScanner.isScanUsePread());</span>
<span class="source-line-no">1914</span><span id="line-1914"> }</span>
<span class="source-line-no">1915</span><span id="line-1915"></span>
<span class="source-line-no">1916</span><span id="line-1916"> @Test</span>
<span class="source-line-no">1917</span><span id="line-1917"> public void testInMemoryCompactionTypeWithLowerCase() throws IOException, InterruptedException {</span>
<span class="source-line-no">1918</span><span id="line-1918"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">1919</span><span id="line-1919"> conf.set("hbase.systemtables.compacting.memstore.type", "eager");</span>
<span class="source-line-no">1920</span><span id="line-1920"> init(name.getMethodName(), conf,</span>
<span class="source-line-no">1921</span><span id="line-1921"> TableDescriptorBuilder.newBuilder(</span>
<span class="source-line-no">1922</span><span id="line-1922"> TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME, "meta".getBytes())),</span>
<span class="source-line-no">1923</span><span id="line-1923"> ColumnFamilyDescriptorBuilder.newBuilder(family)</span>
<span class="source-line-no">1924</span><span id="line-1924"> .setInMemoryCompaction(MemoryCompactionPolicy.NONE).build());</span>
<span class="source-line-no">1925</span><span id="line-1925"> assertTrue(((MemStoreCompactor) ((CompactingMemStore) store.memstore).compactor).toString()</span>
<span class="source-line-no">1926</span><span id="line-1926"> .startsWith("eager".toUpperCase()));</span>
<span class="source-line-no">1927</span><span id="line-1927"> }</span>
<span class="source-line-no">1928</span><span id="line-1928"></span>
<span class="source-line-no">1929</span><span id="line-1929"> @Test</span>
<span class="source-line-no">1930</span><span id="line-1930"> public void testSpaceQuotaChangeAfterReplacement() throws IOException {</span>
<span class="source-line-no">1931</span><span id="line-1931"> final TableName tn = TableName.valueOf(name.getMethodName());</span>
<span class="source-line-no">1932</span><span id="line-1932"> init(name.getMethodName());</span>
<span class="source-line-no">1933</span><span id="line-1933"></span>
<span class="source-line-no">1934</span><span id="line-1934"> RegionSizeStoreImpl sizeStore = new RegionSizeStoreImpl();</span>
<span class="source-line-no">1935</span><span id="line-1935"></span>
<span class="source-line-no">1936</span><span id="line-1936"> HStoreFile sf1 = mockStoreFileWithLength(1024L);</span>
<span class="source-line-no">1937</span><span id="line-1937"> HStoreFile sf2 = mockStoreFileWithLength(2048L);</span>
<span class="source-line-no">1938</span><span id="line-1938"> HStoreFile sf3 = mockStoreFileWithLength(4096L);</span>
<span class="source-line-no">1939</span><span id="line-1939"> HStoreFile sf4 = mockStoreFileWithLength(8192L);</span>
<span class="source-line-no">1940</span><span id="line-1940"></span>
<span class="source-line-no">1941</span><span id="line-1941"> RegionInfo regionInfo = RegionInfoBuilder.newBuilder(tn).setStartKey(Bytes.toBytes("a"))</span>
<span class="source-line-no">1942</span><span id="line-1942"> .setEndKey(Bytes.toBytes("b")).build();</span>
<span class="source-line-no">1943</span><span id="line-1943"></span>
<span class="source-line-no">1944</span><span id="line-1944"> // Compacting two files down to one, reducing size</span>
<span class="source-line-no">1945</span><span id="line-1945"> sizeStore.put(regionInfo, 1024L + 4096L);</span>
<span class="source-line-no">1946</span><span id="line-1946"> store.updateSpaceQuotaAfterFileReplacement(sizeStore, regionInfo, Arrays.asList(sf1, sf3),</span>
<span class="source-line-no">1947</span><span id="line-1947"> Arrays.asList(sf2));</span>
<span class="source-line-no">1948</span><span id="line-1948"></span>
<span class="source-line-no">1949</span><span id="line-1949"> assertEquals(2048L, sizeStore.getRegionSize(regionInfo).getSize());</span>
<span class="source-line-no">1950</span><span id="line-1950"></span>
<span class="source-line-no">1951</span><span id="line-1951"> // The same file length in and out should have no change</span>
<span class="source-line-no">1952</span><span id="line-1952"> store.updateSpaceQuotaAfterFileReplacement(sizeStore, regionInfo, Arrays.asList(sf2),</span>
<span class="source-line-no">1953</span><span id="line-1953"> Arrays.asList(sf2));</span>
<span class="source-line-no">1954</span><span id="line-1954"></span>
<span class="source-line-no">1955</span><span id="line-1955"> assertEquals(2048L, sizeStore.getRegionSize(regionInfo).getSize());</span>
<span class="source-line-no">1956</span><span id="line-1956"></span>
<span class="source-line-no">1957</span><span id="line-1957"> // Increase the total size used</span>
<span class="source-line-no">1958</span><span id="line-1958"> store.updateSpaceQuotaAfterFileReplacement(sizeStore, regionInfo, Arrays.asList(sf2),</span>
<span class="source-line-no">1959</span><span id="line-1959"> Arrays.asList(sf3));</span>
<span class="source-line-no">1960</span><span id="line-1960"></span>
<span class="source-line-no">1961</span><span id="line-1961"> assertEquals(4096L, sizeStore.getRegionSize(regionInfo).getSize());</span>
<span class="source-line-no">1962</span><span id="line-1962"></span>
<span class="source-line-no">1963</span><span id="line-1963"> RegionInfo regionInfo2 = RegionInfoBuilder.newBuilder(tn).setStartKey(Bytes.toBytes("b"))</span>
<span class="source-line-no">1964</span><span id="line-1964"> .setEndKey(Bytes.toBytes("c")).build();</span>
<span class="source-line-no">1965</span><span id="line-1965"> store.updateSpaceQuotaAfterFileReplacement(sizeStore, regionInfo2, null, Arrays.asList(sf4));</span>
<span class="source-line-no">1966</span><span id="line-1966"></span>
<span class="source-line-no">1967</span><span id="line-1967"> assertEquals(8192L, sizeStore.getRegionSize(regionInfo2).getSize());</span>
<span class="source-line-no">1968</span><span id="line-1968"> }</span>
<span class="source-line-no">1969</span><span id="line-1969"></span>
<span class="source-line-no">1970</span><span id="line-1970"> @Test</span>
<span class="source-line-no">1971</span><span id="line-1971"> public void testHFileContextSetWithCFAndTable() throws Exception {</span>
<span class="source-line-no">1972</span><span id="line-1972"> init(this.name.getMethodName());</span>
<span class="source-line-no">1973</span><span id="line-1973"> StoreFileWriter writer = store.getStoreEngine()</span>
<span class="source-line-no">1974</span><span id="line-1974"> .createWriter(CreateStoreFileWriterParams.create().maxKeyCount(10000L)</span>
<span class="source-line-no">1975</span><span id="line-1975"> .compression(Compression.Algorithm.NONE).isCompaction(true).includeMVCCReadpoint(true)</span>
<span class="source-line-no">1976</span><span id="line-1976"> .includesTag(false).shouldDropBehind(true));</span>
<span class="source-line-no">1977</span><span id="line-1977"> HFileContext hFileContext = writer.getLiveFileWriter().getFileContext();</span>
<span class="source-line-no">1978</span><span id="line-1978"> assertArrayEquals(family, hFileContext.getColumnFamily());</span>
<span class="source-line-no">1979</span><span id="line-1979"> assertArrayEquals(table, hFileContext.getTableName());</span>
<span class="source-line-no">1980</span><span id="line-1980"> }</span>
<span class="source-line-no">1981</span><span id="line-1981"></span>
<span class="source-line-no">1982</span><span id="line-1982"> // This test is for HBASE-26026, HBase Write be stuck when active segment has no cell</span>
<span class="source-line-no">1983</span><span id="line-1983"> // but its dataSize exceeds inmemoryFlushSize</span>
<span class="source-line-no">1984</span><span id="line-1984"> @Test</span>
<span class="source-line-no">1985</span><span id="line-1985"> public void testCompactingMemStoreNoCellButDataSizeExceedsInmemoryFlushSize()</span>
<span class="source-line-no">1986</span><span id="line-1986"> throws IOException, InterruptedException {</span>
<span class="source-line-no">1987</span><span id="line-1987"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">1988</span><span id="line-1988"></span>
<span class="source-line-no">1989</span><span id="line-1989"> byte[] smallValue = new byte[3];</span>
<span class="source-line-no">1990</span><span id="line-1990"> byte[] largeValue = new byte[9];</span>
<span class="source-line-no">1991</span><span id="line-1991"> final long timestamp = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">1992</span><span id="line-1992"> final long seqId = 100;</span>
<span class="source-line-no">1993</span><span id="line-1993"> final ExtendedCell smallCell = createCell(qf1, timestamp, seqId, smallValue);</span>
<span class="source-line-no">1994</span><span id="line-1994"> final ExtendedCell largeCell = createCell(qf2, timestamp, seqId, largeValue);</span>
<span class="source-line-no">1995</span><span id="line-1995"> int smallCellByteSize = MutableSegment.getCellLength(smallCell);</span>
<span class="source-line-no">1996</span><span id="line-1996"> int largeCellByteSize = MutableSegment.getCellLength(largeCell);</span>
<span class="source-line-no">1997</span><span id="line-1997"> int flushByteSize = smallCellByteSize + largeCellByteSize - 2;</span>
<span class="source-line-no">1998</span><span id="line-1998"></span>
<span class="source-line-no">1999</span><span id="line-1999"> // set CompactingMemStore.inmemoryFlushSize to flushByteSize.</span>
<span class="source-line-no">2000</span><span id="line-2000"> conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore2.class.getName());</span>
<span class="source-line-no">2001</span><span id="line-2001"> conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.005);</span>
<span class="source-line-no">2002</span><span id="line-2002"> conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushByteSize * 200));</span>
<span class="source-line-no">2003</span><span id="line-2003"></span>
<span class="source-line-no">2004</span><span id="line-2004"> init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)</span>
<span class="source-line-no">2005</span><span id="line-2005"> .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());</span>
<span class="source-line-no">2006</span><span id="line-2006"></span>
<span class="source-line-no">2007</span><span id="line-2007"> MyCompactingMemStore2 myCompactingMemStore = ((MyCompactingMemStore2) store.memstore);</span>
<span class="source-line-no">2008</span><span id="line-2008"> assertTrue((int) (myCompactingMemStore.getInmemoryFlushSize()) == flushByteSize);</span>
<span class="source-line-no">2009</span><span id="line-2009"> myCompactingMemStore.smallCellPreUpdateCounter.set(0);</span>
<span class="source-line-no">2010</span><span id="line-2010"> myCompactingMemStore.largeCellPreUpdateCounter.set(0);</span>
<span class="source-line-no">2011</span><span id="line-2011"></span>
<span class="source-line-no">2012</span><span id="line-2012"> final AtomicReference&lt;Throwable&gt; exceptionRef = new AtomicReference&lt;Throwable&gt;();</span>
<span class="source-line-no">2013</span><span id="line-2013"> Thread smallCellThread = new Thread(() -&gt; {</span>
<span class="source-line-no">2014</span><span id="line-2014"> try {</span>
<span class="source-line-no">2015</span><span id="line-2015"> store.add(smallCell, new NonThreadSafeMemStoreSizing());</span>
<span class="source-line-no">2016</span><span id="line-2016"> } catch (Throwable exception) {</span>
<span class="source-line-no">2017</span><span id="line-2017"> exceptionRef.set(exception);</span>
<span class="source-line-no">2018</span><span id="line-2018"> }</span>
<span class="source-line-no">2019</span><span id="line-2019"> });</span>
<span class="source-line-no">2020</span><span id="line-2020"> smallCellThread.setName(MyCompactingMemStore2.SMALL_CELL_THREAD_NAME);</span>
<span class="source-line-no">2021</span><span id="line-2021"> smallCellThread.start();</span>
<span class="source-line-no">2022</span><span id="line-2022"></span>
<span class="source-line-no">2023</span><span id="line-2023"> String oldThreadName = Thread.currentThread().getName();</span>
<span class="source-line-no">2024</span><span id="line-2024"> try {</span>
<span class="source-line-no">2025</span><span id="line-2025"> /**</span>
<span class="source-line-no">2026</span><span id="line-2026"> * 1.smallCellThread enters CompactingMemStore.checkAndAddToActiveSize first, then</span>
<span class="source-line-no">2027</span><span id="line-2027"> * largeCellThread enters CompactingMemStore.checkAndAddToActiveSize, and then largeCellThread</span>
<span class="source-line-no">2028</span><span id="line-2028"> * invokes flushInMemory.</span>
<span class="source-line-no">2029</span><span id="line-2029"> * &lt;p/&gt;</span>
<span class="source-line-no">2030</span><span id="line-2030"> * 2. After largeCellThread finished CompactingMemStore.flushInMemory method, smallCellThread</span>
<span class="source-line-no">2031</span><span id="line-2031"> * can add cell to currentActive . That is to say when largeCellThread called flushInMemory</span>
<span class="source-line-no">2032</span><span id="line-2032"> * method, CompactingMemStore.active has no cell.</span>
<span class="source-line-no">2033</span><span id="line-2033"> */</span>
<span class="source-line-no">2034</span><span id="line-2034"> Thread.currentThread().setName(MyCompactingMemStore2.LARGE_CELL_THREAD_NAME);</span>
<span class="source-line-no">2035</span><span id="line-2035"> store.add(largeCell, new NonThreadSafeMemStoreSizing());</span>
<span class="source-line-no">2036</span><span id="line-2036"> smallCellThread.join();</span>
<span class="source-line-no">2037</span><span id="line-2037"></span>
<span class="source-line-no">2038</span><span id="line-2038"> for (int i = 0; i &lt; 100; i++) {</span>
<span class="source-line-no">2039</span><span id="line-2039"> long currentTimestamp = timestamp + 100 + i;</span>
<span class="source-line-no">2040</span><span id="line-2040"> ExtendedCell cell = createCell(qf2, currentTimestamp, seqId, largeValue);</span>
<span class="source-line-no">2041</span><span id="line-2041"> store.add(cell, new NonThreadSafeMemStoreSizing());</span>
<span class="source-line-no">2042</span><span id="line-2042"> }</span>
<span class="source-line-no">2043</span><span id="line-2043"> } finally {</span>
<span class="source-line-no">2044</span><span id="line-2044"> Thread.currentThread().setName(oldThreadName);</span>
<span class="source-line-no">2045</span><span id="line-2045"> }</span>
<span class="source-line-no">2046</span><span id="line-2046"></span>
<span class="source-line-no">2047</span><span id="line-2047"> assertTrue(exceptionRef.get() == null);</span>
<span class="source-line-no">2048</span><span id="line-2048"></span>
<span class="source-line-no">2049</span><span id="line-2049"> }</span>
<span class="source-line-no">2050</span><span id="line-2050"></span>
<span class="source-line-no">2051</span><span id="line-2051"> // This test is for HBASE-26210, HBase Write be stuck when there is cell which size exceeds</span>
<span class="source-line-no">2052</span><span id="line-2052"> // InmemoryFlushSize</span>
<span class="source-line-no">2053</span><span id="line-2053"> @Test(timeout = 60000)</span>
<span class="source-line-no">2054</span><span id="line-2054"> public void testCompactingMemStoreCellExceedInmemoryFlushSize() throws Exception {</span>
<span class="source-line-no">2055</span><span id="line-2055"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">2056</span><span id="line-2056"> conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore6.class.getName());</span>
<span class="source-line-no">2057</span><span id="line-2057"></span>
<span class="source-line-no">2058</span><span id="line-2058"> init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)</span>
<span class="source-line-no">2059</span><span id="line-2059"> .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());</span>
<span class="source-line-no">2060</span><span id="line-2060"></span>
<span class="source-line-no">2061</span><span id="line-2061"> MyCompactingMemStore6 myCompactingMemStore = ((MyCompactingMemStore6) store.memstore);</span>
<span class="source-line-no">2062</span><span id="line-2062"></span>
<span class="source-line-no">2063</span><span id="line-2063"> int size = (int) (myCompactingMemStore.getInmemoryFlushSize());</span>
<span class="source-line-no">2064</span><span id="line-2064"> byte[] value = new byte[size + 1];</span>
<span class="source-line-no">2065</span><span id="line-2065"></span>
<span class="source-line-no">2066</span><span id="line-2066"> MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();</span>
<span class="source-line-no">2067</span><span id="line-2067"> long timestamp = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">2068</span><span id="line-2068"> long seqId = 100;</span>
<span class="source-line-no">2069</span><span id="line-2069"> ExtendedCell cell = createCell(qf1, timestamp, seqId, value);</span>
<span class="source-line-no">2070</span><span id="line-2070"> int cellByteSize = MutableSegment.getCellLength(cell);</span>
<span class="source-line-no">2071</span><span id="line-2071"> store.add(cell, memStoreSizing);</span>
<span class="source-line-no">2072</span><span id="line-2072"> assertTrue(memStoreSizing.getCellsCount() == 1);</span>
<span class="source-line-no">2073</span><span id="line-2073"> assertTrue(memStoreSizing.getDataSize() == cellByteSize);</span>
<span class="source-line-no">2074</span><span id="line-2074"> // Waiting the in memory compaction completed, see HBASE-26438</span>
<span class="source-line-no">2075</span><span id="line-2075"> myCompactingMemStore.inMemoryCompactionEndCyclicBarrier.await();</span>
<span class="source-line-no">2076</span><span id="line-2076"> }</span>
<span class="source-line-no">2077</span><span id="line-2077"></span>
<span class="source-line-no">2078</span><span id="line-2078"> /**</span>
<span class="source-line-no">2079</span><span id="line-2079"> * This test is for HBASE-27464, before this JIRA,when init {@link CellChunkImmutableSegment} for</span>
<span class="source-line-no">2080</span><span id="line-2080"> * 'COMPACT' action, we not force copy to current MSLab. When cell size bigger than</span>
<span class="source-line-no">2081</span><span id="line-2081"> * {@link MemStoreLABImpl#maxAlloc}, cell will stay in previous chunk which will recycle after</span>
<span class="source-line-no">2082</span><span id="line-2082"> * segment replace, and we may read wrong data when these chunk reused by others.</span>
<span class="source-line-no">2083</span><span id="line-2083"> */</span>
<span class="source-line-no">2084</span><span id="line-2084"> @Test</span>
<span class="source-line-no">2085</span><span id="line-2085"> public void testForceCloneOfBigCellForCellChunkImmutableSegment() throws Exception {</span>
<span class="source-line-no">2086</span><span id="line-2086"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">2087</span><span id="line-2087"> int maxAllocByteSize = conf.getInt(MemStoreLAB.MAX_ALLOC_KEY, MemStoreLAB.MAX_ALLOC_DEFAULT);</span>
<span class="source-line-no">2088</span><span id="line-2088"></span>
<span class="source-line-no">2089</span><span id="line-2089"> // Construct big cell,which is large than {@link MemStoreLABImpl#maxAlloc}.</span>
<span class="source-line-no">2090</span><span id="line-2090"> byte[] cellValue = new byte[maxAllocByteSize + 1];</span>
<span class="source-line-no">2091</span><span id="line-2091"> final long timestamp = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">2092</span><span id="line-2092"> final long seqId = 100;</span>
<span class="source-line-no">2093</span><span id="line-2093"> final byte[] rowKey1 = Bytes.toBytes("rowKey1");</span>
<span class="source-line-no">2094</span><span id="line-2094"> final ExtendedCell originalCell1 = createCell(rowKey1, qf1, timestamp, seqId, cellValue);</span>
<span class="source-line-no">2095</span><span id="line-2095"> final byte[] rowKey2 = Bytes.toBytes("rowKey2");</span>
<span class="source-line-no">2096</span><span id="line-2096"> final ExtendedCell originalCell2 = createCell(rowKey2, qf1, timestamp, seqId, cellValue);</span>
<span class="source-line-no">2097</span><span id="line-2097"> TreeSet&lt;byte[]&gt; quals = new TreeSet&lt;&gt;(Bytes.BYTES_COMPARATOR);</span>
<span class="source-line-no">2098</span><span id="line-2098"> quals.add(qf1);</span>
<span class="source-line-no">2099</span><span id="line-2099"></span>
<span class="source-line-no">2100</span><span id="line-2100"> int cellByteSize = MutableSegment.getCellLength(originalCell1);</span>
<span class="source-line-no">2101</span><span id="line-2101"> int inMemoryFlushByteSize = cellByteSize - 1;</span>
<span class="source-line-no">2102</span><span id="line-2102"></span>
<span class="source-line-no">2103</span><span id="line-2103"> // set CompactingMemStore.inmemoryFlushSize to flushByteSize.</span>
<span class="source-line-no">2104</span><span id="line-2104"> conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore6.class.getName());</span>
<span class="source-line-no">2105</span><span id="line-2105"> conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.005);</span>
<span class="source-line-no">2106</span><span id="line-2106"> conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(inMemoryFlushByteSize * 200));</span>
<span class="source-line-no">2107</span><span id="line-2107"> conf.setBoolean(WALFactory.WAL_ENABLED, false);</span>
<span class="source-line-no">2108</span><span id="line-2108"></span>
<span class="source-line-no">2109</span><span id="line-2109"> // Use {@link MemoryCompactionPolicy#EAGER} for always compacting.</span>
<span class="source-line-no">2110</span><span id="line-2110"> init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)</span>
<span class="source-line-no">2111</span><span id="line-2111"> .setInMemoryCompaction(MemoryCompactionPolicy.EAGER).build());</span>
<span class="source-line-no">2112</span><span id="line-2112"></span>
<span class="source-line-no">2113</span><span id="line-2113"> MyCompactingMemStore6 myCompactingMemStore = ((MyCompactingMemStore6) store.memstore);</span>
<span class="source-line-no">2114</span><span id="line-2114"> assertTrue((int) (myCompactingMemStore.getInmemoryFlushSize()) == inMemoryFlushByteSize);</span>
<span class="source-line-no">2115</span><span id="line-2115"></span>
<span class="source-line-no">2116</span><span id="line-2116"> // Data chunk Pool is disabled.</span>
<span class="source-line-no">2117</span><span id="line-2117"> assertTrue(ChunkCreator.getInstance().getMaxCount(ChunkType.DATA_CHUNK) == 0);</span>
<span class="source-line-no">2118</span><span id="line-2118"></span>
<span class="source-line-no">2119</span><span id="line-2119"> MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();</span>
<span class="source-line-no">2120</span><span id="line-2120"></span>
<span class="source-line-no">2121</span><span id="line-2121"> // First compact</span>
<span class="source-line-no">2122</span><span id="line-2122"> store.add(originalCell1, memStoreSizing);</span>
<span class="source-line-no">2123</span><span id="line-2123"> // Waiting for the first in-memory compaction finished</span>
<span class="source-line-no">2124</span><span id="line-2124"> myCompactingMemStore.inMemoryCompactionEndCyclicBarrier.await();</span>
<span class="source-line-no">2125</span><span id="line-2125"></span>
<span class="source-line-no">2126</span><span id="line-2126"> StoreScanner storeScanner =</span>
<span class="source-line-no">2127</span><span id="line-2127"> (StoreScanner) store.getScanner(new Scan(new Get(rowKey1)), quals, seqId + 1);</span>
<span class="source-line-no">2128</span><span id="line-2128"> SegmentScanner segmentScanner = getTypeKeyValueScanner(storeScanner, SegmentScanner.class);</span>
<span class="source-line-no">2129</span><span id="line-2129"> ExtendedCell resultCell1 = segmentScanner.next();</span>
<span class="source-line-no">2130</span><span id="line-2130"> assertTrue(PrivateCellUtil.equals(resultCell1, originalCell1));</span>
<span class="source-line-no">2131</span><span id="line-2131"> int cell1ChunkId = resultCell1.getChunkId();</span>
<span class="source-line-no">2132</span><span id="line-2132"> assertTrue(cell1ChunkId != ExtendedCell.CELL_NOT_BASED_ON_CHUNK);</span>
<span class="source-line-no">2133</span><span id="line-2133"> assertNull(segmentScanner.next());</span>
<span class="source-line-no">2134</span><span id="line-2134"> segmentScanner.close();</span>
<span class="source-line-no">2135</span><span id="line-2135"> storeScanner.close();</span>
<span class="source-line-no">2136</span><span id="line-2136"> Segment segment = segmentScanner.segment;</span>
<span class="source-line-no">2137</span><span id="line-2137"> assertTrue(segment instanceof CellChunkImmutableSegment);</span>
<span class="source-line-no">2138</span><span id="line-2138"> MemStoreLABImpl memStoreLAB1 = (MemStoreLABImpl) (segmentScanner.segment.getMemStoreLAB());</span>
<span class="source-line-no">2139</span><span id="line-2139"> assertTrue(!memStoreLAB1.isClosed());</span>
<span class="source-line-no">2140</span><span id="line-2140"> assertTrue(!memStoreLAB1.chunks.isEmpty());</span>
<span class="source-line-no">2141</span><span id="line-2141"> assertTrue(!memStoreLAB1.isReclaimed());</span>
<span class="source-line-no">2142</span><span id="line-2142"></span>
<span class="source-line-no">2143</span><span id="line-2143"> // Second compact</span>
<span class="source-line-no">2144</span><span id="line-2144"> store.add(originalCell2, memStoreSizing);</span>
<span class="source-line-no">2145</span><span id="line-2145"> // Waiting for the second in-memory compaction finished</span>
<span class="source-line-no">2146</span><span id="line-2146"> myCompactingMemStore.inMemoryCompactionEndCyclicBarrier.await();</span>
<span class="source-line-no">2147</span><span id="line-2147"></span>
<span class="source-line-no">2148</span><span id="line-2148"> // Before HBASE-27464, here may throw java.lang.IllegalArgumentException: In CellChunkMap, cell</span>
<span class="source-line-no">2149</span><span id="line-2149"> // must be associated with chunk.. We were looking for a cell at index 0.</span>
<span class="source-line-no">2150</span><span id="line-2150"> // The cause for this exception is because the data chunk Pool is disabled,when the data chunks</span>
<span class="source-line-no">2151</span><span id="line-2151"> // are recycled after the second in-memory compaction finished,the</span>
<span class="source-line-no">2152</span><span id="line-2152"> // {@link ChunkCreator.putbackChunks} method does not put the chunks back to the data chunk</span>
<span class="source-line-no">2153</span><span id="line-2153"> // pool,it just removes them from {@link ChunkCreator#chunkIdMap},so in</span>
<span class="source-line-no">2154</span><span id="line-2154"> // {@link CellChunkMap#getCell} we could not get the data chunk by chunkId.</span>
<span class="source-line-no">2155</span><span id="line-2155"> storeScanner = (StoreScanner) store.getScanner(new Scan(new Get(rowKey1)), quals, seqId + 1);</span>
<span class="source-line-no">2156</span><span id="line-2156"> segmentScanner = getTypeKeyValueScanner(storeScanner, SegmentScanner.class);</span>
<span class="source-line-no">2157</span><span id="line-2157"> ExtendedCell newResultCell1 = segmentScanner.next();</span>
<span class="source-line-no">2158</span><span id="line-2158"> assertTrue(newResultCell1 != resultCell1);</span>
<span class="source-line-no">2159</span><span id="line-2159"> assertTrue(PrivateCellUtil.equals(newResultCell1, originalCell1));</span>
<span class="source-line-no">2160</span><span id="line-2160"></span>
<span class="source-line-no">2161</span><span id="line-2161"> ExtendedCell resultCell2 = segmentScanner.next();</span>
<span class="source-line-no">2162</span><span id="line-2162"> assertTrue(PrivateCellUtil.equals(resultCell2, originalCell2));</span>
<span class="source-line-no">2163</span><span id="line-2163"> assertNull(segmentScanner.next());</span>
<span class="source-line-no">2164</span><span id="line-2164"> segmentScanner.close();</span>
<span class="source-line-no">2165</span><span id="line-2165"> storeScanner.close();</span>
<span class="source-line-no">2166</span><span id="line-2166"></span>
<span class="source-line-no">2167</span><span id="line-2167"> segment = segmentScanner.segment;</span>
<span class="source-line-no">2168</span><span id="line-2168"> assertTrue(segment instanceof CellChunkImmutableSegment);</span>
<span class="source-line-no">2169</span><span id="line-2169"> MemStoreLABImpl memStoreLAB2 = (MemStoreLABImpl) (segmentScanner.segment.getMemStoreLAB());</span>
<span class="source-line-no">2170</span><span id="line-2170"> assertTrue(!memStoreLAB2.isClosed());</span>
<span class="source-line-no">2171</span><span id="line-2171"> assertTrue(!memStoreLAB2.chunks.isEmpty());</span>
<span class="source-line-no">2172</span><span id="line-2172"> assertTrue(!memStoreLAB2.isReclaimed());</span>
<span class="source-line-no">2173</span><span id="line-2173"> assertTrue(memStoreLAB1.isClosed());</span>
<span class="source-line-no">2174</span><span id="line-2174"> assertTrue(memStoreLAB1.chunks.isEmpty());</span>
<span class="source-line-no">2175</span><span id="line-2175"> assertTrue(memStoreLAB1.isReclaimed());</span>
<span class="source-line-no">2176</span><span id="line-2176"> }</span>
<span class="source-line-no">2177</span><span id="line-2177"></span>
<span class="source-line-no">2178</span><span id="line-2178"> // This test is for HBASE-26210 also, test write large cell and small cell concurrently when</span>
<span class="source-line-no">2179</span><span id="line-2179"> // InmemoryFlushSize is smaller,equal with and larger than cell size.</span>
<span class="source-line-no">2180</span><span id="line-2180"> @Test</span>
<span class="source-line-no">2181</span><span id="line-2181"> public void testCompactingMemStoreWriteLargeCellAndSmallCellConcurrently()</span>
<span class="source-line-no">2182</span><span id="line-2182"> throws IOException, InterruptedException {</span>
<span class="source-line-no">2183</span><span id="line-2183"> doWriteTestLargeCellAndSmallCellConcurrently(</span>
<span class="source-line-no">2184</span><span id="line-2184"> (smallCellByteSize, largeCellByteSize) -&gt; largeCellByteSize - 1);</span>
<span class="source-line-no">2185</span><span id="line-2185"> doWriteTestLargeCellAndSmallCellConcurrently(</span>
<span class="source-line-no">2186</span><span id="line-2186"> (smallCellByteSize, largeCellByteSize) -&gt; largeCellByteSize);</span>
<span class="source-line-no">2187</span><span id="line-2187"> doWriteTestLargeCellAndSmallCellConcurrently(</span>
<span class="source-line-no">2188</span><span id="line-2188"> (smallCellByteSize, largeCellByteSize) -&gt; smallCellByteSize + largeCellByteSize - 1);</span>
<span class="source-line-no">2189</span><span id="line-2189"> doWriteTestLargeCellAndSmallCellConcurrently(</span>
<span class="source-line-no">2190</span><span id="line-2190"> (smallCellByteSize, largeCellByteSize) -&gt; smallCellByteSize + largeCellByteSize);</span>
<span class="source-line-no">2191</span><span id="line-2191"> doWriteTestLargeCellAndSmallCellConcurrently(</span>
<span class="source-line-no">2192</span><span id="line-2192"> (smallCellByteSize, largeCellByteSize) -&gt; smallCellByteSize + largeCellByteSize + 1);</span>
<span class="source-line-no">2193</span><span id="line-2193"> }</span>
<span class="source-line-no">2194</span><span id="line-2194"></span>
<span class="source-line-no">2195</span><span id="line-2195"> private void doWriteTestLargeCellAndSmallCellConcurrently(IntBinaryOperator getFlushByteSize)</span>
<span class="source-line-no">2196</span><span id="line-2196"> throws IOException, InterruptedException {</span>
<span class="source-line-no">2197</span><span id="line-2197"></span>
<span class="source-line-no">2198</span><span id="line-2198"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">2199</span><span id="line-2199"></span>
<span class="source-line-no">2200</span><span id="line-2200"> byte[] smallValue = new byte[3];</span>
<span class="source-line-no">2201</span><span id="line-2201"> byte[] largeValue = new byte[100];</span>
<span class="source-line-no">2202</span><span id="line-2202"> final long timestamp = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">2203</span><span id="line-2203"> final long seqId = 100;</span>
<span class="source-line-no">2204</span><span id="line-2204"> final Cell smallCell = createCell(qf1, timestamp, seqId, smallValue);</span>
<span class="source-line-no">2205</span><span id="line-2205"> final Cell largeCell = createCell(qf2, timestamp, seqId, largeValue);</span>
<span class="source-line-no">2206</span><span id="line-2206"> int smallCellByteSize = MutableSegment.getCellLength(smallCell);</span>
<span class="source-line-no">2207</span><span id="line-2207"> int largeCellByteSize = MutableSegment.getCellLength(largeCell);</span>
<span class="source-line-no">2208</span><span id="line-2208"> int flushByteSize = getFlushByteSize.applyAsInt(smallCellByteSize, largeCellByteSize);</span>
<span class="source-line-no">2209</span><span id="line-2209"> boolean flushByteSizeLessThanSmallAndLargeCellSize =</span>
<span class="source-line-no">2210</span><span id="line-2210"> flushByteSize &lt; (smallCellByteSize + largeCellByteSize);</span>
<span class="source-line-no">2211</span><span id="line-2211"></span>
<span class="source-line-no">2212</span><span id="line-2212"> conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore3.class.getName());</span>
<span class="source-line-no">2213</span><span id="line-2213"> conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.005);</span>
<span class="source-line-no">2214</span><span id="line-2214"> conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushByteSize * 200));</span>
<span class="source-line-no">2215</span><span id="line-2215"></span>
<span class="source-line-no">2216</span><span id="line-2216"> init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)</span>
<span class="source-line-no">2217</span><span id="line-2217"> .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());</span>
<span class="source-line-no">2218</span><span id="line-2218"></span>
<span class="source-line-no">2219</span><span id="line-2219"> MyCompactingMemStore3 myCompactingMemStore = ((MyCompactingMemStore3) store.memstore);</span>
<span class="source-line-no">2220</span><span id="line-2220"> assertTrue((int) (myCompactingMemStore.getInmemoryFlushSize()) == flushByteSize);</span>
<span class="source-line-no">2221</span><span id="line-2221"> myCompactingMemStore.disableCompaction();</span>
<span class="source-line-no">2222</span><span id="line-2222"> if (flushByteSizeLessThanSmallAndLargeCellSize) {</span>
<span class="source-line-no">2223</span><span id="line-2223"> myCompactingMemStore.flushByteSizeLessThanSmallAndLargeCellSize = true;</span>
<span class="source-line-no">2224</span><span id="line-2224"> } else {</span>
<span class="source-line-no">2225</span><span id="line-2225"> myCompactingMemStore.flushByteSizeLessThanSmallAndLargeCellSize = false;</span>
<span class="source-line-no">2226</span><span id="line-2226"> }</span>
<span class="source-line-no">2227</span><span id="line-2227"></span>
<span class="source-line-no">2228</span><span id="line-2228"> final ThreadSafeMemStoreSizing memStoreSizing = new ThreadSafeMemStoreSizing();</span>
<span class="source-line-no">2229</span><span id="line-2229"> final AtomicLong totalCellByteSize = new AtomicLong(0);</span>
<span class="source-line-no">2230</span><span id="line-2230"> final AtomicReference&lt;Throwable&gt; exceptionRef = new AtomicReference&lt;Throwable&gt;();</span>
<span class="source-line-no">2231</span><span id="line-2231"> Thread smallCellThread = new Thread(() -&gt; {</span>
<span class="source-line-no">2232</span><span id="line-2232"> try {</span>
<span class="source-line-no">2233</span><span id="line-2233"> for (int i = 1; i &lt;= MyCompactingMemStore3.CELL_COUNT; i++) {</span>
<span class="source-line-no">2234</span><span id="line-2234"> long currentTimestamp = timestamp + i;</span>
<span class="source-line-no">2235</span><span id="line-2235"> ExtendedCell cell = createCell(qf1, currentTimestamp, seqId, smallValue);</span>
<span class="source-line-no">2236</span><span id="line-2236"> totalCellByteSize.addAndGet(MutableSegment.getCellLength(cell));</span>
<span class="source-line-no">2237</span><span id="line-2237"> store.add(cell, memStoreSizing);</span>
<span class="source-line-no">2238</span><span id="line-2238"> }</span>
<span class="source-line-no">2239</span><span id="line-2239"> } catch (Throwable exception) {</span>
<span class="source-line-no">2240</span><span id="line-2240"> exceptionRef.set(exception);</span>
<span class="source-line-no">2241</span><span id="line-2241"></span>
<span class="source-line-no">2242</span><span id="line-2242"> }</span>
<span class="source-line-no">2243</span><span id="line-2243"> });</span>
<span class="source-line-no">2244</span><span id="line-2244"> smallCellThread.setName(MyCompactingMemStore3.SMALL_CELL_THREAD_NAME);</span>
<span class="source-line-no">2245</span><span id="line-2245"> smallCellThread.start();</span>
<span class="source-line-no">2246</span><span id="line-2246"></span>
<span class="source-line-no">2247</span><span id="line-2247"> String oldThreadName = Thread.currentThread().getName();</span>
<span class="source-line-no">2248</span><span id="line-2248"> try {</span>
<span class="source-line-no">2249</span><span id="line-2249"> /**</span>
<span class="source-line-no">2250</span><span id="line-2250"> * When flushByteSizeLessThanSmallAndLargeCellSize is true:</span>
<span class="source-line-no">2251</span><span id="line-2251"> * &lt;/p&gt;</span>
<span class="source-line-no">2252</span><span id="line-2252"> * 1.smallCellThread enters MyCompactingMemStore3.checkAndAddToActiveSize first, then</span>
<span class="source-line-no">2253</span><span id="line-2253"> * largeCellThread enters MyCompactingMemStore3.checkAndAddToActiveSize, and then</span>
<span class="source-line-no">2254</span><span id="line-2254"> * largeCellThread invokes flushInMemory.</span>
<span class="source-line-no">2255</span><span id="line-2255"> * &lt;p/&gt;</span>
<span class="source-line-no">2256</span><span id="line-2256"> * 2. After largeCellThread finished CompactingMemStore.flushInMemory method, smallCellThread</span>
<span class="source-line-no">2257</span><span id="line-2257"> * can run into MyCompactingMemStore3.checkAndAddToActiveSize again.</span>
<span class="source-line-no">2258</span><span id="line-2258"> * &lt;p/&gt;</span>
<span class="source-line-no">2259</span><span id="line-2259"> * When flushByteSizeLessThanSmallAndLargeCellSize is false: smallCellThread and</span>
<span class="source-line-no">2260</span><span id="line-2260"> * largeCellThread concurrently write one cell and wait each other, and then write another</span>
<span class="source-line-no">2261</span><span id="line-2261"> * cell etc.</span>
<span class="source-line-no">2262</span><span id="line-2262"> */</span>
<span class="source-line-no">2263</span><span id="line-2263"> Thread.currentThread().setName(MyCompactingMemStore3.LARGE_CELL_THREAD_NAME);</span>
<span class="source-line-no">2264</span><span id="line-2264"> for (int i = 1; i &lt;= MyCompactingMemStore3.CELL_COUNT; i++) {</span>
<span class="source-line-no">2265</span><span id="line-2265"> long currentTimestamp = timestamp + i;</span>
<span class="source-line-no">2266</span><span id="line-2266"> ExtendedCell cell = createCell(qf2, currentTimestamp, seqId, largeValue);</span>
<span class="source-line-no">2267</span><span id="line-2267"> totalCellByteSize.addAndGet(MutableSegment.getCellLength(cell));</span>
<span class="source-line-no">2268</span><span id="line-2268"> store.add(cell, memStoreSizing);</span>
<span class="source-line-no">2269</span><span id="line-2269"> }</span>
<span class="source-line-no">2270</span><span id="line-2270"> smallCellThread.join();</span>
<span class="source-line-no">2271</span><span id="line-2271"></span>
<span class="source-line-no">2272</span><span id="line-2272"> assertTrue(exceptionRef.get() == null);</span>
<span class="source-line-no">2273</span><span id="line-2273"> assertTrue(memStoreSizing.getCellsCount() == (MyCompactingMemStore3.CELL_COUNT * 2));</span>
<span class="source-line-no">2274</span><span id="line-2274"> assertTrue(memStoreSizing.getDataSize() == totalCellByteSize.get());</span>
<span class="source-line-no">2275</span><span id="line-2275"> if (flushByteSizeLessThanSmallAndLargeCellSize) {</span>
<span class="source-line-no">2276</span><span id="line-2276"> assertTrue(myCompactingMemStore.flushCounter.get() == MyCompactingMemStore3.CELL_COUNT);</span>
<span class="source-line-no">2277</span><span id="line-2277"> } else {</span>
<span class="source-line-no">2278</span><span id="line-2278"> assertTrue(</span>
<span class="source-line-no">2279</span><span id="line-2279"> myCompactingMemStore.flushCounter.get() &lt;= (MyCompactingMemStore3.CELL_COUNT - 1));</span>
<span class="source-line-no">2280</span><span id="line-2280"> }</span>
<span class="source-line-no">2281</span><span id="line-2281"> } finally {</span>
<span class="source-line-no">2282</span><span id="line-2282"> Thread.currentThread().setName(oldThreadName);</span>
<span class="source-line-no">2283</span><span id="line-2283"> }</span>
<span class="source-line-no">2284</span><span id="line-2284"> }</span>
<span class="source-line-no">2285</span><span id="line-2285"></span>
<span class="source-line-no">2286</span><span id="line-2286"> /**</span>
<span class="source-line-no">2287</span><span id="line-2287"> * &lt;pre&gt;</span>
<span class="source-line-no">2288</span><span id="line-2288"> * This test is for HBASE-26384,</span>
<span class="source-line-no">2289</span><span id="line-2289"> * test {@link CompactingMemStore#flattenOneSegment} and {@link CompactingMemStore#snapshot()}</span>
<span class="source-line-no">2290</span><span id="line-2290"> * execute concurrently.</span>
<span class="source-line-no">2291</span><span id="line-2291"> * The threads sequence before HBASE-26384 is(The bug only exists for branch-2,and I add UTs</span>
<span class="source-line-no">2292</span><span id="line-2292"> * for both branch-2 and master):</span>
<span class="source-line-no">2293</span><span id="line-2293"> * 1. The {@link CompactingMemStore} size exceeds</span>
<span class="source-line-no">2294</span><span id="line-2294"> * {@link CompactingMemStore#getInmemoryFlushSize()},the write thread adds a new</span>
<span class="source-line-no">2295</span><span id="line-2295"> * {@link ImmutableSegment} to the head of {@link CompactingMemStore#pipeline},and start a</span>
<span class="source-line-no">2296</span><span id="line-2296"> * in memory compact thread to execute {@link CompactingMemStore#inMemoryCompaction}.</span>
<span class="source-line-no">2297</span><span id="line-2297"> * 2. The in memory compact thread starts and then stopping before</span>
<span class="source-line-no">2298</span><span id="line-2298"> * {@link CompactingMemStore#flattenOneSegment}.</span>
<span class="source-line-no">2299</span><span id="line-2299"> * 3. The snapshot thread starts {@link CompactingMemStore#snapshot} concurrently,after the</span>
<span class="source-line-no">2300</span><span id="line-2300"> * snapshot thread executing {@link CompactingMemStore#getImmutableSegments},the in memory</span>
<span class="source-line-no">2301</span><span id="line-2301"> * compact thread continues.</span>
<span class="source-line-no">2302</span><span id="line-2302"> * Assuming {@link VersionedSegmentsList#version} returned from</span>
<span class="source-line-no">2303</span><span id="line-2303"> * {@link CompactingMemStore#getImmutableSegments} is v.</span>
<span class="source-line-no">2304</span><span id="line-2304"> * 4. The snapshot thread stopping before {@link CompactingMemStore#swapPipelineWithNull}.</span>
<span class="source-line-no">2305</span><span id="line-2305"> * 5. The in memory compact thread completes {@link CompactingMemStore#flattenOneSegment},</span>
<span class="source-line-no">2306</span><span id="line-2306"> * {@link CompactionPipeline#version} is still v.</span>
<span class="source-line-no">2307</span><span id="line-2307"> * 6. The snapshot thread continues {@link CompactingMemStore#swapPipelineWithNull}, and because</span>
<span class="source-line-no">2308</span><span id="line-2308"> * {@link CompactionPipeline#version} is v, {@link CompactingMemStore#swapPipelineWithNull}</span>
<span class="source-line-no">2309</span><span id="line-2309"> * thinks it is successful and continue flushing,but the {@link ImmutableSegment} in</span>
<span class="source-line-no">2310</span><span id="line-2310"> * {@link CompactionPipeline} has changed because</span>
<span class="source-line-no">2311</span><span id="line-2311"> * {@link CompactingMemStore#flattenOneSegment},so the {@link ImmutableSegment} is not</span>
<span class="source-line-no">2312</span><span id="line-2312"> * removed in fact and still remaining in {@link CompactionPipeline}.</span>
<span class="source-line-no">2313</span><span id="line-2313"> *</span>
<span class="source-line-no">2314</span><span id="line-2314"> * After HBASE-26384, the 5-6 step is changed to following, which is expected behavior:</span>
<span class="source-line-no">2315</span><span id="line-2315"> * 5. The in memory compact thread completes {@link CompactingMemStore#flattenOneSegment},</span>
<span class="source-line-no">2316</span><span id="line-2316"> * {@link CompactingMemStore#flattenOneSegment} change {@link CompactionPipeline#version} to</span>
<span class="source-line-no">2317</span><span id="line-2317"> * v+1.</span>
<span class="source-line-no">2318</span><span id="line-2318"> * 6. The snapshot thread continues {@link CompactingMemStore#swapPipelineWithNull}, and because</span>
<span class="source-line-no">2319</span><span id="line-2319"> * {@link CompactionPipeline#version} is v+1, {@link CompactingMemStore#swapPipelineWithNull}</span>
<span class="source-line-no">2320</span><span id="line-2320"> * failed and retry the while loop in {@link CompactingMemStore#pushPipelineToSnapshot} once</span>
<span class="source-line-no">2321</span><span id="line-2321"> * again, because there is no concurrent {@link CompactingMemStore#inMemoryCompaction} now,</span>
<span class="source-line-no">2322</span><span id="line-2322"> * {@link CompactingMemStore#swapPipelineWithNull} succeeds.</span>
<span class="source-line-no">2323</span><span id="line-2323"> * &lt;/pre&gt;</span>
<span class="source-line-no">2324</span><span id="line-2324"> */</span>
<span class="source-line-no">2325</span><span id="line-2325"> @Test</span>
<span class="source-line-no">2326</span><span id="line-2326"> public void testFlattenAndSnapshotCompactingMemStoreConcurrently() throws Exception {</span>
<span class="source-line-no">2327</span><span id="line-2327"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">2328</span><span id="line-2328"></span>
<span class="source-line-no">2329</span><span id="line-2329"> byte[] smallValue = new byte[3];</span>
<span class="source-line-no">2330</span><span id="line-2330"> byte[] largeValue = new byte[9];</span>
<span class="source-line-no">2331</span><span id="line-2331"> final long timestamp = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">2332</span><span id="line-2332"> final long seqId = 100;</span>
<span class="source-line-no">2333</span><span id="line-2333"> final ExtendedCell smallCell = createCell(qf1, timestamp, seqId, smallValue);</span>
<span class="source-line-no">2334</span><span id="line-2334"> final ExtendedCell largeCell = createCell(qf2, timestamp, seqId, largeValue);</span>
<span class="source-line-no">2335</span><span id="line-2335"> int smallCellByteSize = MutableSegment.getCellLength(smallCell);</span>
<span class="source-line-no">2336</span><span id="line-2336"> int largeCellByteSize = MutableSegment.getCellLength(largeCell);</span>
<span class="source-line-no">2337</span><span id="line-2337"> int totalCellByteSize = (smallCellByteSize + largeCellByteSize);</span>
<span class="source-line-no">2338</span><span id="line-2338"> int flushByteSize = totalCellByteSize - 2;</span>
<span class="source-line-no">2339</span><span id="line-2339"></span>
<span class="source-line-no">2340</span><span id="line-2340"> // set CompactingMemStore.inmemoryFlushSize to flushByteSize.</span>
<span class="source-line-no">2341</span><span id="line-2341"> conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore4.class.getName());</span>
<span class="source-line-no">2342</span><span id="line-2342"> conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.005);</span>
<span class="source-line-no">2343</span><span id="line-2343"> conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushByteSize * 200));</span>
<span class="source-line-no">2344</span><span id="line-2344"></span>
<span class="source-line-no">2345</span><span id="line-2345"> init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)</span>
<span class="source-line-no">2346</span><span id="line-2346"> .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());</span>
<span class="source-line-no">2347</span><span id="line-2347"></span>
<span class="source-line-no">2348</span><span id="line-2348"> MyCompactingMemStore4 myCompactingMemStore = ((MyCompactingMemStore4) store.memstore);</span>
<span class="source-line-no">2349</span><span id="line-2349"> assertTrue((int) (myCompactingMemStore.getInmemoryFlushSize()) == flushByteSize);</span>
<span class="source-line-no">2350</span><span id="line-2350"></span>
<span class="source-line-no">2351</span><span id="line-2351"> store.add(smallCell, new NonThreadSafeMemStoreSizing());</span>
<span class="source-line-no">2352</span><span id="line-2352"> store.add(largeCell, new NonThreadSafeMemStoreSizing());</span>
<span class="source-line-no">2353</span><span id="line-2353"></span>
<span class="source-line-no">2354</span><span id="line-2354"> String oldThreadName = Thread.currentThread().getName();</span>
<span class="source-line-no">2355</span><span id="line-2355"> try {</span>
<span class="source-line-no">2356</span><span id="line-2356"> Thread.currentThread().setName(MyCompactingMemStore4.TAKE_SNAPSHOT_THREAD_NAME);</span>
<span class="source-line-no">2357</span><span id="line-2357"> /**</span>
<span class="source-line-no">2358</span><span id="line-2358"> * {@link CompactingMemStore#snapshot} must wait the in memory compact thread enters</span>
<span class="source-line-no">2359</span><span id="line-2359"> * {@link CompactingMemStore#flattenOneSegment},because {@link CompactingMemStore#snapshot}</span>
<span class="source-line-no">2360</span><span id="line-2360"> * would invoke {@link CompactingMemStore#stopCompaction}.</span>
<span class="source-line-no">2361</span><span id="line-2361"> */</span>
<span class="source-line-no">2362</span><span id="line-2362"> myCompactingMemStore.snapShotStartCyclicCyclicBarrier.await();</span>
<span class="source-line-no">2363</span><span id="line-2363"></span>
<span class="source-line-no">2364</span><span id="line-2364"> MemStoreSnapshot memStoreSnapshot = myCompactingMemStore.snapshot();</span>
<span class="source-line-no">2365</span><span id="line-2365"> myCompactingMemStore.inMemoryCompactionEndCyclicBarrier.await();</span>
<span class="source-line-no">2366</span><span id="line-2366"></span>
<span class="source-line-no">2367</span><span id="line-2367"> assertTrue(memStoreSnapshot.getCellsCount() == 2);</span>
<span class="source-line-no">2368</span><span id="line-2368"> assertTrue(((int) (memStoreSnapshot.getDataSize())) == totalCellByteSize);</span>
<span class="source-line-no">2369</span><span id="line-2369"> VersionedSegmentsList segments = myCompactingMemStore.getImmutableSegments();</span>
<span class="source-line-no">2370</span><span id="line-2370"> assertTrue(segments.getNumOfSegments() == 0);</span>
<span class="source-line-no">2371</span><span id="line-2371"> assertTrue(segments.getNumOfCells() == 0);</span>
<span class="source-line-no">2372</span><span id="line-2372"> assertTrue(myCompactingMemStore.setInMemoryCompactionFlagCounter.get() == 1);</span>
<span class="source-line-no">2373</span><span id="line-2373"> assertTrue(myCompactingMemStore.swapPipelineWithNullCounter.get() == 2);</span>
<span class="source-line-no">2374</span><span id="line-2374"> } finally {</span>
<span class="source-line-no">2375</span><span id="line-2375"> Thread.currentThread().setName(oldThreadName);</span>
<span class="source-line-no">2376</span><span id="line-2376"> }</span>
<span class="source-line-no">2377</span><span id="line-2377"> }</span>
<span class="source-line-no">2378</span><span id="line-2378"></span>
<span class="source-line-no">2379</span><span id="line-2379"> /**</span>
<span class="source-line-no">2380</span><span id="line-2380"> * &lt;pre&gt;</span>
<span class="source-line-no">2381</span><span id="line-2381"> * This test is for HBASE-26384,</span>
<span class="source-line-no">2382</span><span id="line-2382"> * test {@link CompactingMemStore#flattenOneSegment}{@link CompactingMemStore#snapshot()}</span>
<span class="source-line-no">2383</span><span id="line-2383"> * and writeMemStore execute concurrently.</span>
<span class="source-line-no">2384</span><span id="line-2384"> * The threads sequence before HBASE-26384 is(The bug only exists for branch-2,and I add UTs</span>
<span class="source-line-no">2385</span><span id="line-2385"> * for both branch-2 and master):</span>
<span class="source-line-no">2386</span><span id="line-2386"> * 1. The {@link CompactingMemStore} size exceeds</span>
<span class="source-line-no">2387</span><span id="line-2387"> * {@link CompactingMemStore#getInmemoryFlushSize()},the write thread adds a new</span>
<span class="source-line-no">2388</span><span id="line-2388"> * {@link ImmutableSegment} to the head of {@link CompactingMemStore#pipeline},and start a</span>
<span class="source-line-no">2389</span><span id="line-2389"> * in memory compact thread to execute {@link CompactingMemStore#inMemoryCompaction}.</span>
<span class="source-line-no">2390</span><span id="line-2390"> * 2. The in memory compact thread starts and then stopping before</span>
<span class="source-line-no">2391</span><span id="line-2391"> * {@link CompactingMemStore#flattenOneSegment}.</span>
<span class="source-line-no">2392</span><span id="line-2392"> * 3. The snapshot thread starts {@link CompactingMemStore#snapshot} concurrently,after the</span>
<span class="source-line-no">2393</span><span id="line-2393"> * snapshot thread executing {@link CompactingMemStore#getImmutableSegments},the in memory</span>
<span class="source-line-no">2394</span><span id="line-2394"> * compact thread continues.</span>
<span class="source-line-no">2395</span><span id="line-2395"> * Assuming {@link VersionedSegmentsList#version} returned from</span>
<span class="source-line-no">2396</span><span id="line-2396"> * {@link CompactingMemStore#getImmutableSegments} is v.</span>
<span class="source-line-no">2397</span><span id="line-2397"> * 4. The snapshot thread stopping before {@link CompactingMemStore#swapPipelineWithNull}.</span>
<span class="source-line-no">2398</span><span id="line-2398"> * 5. The in memory compact thread completes {@link CompactingMemStore#flattenOneSegment},</span>
<span class="source-line-no">2399</span><span id="line-2399"> * {@link CompactionPipeline#version} is still v.</span>
<span class="source-line-no">2400</span><span id="line-2400"> * 6. The snapshot thread continues {@link CompactingMemStore#swapPipelineWithNull}, and because</span>
<span class="source-line-no">2401</span><span id="line-2401"> * {@link CompactionPipeline#version} is v, {@link CompactingMemStore#swapPipelineWithNull}</span>
<span class="source-line-no">2402</span><span id="line-2402"> * thinks it is successful and continue flushing,but the {@link ImmutableSegment} in</span>
<span class="source-line-no">2403</span><span id="line-2403"> * {@link CompactionPipeline} has changed because</span>
<span class="source-line-no">2404</span><span id="line-2404"> * {@link CompactingMemStore#flattenOneSegment},so the {@link ImmutableSegment} is not</span>
<span class="source-line-no">2405</span><span id="line-2405"> * removed in fact and still remaining in {@link CompactionPipeline}.</span>
<span class="source-line-no">2406</span><span id="line-2406"> *</span>
<span class="source-line-no">2407</span><span id="line-2407"> * After HBASE-26384, the 5-6 step is changed to following, which is expected behavior,</span>
<span class="source-line-no">2408</span><span id="line-2408"> * and I add step 7-8 to test there is new segment added before retry.</span>
<span class="source-line-no">2409</span><span id="line-2409"> * 5. The in memory compact thread completes {@link CompactingMemStore#flattenOneSegment},</span>
<span class="source-line-no">2410</span><span id="line-2410"> * {@link CompactingMemStore#flattenOneSegment} change {@link CompactionPipeline#version} to</span>
<span class="source-line-no">2411</span><span id="line-2411"> * v+1.</span>
<span class="source-line-no">2412</span><span id="line-2412"> * 6. The snapshot thread continues {@link CompactingMemStore#swapPipelineWithNull}, and because</span>
<span class="source-line-no">2413</span><span id="line-2413"> * {@link CompactionPipeline#version} is v+1, {@link CompactingMemStore#swapPipelineWithNull}</span>
<span class="source-line-no">2414</span><span id="line-2414"> * failed and retry,{@link VersionedSegmentsList#version} returned from</span>
<span class="source-line-no">2415</span><span id="line-2415"> * {@link CompactingMemStore#getImmutableSegments} is v+1.</span>
<span class="source-line-no">2416</span><span id="line-2416"> * 7. The write thread continues writing to {@link CompactingMemStore} and</span>
<span class="source-line-no">2417</span><span id="line-2417"> * {@link CompactingMemStore} size exceeds {@link CompactingMemStore#getInmemoryFlushSize()},</span>
<span class="source-line-no">2418</span><span id="line-2418"> * {@link CompactingMemStore#flushInMemory(MutableSegment)} is called and a new</span>
<span class="source-line-no">2419</span><span id="line-2419"> * {@link ImmutableSegment} is added to the head of {@link CompactingMemStore#pipeline},</span>
<span class="source-line-no">2420</span><span id="line-2420"> * {@link CompactionPipeline#version} is still v+1.</span>
<span class="source-line-no">2421</span><span id="line-2421"> * 8. The snapshot thread continues {@link CompactingMemStore#swapPipelineWithNull}, and because</span>
<span class="source-line-no">2422</span><span id="line-2422"> * {@link CompactionPipeline#version} is still v+1,</span>
<span class="source-line-no">2423</span><span id="line-2423"> * {@link CompactingMemStore#swapPipelineWithNull} succeeds.The new {@link ImmutableSegment}</span>
<span class="source-line-no">2424</span><span id="line-2424"> * remained at the head of {@link CompactingMemStore#pipeline},the old is removed by</span>
<span class="source-line-no">2425</span><span id="line-2425"> * {@link CompactingMemStore#swapPipelineWithNull}.</span>
<span class="source-line-no">2426</span><span id="line-2426"> * &lt;/pre&gt;</span>
<span class="source-line-no">2427</span><span id="line-2427"> */</span>
<span class="source-line-no">2428</span><span id="line-2428"> @Test</span>
<span class="source-line-no">2429</span><span id="line-2429"> public void testFlattenSnapshotWriteCompactingMemeStoreConcurrently() throws Exception {</span>
<span class="source-line-no">2430</span><span id="line-2430"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">2431</span><span id="line-2431"></span>
<span class="source-line-no">2432</span><span id="line-2432"> byte[] smallValue = new byte[3];</span>
<span class="source-line-no">2433</span><span id="line-2433"> byte[] largeValue = new byte[9];</span>
<span class="source-line-no">2434</span><span id="line-2434"> final long timestamp = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">2435</span><span id="line-2435"> final long seqId = 100;</span>
<span class="source-line-no">2436</span><span id="line-2436"> final ExtendedCell smallCell = createCell(qf1, timestamp, seqId, smallValue);</span>
<span class="source-line-no">2437</span><span id="line-2437"> final ExtendedCell largeCell = createCell(qf2, timestamp, seqId, largeValue);</span>
<span class="source-line-no">2438</span><span id="line-2438"> int smallCellByteSize = MutableSegment.getCellLength(smallCell);</span>
<span class="source-line-no">2439</span><span id="line-2439"> int largeCellByteSize = MutableSegment.getCellLength(largeCell);</span>
<span class="source-line-no">2440</span><span id="line-2440"> int firstWriteCellByteSize = (smallCellByteSize + largeCellByteSize);</span>
<span class="source-line-no">2441</span><span id="line-2441"> int flushByteSize = firstWriteCellByteSize - 2;</span>
<span class="source-line-no">2442</span><span id="line-2442"></span>
<span class="source-line-no">2443</span><span id="line-2443"> // set CompactingMemStore.inmemoryFlushSize to flushByteSize.</span>
<span class="source-line-no">2444</span><span id="line-2444"> conf.set(HStore.MEMSTORE_CLASS_NAME, MyCompactingMemStore5.class.getName());</span>
<span class="source-line-no">2445</span><span id="line-2445"> conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.005);</span>
<span class="source-line-no">2446</span><span id="line-2446"> conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushByteSize * 200));</span>
<span class="source-line-no">2447</span><span id="line-2447"></span>
<span class="source-line-no">2448</span><span id="line-2448"> init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)</span>
<span class="source-line-no">2449</span><span id="line-2449"> .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());</span>
<span class="source-line-no">2450</span><span id="line-2450"></span>
<span class="source-line-no">2451</span><span id="line-2451"> final MyCompactingMemStore5 myCompactingMemStore = ((MyCompactingMemStore5) store.memstore);</span>
<span class="source-line-no">2452</span><span id="line-2452"> assertTrue((int) (myCompactingMemStore.getInmemoryFlushSize()) == flushByteSize);</span>
<span class="source-line-no">2453</span><span id="line-2453"></span>
<span class="source-line-no">2454</span><span id="line-2454"> store.add(smallCell, new NonThreadSafeMemStoreSizing());</span>
<span class="source-line-no">2455</span><span id="line-2455"> store.add(largeCell, new NonThreadSafeMemStoreSizing());</span>
<span class="source-line-no">2456</span><span id="line-2456"></span>
<span class="source-line-no">2457</span><span id="line-2457"> final AtomicReference&lt;Throwable&gt; exceptionRef = new AtomicReference&lt;Throwable&gt;();</span>
<span class="source-line-no">2458</span><span id="line-2458"> final ExtendedCell writeAgainCell1 = createCell(qf3, timestamp, seqId + 1, largeValue);</span>
<span class="source-line-no">2459</span><span id="line-2459"> final ExtendedCell writeAgainCell2 = createCell(qf4, timestamp, seqId + 1, largeValue);</span>
<span class="source-line-no">2460</span><span id="line-2460"> final int writeAgainCellByteSize =</span>
<span class="source-line-no">2461</span><span id="line-2461"> MutableSegment.getCellLength(writeAgainCell1) + MutableSegment.getCellLength(writeAgainCell2);</span>
<span class="source-line-no">2462</span><span id="line-2462"> final Thread writeAgainThread = new Thread(() -&gt; {</span>
<span class="source-line-no">2463</span><span id="line-2463"> try {</span>
<span class="source-line-no">2464</span><span id="line-2464"> myCompactingMemStore.writeMemStoreAgainStartCyclicBarrier.await();</span>
<span class="source-line-no">2465</span><span id="line-2465"></span>
<span class="source-line-no">2466</span><span id="line-2466"> store.add(writeAgainCell1, new NonThreadSafeMemStoreSizing());</span>
<span class="source-line-no">2467</span><span id="line-2467"> store.add(writeAgainCell2, new NonThreadSafeMemStoreSizing());</span>
<span class="source-line-no">2468</span><span id="line-2468"></span>
<span class="source-line-no">2469</span><span id="line-2469"> myCompactingMemStore.writeMemStoreAgainEndCyclicBarrier.await();</span>
<span class="source-line-no">2470</span><span id="line-2470"> } catch (Throwable exception) {</span>
<span class="source-line-no">2471</span><span id="line-2471"> exceptionRef.set(exception);</span>
<span class="source-line-no">2472</span><span id="line-2472"> }</span>
<span class="source-line-no">2473</span><span id="line-2473"> });</span>
<span class="source-line-no">2474</span><span id="line-2474"> writeAgainThread.setName(MyCompactingMemStore5.WRITE_AGAIN_THREAD_NAME);</span>
<span class="source-line-no">2475</span><span id="line-2475"> writeAgainThread.start();</span>
<span class="source-line-no">2476</span><span id="line-2476"></span>
<span class="source-line-no">2477</span><span id="line-2477"> String oldThreadName = Thread.currentThread().getName();</span>
<span class="source-line-no">2478</span><span id="line-2478"> try {</span>
<span class="source-line-no">2479</span><span id="line-2479"> Thread.currentThread().setName(MyCompactingMemStore5.TAKE_SNAPSHOT_THREAD_NAME);</span>
<span class="source-line-no">2480</span><span id="line-2480"> /**</span>
<span class="source-line-no">2481</span><span id="line-2481"> * {@link CompactingMemStore#snapshot} must wait the in memory compact thread enters</span>
<span class="source-line-no">2482</span><span id="line-2482"> * {@link CompactingMemStore#flattenOneSegment},because {@link CompactingMemStore#snapshot}</span>
<span class="source-line-no">2483</span><span id="line-2483"> * would invoke {@link CompactingMemStore#stopCompaction}.</span>
<span class="source-line-no">2484</span><span id="line-2484"> */</span>
<span class="source-line-no">2485</span><span id="line-2485"> myCompactingMemStore.snapShotStartCyclicCyclicBarrier.await();</span>
<span class="source-line-no">2486</span><span id="line-2486"> MemStoreSnapshot memStoreSnapshot = myCompactingMemStore.snapshot();</span>
<span class="source-line-no">2487</span><span id="line-2487"> myCompactingMemStore.inMemoryCompactionEndCyclicBarrier.await();</span>
<span class="source-line-no">2488</span><span id="line-2488"> writeAgainThread.join();</span>
<span class="source-line-no">2489</span><span id="line-2489"></span>
<span class="source-line-no">2490</span><span id="line-2490"> assertTrue(memStoreSnapshot.getCellsCount() == 2);</span>
<span class="source-line-no">2491</span><span id="line-2491"> assertTrue(((int) (memStoreSnapshot.getDataSize())) == firstWriteCellByteSize);</span>
<span class="source-line-no">2492</span><span id="line-2492"> VersionedSegmentsList segments = myCompactingMemStore.getImmutableSegments();</span>
<span class="source-line-no">2493</span><span id="line-2493"> assertTrue(segments.getNumOfSegments() == 1);</span>
<span class="source-line-no">2494</span><span id="line-2494"> assertTrue(</span>
<span class="source-line-no">2495</span><span id="line-2495"> ((int) (segments.getStoreSegments().get(0).getDataSize())) == writeAgainCellByteSize);</span>
<span class="source-line-no">2496</span><span id="line-2496"> assertTrue(segments.getNumOfCells() == 2);</span>
<span class="source-line-no">2497</span><span id="line-2497"> assertTrue(myCompactingMemStore.setInMemoryCompactionFlagCounter.get() == 2);</span>
<span class="source-line-no">2498</span><span id="line-2498"> assertTrue(exceptionRef.get() == null);</span>
<span class="source-line-no">2499</span><span id="line-2499"> assertTrue(myCompactingMemStore.swapPipelineWithNullCounter.get() == 2);</span>
<span class="source-line-no">2500</span><span id="line-2500"> } finally {</span>
<span class="source-line-no">2501</span><span id="line-2501"> Thread.currentThread().setName(oldThreadName);</span>
<span class="source-line-no">2502</span><span id="line-2502"> }</span>
<span class="source-line-no">2503</span><span id="line-2503"> }</span>
<span class="source-line-no">2504</span><span id="line-2504"></span>
<span class="source-line-no">2505</span><span id="line-2505"> /**</span>
<span class="source-line-no">2506</span><span id="line-2506"> * &lt;pre&gt;</span>
<span class="source-line-no">2507</span><span id="line-2507"> * This test is for HBASE-26465,</span>
<span class="source-line-no">2508</span><span id="line-2508"> * test {@link DefaultMemStore#clearSnapshot} and {@link DefaultMemStore#getScanners} execute</span>
<span class="source-line-no">2509</span><span id="line-2509"> * concurrently. The threads sequence before HBASE-26465 is:</span>
<span class="source-line-no">2510</span><span id="line-2510"> * 1.The flush thread starts {@link DefaultMemStore} flushing after some cells have be added to</span>
<span class="source-line-no">2511</span><span id="line-2511"> * {@link DefaultMemStore}.</span>
<span class="source-line-no">2512</span><span id="line-2512"> * 2.The flush thread stopping before {@link DefaultMemStore#clearSnapshot} in</span>
<span class="source-line-no">2513</span><span id="line-2513"> * {@link HStore#updateStorefiles} after completed flushing memStore to hfile.</span>
<span class="source-line-no">2514</span><span id="line-2514"> * 3.The scan thread starts and stopping after {@link DefaultMemStore#getSnapshotSegments} in</span>
<span class="source-line-no">2515</span><span id="line-2515"> * {@link DefaultMemStore#getScanners},here the scan thread gets the</span>
<span class="source-line-no">2516</span><span id="line-2516"> * {@link DefaultMemStore#snapshot} which is created by the flush thread.</span>
<span class="source-line-no">2517</span><span id="line-2517"> * 4.The flush thread continues {@link DefaultMemStore#clearSnapshot} and close</span>
<span class="source-line-no">2518</span><span id="line-2518"> * {@link DefaultMemStore#snapshot},because the reference count of the corresponding</span>
<span class="source-line-no">2519</span><span id="line-2519"> * {@link MemStoreLABImpl} is 0, the {@link Chunk}s in corresponding {@link MemStoreLABImpl}</span>
<span class="source-line-no">2520</span><span id="line-2520"> * are recycled.</span>
<span class="source-line-no">2521</span><span id="line-2521"> * 5.The scan thread continues {@link DefaultMemStore#getScanners},and create a</span>
<span class="source-line-no">2522</span><span id="line-2522"> * {@link SegmentScanner} for this {@link DefaultMemStore#snapshot}, and increase the</span>
<span class="source-line-no">2523</span><span id="line-2523"> * reference count of the corresponding {@link MemStoreLABImpl}, but {@link Chunk}s in</span>
<span class="source-line-no">2524</span><span id="line-2524"> * corresponding {@link MemStoreLABImpl} are recycled by step 4, and these {@link Chunk}s may</span>
<span class="source-line-no">2525</span><span id="line-2525"> * be overwritten by other write threads,which may cause serious problem.</span>
<span class="source-line-no">2526</span><span id="line-2526"> * After HBASE-26465,{@link DefaultMemStore#getScanners} and</span>
<span class="source-line-no">2527</span><span id="line-2527"> * {@link DefaultMemStore#clearSnapshot} could not execute concurrently.</span>
<span class="source-line-no">2528</span><span id="line-2528"> * &lt;/pre&gt;</span>
<span class="source-line-no">2529</span><span id="line-2529"> */</span>
<span class="source-line-no">2530</span><span id="line-2530"> @Test</span>
<span class="source-line-no">2531</span><span id="line-2531"> public void testClearSnapshotGetScannerConcurrently() throws Exception {</span>
<span class="source-line-no">2532</span><span id="line-2532"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">2533</span><span id="line-2533"></span>
<span class="source-line-no">2534</span><span id="line-2534"> byte[] smallValue = new byte[3];</span>
<span class="source-line-no">2535</span><span id="line-2535"> byte[] largeValue = new byte[9];</span>
<span class="source-line-no">2536</span><span id="line-2536"> final long timestamp = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">2537</span><span id="line-2537"> final long seqId = 100;</span>
<span class="source-line-no">2538</span><span id="line-2538"> final ExtendedCell smallCell = createCell(qf1, timestamp, seqId, smallValue);</span>
<span class="source-line-no">2539</span><span id="line-2539"> final ExtendedCell largeCell = createCell(qf2, timestamp, seqId, largeValue);</span>
<span class="source-line-no">2540</span><span id="line-2540"> TreeSet&lt;byte[]&gt; quals = new TreeSet&lt;&gt;(Bytes.BYTES_COMPARATOR);</span>
<span class="source-line-no">2541</span><span id="line-2541"> quals.add(qf1);</span>
<span class="source-line-no">2542</span><span id="line-2542"> quals.add(qf2);</span>
<span class="source-line-no">2543</span><span id="line-2543"></span>
<span class="source-line-no">2544</span><span id="line-2544"> conf.set(HStore.MEMSTORE_CLASS_NAME, MyDefaultMemStore.class.getName());</span>
<span class="source-line-no">2545</span><span id="line-2545"> conf.setBoolean(WALFactory.WAL_ENABLED, false);</span>
<span class="source-line-no">2546</span><span id="line-2546"></span>
<span class="source-line-no">2547</span><span id="line-2547"> init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family).build());</span>
<span class="source-line-no">2548</span><span id="line-2548"> MyDefaultMemStore myDefaultMemStore = (MyDefaultMemStore) (store.memstore);</span>
<span class="source-line-no">2549</span><span id="line-2549"> myDefaultMemStore.store = store;</span>
<span class="source-line-no">2550</span><span id="line-2550"></span>
<span class="source-line-no">2551</span><span id="line-2551"> MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();</span>
<span class="source-line-no">2552</span><span id="line-2552"> store.add(smallCell, memStoreSizing);</span>
<span class="source-line-no">2553</span><span id="line-2553"> store.add(largeCell, memStoreSizing);</span>
<span class="source-line-no">2554</span><span id="line-2554"></span>
<span class="source-line-no">2555</span><span id="line-2555"> final AtomicReference&lt;Throwable&gt; exceptionRef = new AtomicReference&lt;Throwable&gt;();</span>
<span class="source-line-no">2556</span><span id="line-2556"> final Thread flushThread = new Thread(() -&gt; {</span>
<span class="source-line-no">2557</span><span id="line-2557"> try {</span>
<span class="source-line-no">2558</span><span id="line-2558"> flushStore(store, id++);</span>
<span class="source-line-no">2559</span><span id="line-2559"> } catch (Throwable exception) {</span>
<span class="source-line-no">2560</span><span id="line-2560"> exceptionRef.set(exception);</span>
<span class="source-line-no">2561</span><span id="line-2561"> }</span>
<span class="source-line-no">2562</span><span id="line-2562"> });</span>
<span class="source-line-no">2563</span><span id="line-2563"> flushThread.setName(MyDefaultMemStore.FLUSH_THREAD_NAME);</span>
<span class="source-line-no">2564</span><span id="line-2564"> flushThread.start();</span>
<span class="source-line-no">2565</span><span id="line-2565"></span>
<span class="source-line-no">2566</span><span id="line-2566"> String oldThreadName = Thread.currentThread().getName();</span>
<span class="source-line-no">2567</span><span id="line-2567"> StoreScanner storeScanner = null;</span>
<span class="source-line-no">2568</span><span id="line-2568"> try {</span>
<span class="source-line-no">2569</span><span id="line-2569"> Thread.currentThread().setName(MyDefaultMemStore.GET_SCANNER_THREAD_NAME);</span>
<span class="source-line-no">2570</span><span id="line-2570"></span>
<span class="source-line-no">2571</span><span id="line-2571"> /**</span>
<span class="source-line-no">2572</span><span id="line-2572"> * Wait flush thread stopping before {@link DefaultMemStore#doClearSnapshot}</span>
<span class="source-line-no">2573</span><span id="line-2573"> */</span>
<span class="source-line-no">2574</span><span id="line-2574"> myDefaultMemStore.getScannerCyclicBarrier.await();</span>
<span class="source-line-no">2575</span><span id="line-2575"></span>
<span class="source-line-no">2576</span><span id="line-2576"> storeScanner = (StoreScanner) store.getScanner(new Scan(new Get(row)), quals, seqId + 1);</span>
<span class="source-line-no">2577</span><span id="line-2577"> flushThread.join();</span>
<span class="source-line-no">2578</span><span id="line-2578"></span>
<span class="source-line-no">2579</span><span id="line-2579"> if (myDefaultMemStore.shouldWait) {</span>
<span class="source-line-no">2580</span><span id="line-2580"> SegmentScanner segmentScanner = getTypeKeyValueScanner(storeScanner, SegmentScanner.class);</span>
<span class="source-line-no">2581</span><span id="line-2581"> MemStoreLABImpl memStoreLAB = (MemStoreLABImpl) (segmentScanner.segment.getMemStoreLAB());</span>
<span class="source-line-no">2582</span><span id="line-2582"> assertTrue(memStoreLAB.isClosed());</span>
<span class="source-line-no">2583</span><span id="line-2583"> assertTrue(!memStoreLAB.chunks.isEmpty());</span>
<span class="source-line-no">2584</span><span id="line-2584"> assertTrue(!memStoreLAB.isReclaimed());</span>
<span class="source-line-no">2585</span><span id="line-2585"></span>
<span class="source-line-no">2586</span><span id="line-2586"> ExtendedCell cell1 = segmentScanner.next();</span>
<span class="source-line-no">2587</span><span id="line-2587"> PrivateCellUtil.equals(smallCell, cell1);</span>
<span class="source-line-no">2588</span><span id="line-2588"> ExtendedCell cell2 = segmentScanner.next();</span>
<span class="source-line-no">2589</span><span id="line-2589"> PrivateCellUtil.equals(largeCell, cell2);</span>
<span class="source-line-no">2590</span><span id="line-2590"> assertNull(segmentScanner.next());</span>
<span class="source-line-no">2591</span><span id="line-2591"> } else {</span>
<span class="source-line-no">2592</span><span id="line-2592"> List&lt;ExtendedCell&gt; results = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">2593</span><span id="line-2593"> storeScanner.next(results);</span>
<span class="source-line-no">2594</span><span id="line-2594"> assertEquals(2, results.size());</span>
<span class="source-line-no">2595</span><span id="line-2595"> PrivateCellUtil.equals(smallCell, results.get(0));</span>
<span class="source-line-no">2596</span><span id="line-2596"> PrivateCellUtil.equals(largeCell, results.get(1));</span>
<span class="source-line-no">2597</span><span id="line-2597"> }</span>
<span class="source-line-no">2598</span><span id="line-2598"> assertTrue(exceptionRef.get() == null);</span>
<span class="source-line-no">2599</span><span id="line-2599"> } finally {</span>
<span class="source-line-no">2600</span><span id="line-2600"> if (storeScanner != null) {</span>
<span class="source-line-no">2601</span><span id="line-2601"> storeScanner.close();</span>
<span class="source-line-no">2602</span><span id="line-2602"> }</span>
<span class="source-line-no">2603</span><span id="line-2603"> Thread.currentThread().setName(oldThreadName);</span>
<span class="source-line-no">2604</span><span id="line-2604"> }</span>
<span class="source-line-no">2605</span><span id="line-2605"> }</span>
<span class="source-line-no">2606</span><span id="line-2606"></span>
<span class="source-line-no">2607</span><span id="line-2607"> @SuppressWarnings("unchecked")</span>
<span class="source-line-no">2608</span><span id="line-2608"> private &lt;T&gt; T getTypeKeyValueScanner(StoreScanner storeScanner, Class&lt;T&gt; keyValueScannerClass) {</span>
<span class="source-line-no">2609</span><span id="line-2609"> List&lt;T&gt; resultScanners = new ArrayList&lt;T&gt;();</span>
<span class="source-line-no">2610</span><span id="line-2610"> for (KeyValueScanner keyValueScanner : storeScanner.currentScanners) {</span>
<span class="source-line-no">2611</span><span id="line-2611"> if (keyValueScannerClass.isInstance(keyValueScanner)) {</span>
<span class="source-line-no">2612</span><span id="line-2612"> resultScanners.add((T) keyValueScanner);</span>
<span class="source-line-no">2613</span><span id="line-2613"> }</span>
<span class="source-line-no">2614</span><span id="line-2614"> }</span>
<span class="source-line-no">2615</span><span id="line-2615"> assertTrue(resultScanners.size() == 1);</span>
<span class="source-line-no">2616</span><span id="line-2616"> return resultScanners.get(0);</span>
<span class="source-line-no">2617</span><span id="line-2617"> }</span>
<span class="source-line-no">2618</span><span id="line-2618"></span>
<span class="source-line-no">2619</span><span id="line-2619"> @Test</span>
<span class="source-line-no">2620</span><span id="line-2620"> public void testOnConfigurationChange() throws IOException {</span>
<span class="source-line-no">2621</span><span id="line-2621"> final int COMMON_MAX_FILES_TO_COMPACT = 10;</span>
<span class="source-line-no">2622</span><span id="line-2622"> final int NEW_COMMON_MAX_FILES_TO_COMPACT = 8;</span>
<span class="source-line-no">2623</span><span id="line-2623"> final int STORE_MAX_FILES_TO_COMPACT = 6;</span>
<span class="source-line-no">2624</span><span id="line-2624"></span>
<span class="source-line-no">2625</span><span id="line-2625"> // Build a table that its maxFileToCompact different from common configuration.</span>
<span class="source-line-no">2626</span><span id="line-2626"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">2627</span><span id="line-2627"> conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_KEY,</span>
<span class="source-line-no">2628</span><span id="line-2628"> COMMON_MAX_FILES_TO_COMPACT);</span>
<span class="source-line-no">2629</span><span id="line-2629"> conf.setBoolean(CACHE_DATA_ON_READ_KEY, false);</span>
<span class="source-line-no">2630</span><span id="line-2630"> conf.setBoolean(CACHE_BLOCKS_ON_WRITE_KEY, true);</span>
<span class="source-line-no">2631</span><span id="line-2631"> conf.setBoolean(EVICT_BLOCKS_ON_CLOSE_KEY, true);</span>
<span class="source-line-no">2632</span><span id="line-2632"> ColumnFamilyDescriptor hcd = ColumnFamilyDescriptorBuilder.newBuilder(family)</span>
<span class="source-line-no">2633</span><span id="line-2633"> .setConfiguration(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_KEY,</span>
<span class="source-line-no">2634</span><span id="line-2634"> String.valueOf(STORE_MAX_FILES_TO_COMPACT))</span>
<span class="source-line-no">2635</span><span id="line-2635"> .build();</span>
<span class="source-line-no">2636</span><span id="line-2636"> init(this.name.getMethodName(), conf, hcd);</span>
<span class="source-line-no">2637</span><span id="line-2637"></span>
<span class="source-line-no">2638</span><span id="line-2638"> // After updating common configuration, the conf in HStore itself must not be changed.</span>
<span class="source-line-no">2639</span><span id="line-2639"> conf.setInt(CompactionConfiguration.HBASE_HSTORE_COMPACTION_MAX_KEY,</span>
<span class="source-line-no">2640</span><span id="line-2640"> NEW_COMMON_MAX_FILES_TO_COMPACT);</span>
<span class="source-line-no">2641</span><span id="line-2641"> this.store.onConfigurationChange(conf);</span>
<span class="source-line-no">2642</span><span id="line-2642"></span>
<span class="source-line-no">2643</span><span id="line-2643"> assertEquals(STORE_MAX_FILES_TO_COMPACT,</span>
<span class="source-line-no">2644</span><span id="line-2644"> store.getStoreEngine().getCompactionPolicy().getConf().getMaxFilesToCompact());</span>
<span class="source-line-no">2645</span><span id="line-2645"></span>
<span class="source-line-no">2646</span><span id="line-2646"> assertEquals(conf.getBoolean(CACHE_DATA_ON_READ_KEY, DEFAULT_CACHE_DATA_ON_READ), false);</span>
<span class="source-line-no">2647</span><span id="line-2647"> assertEquals(conf.getBoolean(CACHE_BLOCKS_ON_WRITE_KEY, DEFAULT_CACHE_DATA_ON_WRITE), true);</span>
<span class="source-line-no">2648</span><span id="line-2648"> assertEquals(conf.getBoolean(EVICT_BLOCKS_ON_CLOSE_KEY, DEFAULT_EVICT_ON_CLOSE), true);</span>
<span class="source-line-no">2649</span><span id="line-2649"></span>
<span class="source-line-no">2650</span><span id="line-2650"> // reset to default values</span>
<span class="source-line-no">2651</span><span id="line-2651"> conf.getBoolean(CACHE_DATA_ON_READ_KEY, DEFAULT_CACHE_DATA_ON_READ);</span>
<span class="source-line-no">2652</span><span id="line-2652"> conf.getBoolean(CACHE_BLOCKS_ON_WRITE_KEY, DEFAULT_CACHE_DATA_ON_WRITE);</span>
<span class="source-line-no">2653</span><span id="line-2653"> conf.getBoolean(EVICT_BLOCKS_ON_CLOSE_KEY, DEFAULT_EVICT_ON_CLOSE);</span>
<span class="source-line-no">2654</span><span id="line-2654"> this.store.onConfigurationChange(conf);</span>
<span class="source-line-no">2655</span><span id="line-2655"> }</span>
<span class="source-line-no">2656</span><span id="line-2656"></span>
<span class="source-line-no">2657</span><span id="line-2657"> /**</span>
<span class="source-line-no">2658</span><span id="line-2658"> * This test is for HBASE-26476</span>
<span class="source-line-no">2659</span><span id="line-2659"> */</span>
<span class="source-line-no">2660</span><span id="line-2660"> @Test</span>
<span class="source-line-no">2661</span><span id="line-2661"> public void testExtendsDefaultMemStore() throws Exception {</span>
<span class="source-line-no">2662</span><span id="line-2662"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">2663</span><span id="line-2663"> conf.setBoolean(WALFactory.WAL_ENABLED, false);</span>
<span class="source-line-no">2664</span><span id="line-2664"></span>
<span class="source-line-no">2665</span><span id="line-2665"> init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family).build());</span>
<span class="source-line-no">2666</span><span id="line-2666"> assertTrue(this.store.memstore.getClass() == DefaultMemStore.class);</span>
<span class="source-line-no">2667</span><span id="line-2667"> tearDown();</span>
<span class="source-line-no">2668</span><span id="line-2668"></span>
<span class="source-line-no">2669</span><span id="line-2669"> conf.set(HStore.MEMSTORE_CLASS_NAME, CustomDefaultMemStore.class.getName());</span>
<span class="source-line-no">2670</span><span id="line-2670"> init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family).build());</span>
<span class="source-line-no">2671</span><span id="line-2671"> assertTrue(this.store.memstore.getClass() == CustomDefaultMemStore.class);</span>
<span class="source-line-no">2672</span><span id="line-2672"> }</span>
<span class="source-line-no">2673</span><span id="line-2673"></span>
<span class="source-line-no">2674</span><span id="line-2674"> static class CustomDefaultMemStore extends DefaultMemStore {</span>
<span class="source-line-no">2675</span><span id="line-2675"></span>
<span class="source-line-no">2676</span><span id="line-2676"> public CustomDefaultMemStore(Configuration conf, CellComparator c,</span>
<span class="source-line-no">2677</span><span id="line-2677"> RegionServicesForStores regionServices) {</span>
<span class="source-line-no">2678</span><span id="line-2678"> super(conf, c, regionServices);</span>
<span class="source-line-no">2679</span><span id="line-2679"> }</span>
<span class="source-line-no">2680</span><span id="line-2680"></span>
<span class="source-line-no">2681</span><span id="line-2681"> }</span>
<span class="source-line-no">2682</span><span id="line-2682"></span>
<span class="source-line-no">2683</span><span id="line-2683"> /**</span>
<span class="source-line-no">2684</span><span id="line-2684"> * This test is for HBASE-26488</span>
<span class="source-line-no">2685</span><span id="line-2685"> */</span>
<span class="source-line-no">2686</span><span id="line-2686"> @Test</span>
<span class="source-line-no">2687</span><span id="line-2687"> public void testMemoryLeakWhenFlushMemStoreRetrying() throws Exception {</span>
<span class="source-line-no">2688</span><span id="line-2688"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">2689</span><span id="line-2689"></span>
<span class="source-line-no">2690</span><span id="line-2690"> byte[] smallValue = new byte[3];</span>
<span class="source-line-no">2691</span><span id="line-2691"> byte[] largeValue = new byte[9];</span>
<span class="source-line-no">2692</span><span id="line-2692"> final long timestamp = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">2693</span><span id="line-2693"> final long seqId = 100;</span>
<span class="source-line-no">2694</span><span id="line-2694"> final ExtendedCell smallCell = createCell(qf1, timestamp, seqId, smallValue);</span>
<span class="source-line-no">2695</span><span id="line-2695"> final ExtendedCell largeCell = createCell(qf2, timestamp, seqId, largeValue);</span>
<span class="source-line-no">2696</span><span id="line-2696"> TreeSet&lt;byte[]&gt; quals = new TreeSet&lt;&gt;(Bytes.BYTES_COMPARATOR);</span>
<span class="source-line-no">2697</span><span id="line-2697"> quals.add(qf1);</span>
<span class="source-line-no">2698</span><span id="line-2698"> quals.add(qf2);</span>
<span class="source-line-no">2699</span><span id="line-2699"></span>
<span class="source-line-no">2700</span><span id="line-2700"> conf.set(HStore.MEMSTORE_CLASS_NAME, MyDefaultMemStore1.class.getName());</span>
<span class="source-line-no">2701</span><span id="line-2701"> conf.setBoolean(WALFactory.WAL_ENABLED, false);</span>
<span class="source-line-no">2702</span><span id="line-2702"> conf.set(DefaultStoreEngine.DEFAULT_STORE_FLUSHER_CLASS_KEY,</span>
<span class="source-line-no">2703</span><span id="line-2703"> MyDefaultStoreFlusher.class.getName());</span>
<span class="source-line-no">2704</span><span id="line-2704"></span>
<span class="source-line-no">2705</span><span id="line-2705"> init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family).build());</span>
<span class="source-line-no">2706</span><span id="line-2706"> MyDefaultMemStore1 myDefaultMemStore = (MyDefaultMemStore1) (store.memstore);</span>
<span class="source-line-no">2707</span><span id="line-2707"> assertTrue((store.storeEngine.getStoreFlusher()) instanceof MyDefaultStoreFlusher);</span>
<span class="source-line-no">2708</span><span id="line-2708"></span>
<span class="source-line-no">2709</span><span id="line-2709"> MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();</span>
<span class="source-line-no">2710</span><span id="line-2710"> store.add(smallCell, memStoreSizing);</span>
<span class="source-line-no">2711</span><span id="line-2711"> store.add(largeCell, memStoreSizing);</span>
<span class="source-line-no">2712</span><span id="line-2712"> flushStore(store, id++);</span>
<span class="source-line-no">2713</span><span id="line-2713"></span>
<span class="source-line-no">2714</span><span id="line-2714"> MemStoreLABImpl memStoreLAB =</span>
<span class="source-line-no">2715</span><span id="line-2715"> (MemStoreLABImpl) (myDefaultMemStore.snapshotImmutableSegment.getMemStoreLAB());</span>
<span class="source-line-no">2716</span><span id="line-2716"> assertTrue(memStoreLAB.isClosed());</span>
<span class="source-line-no">2717</span><span id="line-2717"> assertTrue(memStoreLAB.getRefCntValue() == 0);</span>
<span class="source-line-no">2718</span><span id="line-2718"> assertTrue(memStoreLAB.isReclaimed());</span>
<span class="source-line-no">2719</span><span id="line-2719"> assertTrue(memStoreLAB.chunks.isEmpty());</span>
<span class="source-line-no">2720</span><span id="line-2720"> StoreScanner storeScanner = null;</span>
<span class="source-line-no">2721</span><span id="line-2721"> try {</span>
<span class="source-line-no">2722</span><span id="line-2722"> storeScanner = (StoreScanner) store.getScanner(new Scan(new Get(row)), quals, seqId + 1);</span>
<span class="source-line-no">2723</span><span id="line-2723"> assertTrue(store.storeEngine.getStoreFileManager().getStorefileCount() == 1);</span>
<span class="source-line-no">2724</span><span id="line-2724"> assertTrue(store.memstore.size().getCellsCount() == 0);</span>
<span class="source-line-no">2725</span><span id="line-2725"> assertTrue(store.memstore.getSnapshotSize().getCellsCount() == 0);</span>
<span class="source-line-no">2726</span><span id="line-2726"> assertTrue(storeScanner.currentScanners.size() == 1);</span>
<span class="source-line-no">2727</span><span id="line-2727"> assertTrue(storeScanner.currentScanners.get(0) instanceof StoreFileScanner);</span>
<span class="source-line-no">2728</span><span id="line-2728"></span>
<span class="source-line-no">2729</span><span id="line-2729"> List&lt;ExtendedCell&gt; results = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">2730</span><span id="line-2730"> storeScanner.next(results);</span>
<span class="source-line-no">2731</span><span id="line-2731"> assertEquals(2, results.size());</span>
<span class="source-line-no">2732</span><span id="line-2732"> PrivateCellUtil.equals(smallCell, results.get(0));</span>
<span class="source-line-no">2733</span><span id="line-2733"> PrivateCellUtil.equals(largeCell, results.get(1));</span>
<span class="source-line-no">2734</span><span id="line-2734"> } finally {</span>
<span class="source-line-no">2735</span><span id="line-2735"> if (storeScanner != null) {</span>
<span class="source-line-no">2736</span><span id="line-2736"> storeScanner.close();</span>
<span class="source-line-no">2737</span><span id="line-2737"> }</span>
<span class="source-line-no">2738</span><span id="line-2738"> }</span>
<span class="source-line-no">2739</span><span id="line-2739"> }</span>
<span class="source-line-no">2740</span><span id="line-2740"></span>
<span class="source-line-no">2741</span><span id="line-2741"> static class MyDefaultMemStore1 extends DefaultMemStore {</span>
<span class="source-line-no">2742</span><span id="line-2742"></span>
<span class="source-line-no">2743</span><span id="line-2743"> private ImmutableSegment snapshotImmutableSegment;</span>
<span class="source-line-no">2744</span><span id="line-2744"></span>
<span class="source-line-no">2745</span><span id="line-2745"> public MyDefaultMemStore1(Configuration conf, CellComparator c,</span>
<span class="source-line-no">2746</span><span id="line-2746"> RegionServicesForStores regionServices) {</span>
<span class="source-line-no">2747</span><span id="line-2747"> super(conf, c, regionServices);</span>
<span class="source-line-no">2748</span><span id="line-2748"> }</span>
<span class="source-line-no">2749</span><span id="line-2749"></span>
<span class="source-line-no">2750</span><span id="line-2750"> @Override</span>
<span class="source-line-no">2751</span><span id="line-2751"> public MemStoreSnapshot snapshot() {</span>
<span class="source-line-no">2752</span><span id="line-2752"> MemStoreSnapshot result = super.snapshot();</span>
<span class="source-line-no">2753</span><span id="line-2753"> this.snapshotImmutableSegment = snapshot;</span>
<span class="source-line-no">2754</span><span id="line-2754"> return result;</span>
<span class="source-line-no">2755</span><span id="line-2755"> }</span>
<span class="source-line-no">2756</span><span id="line-2756"></span>
<span class="source-line-no">2757</span><span id="line-2757"> }</span>
<span class="source-line-no">2758</span><span id="line-2758"></span>
<span class="source-line-no">2759</span><span id="line-2759"> public static class MyDefaultStoreFlusher extends DefaultStoreFlusher {</span>
<span class="source-line-no">2760</span><span id="line-2760"> private static final AtomicInteger failCounter = new AtomicInteger(1);</span>
<span class="source-line-no">2761</span><span id="line-2761"> private static final AtomicInteger counter = new AtomicInteger(0);</span>
<span class="source-line-no">2762</span><span id="line-2762"></span>
<span class="source-line-no">2763</span><span id="line-2763"> public MyDefaultStoreFlusher(Configuration conf, HStore store) {</span>
<span class="source-line-no">2764</span><span id="line-2764"> super(conf, store);</span>
<span class="source-line-no">2765</span><span id="line-2765"> }</span>
<span class="source-line-no">2766</span><span id="line-2766"></span>
<span class="source-line-no">2767</span><span id="line-2767"> @Override</span>
<span class="source-line-no">2768</span><span id="line-2768"> public List&lt;Path&gt; flushSnapshot(MemStoreSnapshot snapshot, long cacheFlushId,</span>
<span class="source-line-no">2769</span><span id="line-2769"> MonitoredTask status, ThroughputController throughputController,</span>
<span class="source-line-no">2770</span><span id="line-2770"> FlushLifeCycleTracker tracker, Consumer&lt;Path&gt; writerCreationTracker) throws IOException {</span>
<span class="source-line-no">2771</span><span id="line-2771"> counter.incrementAndGet();</span>
<span class="source-line-no">2772</span><span id="line-2772"> return super.flushSnapshot(snapshot, cacheFlushId, status, throughputController, tracker,</span>
<span class="source-line-no">2773</span><span id="line-2773"> writerCreationTracker);</span>
<span class="source-line-no">2774</span><span id="line-2774"> }</span>
<span class="source-line-no">2775</span><span id="line-2775"></span>
<span class="source-line-no">2776</span><span id="line-2776"> @Override</span>
<span class="source-line-no">2777</span><span id="line-2777"> protected void performFlush(InternalScanner scanner, final CellSink sink,</span>
<span class="source-line-no">2778</span><span id="line-2778"> ThroughputController throughputController) throws IOException {</span>
<span class="source-line-no">2779</span><span id="line-2779"></span>
<span class="source-line-no">2780</span><span id="line-2780"> final int currentCount = counter.get();</span>
<span class="source-line-no">2781</span><span id="line-2781"> CellSink newCellSink = (cell) -&gt; {</span>
<span class="source-line-no">2782</span><span id="line-2782"> if (currentCount &lt;= failCounter.get()) {</span>
<span class="source-line-no">2783</span><span id="line-2783"> throw new IOException("Simulated exception by tests");</span>
<span class="source-line-no">2784</span><span id="line-2784"> }</span>
<span class="source-line-no">2785</span><span id="line-2785"> sink.append(cell);</span>
<span class="source-line-no">2786</span><span id="line-2786"> };</span>
<span class="source-line-no">2787</span><span id="line-2787"> super.performFlush(scanner, newCellSink, throughputController);</span>
<span class="source-line-no">2788</span><span id="line-2788"> }</span>
<span class="source-line-no">2789</span><span id="line-2789"> }</span>
<span class="source-line-no">2790</span><span id="line-2790"></span>
<span class="source-line-no">2791</span><span id="line-2791"> /**</span>
<span class="source-line-no">2792</span><span id="line-2792"> * This test is for HBASE-26494, test the {@link RefCnt} behaviors in {@link ImmutableMemStoreLAB}</span>
<span class="source-line-no">2793</span><span id="line-2793"> */</span>
<span class="source-line-no">2794</span><span id="line-2794"> @Test</span>
<span class="source-line-no">2795</span><span id="line-2795"> public void testImmutableMemStoreLABRefCnt() throws Exception {</span>
<span class="source-line-no">2796</span><span id="line-2796"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">2797</span><span id="line-2797"></span>
<span class="source-line-no">2798</span><span id="line-2798"> byte[] smallValue = new byte[3];</span>
<span class="source-line-no">2799</span><span id="line-2799"> byte[] largeValue = new byte[9];</span>
<span class="source-line-no">2800</span><span id="line-2800"> final long timestamp = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">2801</span><span id="line-2801"> final long seqId = 100;</span>
<span class="source-line-no">2802</span><span id="line-2802"> final ExtendedCell smallCell1 = createCell(qf1, timestamp, seqId, smallValue);</span>
<span class="source-line-no">2803</span><span id="line-2803"> final ExtendedCell largeCell1 = createCell(qf2, timestamp, seqId, largeValue);</span>
<span class="source-line-no">2804</span><span id="line-2804"> final ExtendedCell smallCell2 = createCell(qf3, timestamp, seqId + 1, smallValue);</span>
<span class="source-line-no">2805</span><span id="line-2805"> final ExtendedCell largeCell2 = createCell(qf4, timestamp, seqId + 1, largeValue);</span>
<span class="source-line-no">2806</span><span id="line-2806"> final ExtendedCell smallCell3 = createCell(qf5, timestamp, seqId + 2, smallValue);</span>
<span class="source-line-no">2807</span><span id="line-2807"> final ExtendedCell largeCell3 = createCell(qf6, timestamp, seqId + 2, largeValue);</span>
<span class="source-line-no">2808</span><span id="line-2808"></span>
<span class="source-line-no">2809</span><span id="line-2809"> int smallCellByteSize = MutableSegment.getCellLength(smallCell1);</span>
<span class="source-line-no">2810</span><span id="line-2810"> int largeCellByteSize = MutableSegment.getCellLength(largeCell1);</span>
<span class="source-line-no">2811</span><span id="line-2811"> int firstWriteCellByteSize = (smallCellByteSize + largeCellByteSize);</span>
<span class="source-line-no">2812</span><span id="line-2812"> int flushByteSize = firstWriteCellByteSize - 2;</span>
<span class="source-line-no">2813</span><span id="line-2813"></span>
<span class="source-line-no">2814</span><span id="line-2814"> // set CompactingMemStore.inmemoryFlushSize to flushByteSize.</span>
<span class="source-line-no">2815</span><span id="line-2815"> conf.set(HStore.MEMSTORE_CLASS_NAME, CompactingMemStore.class.getName());</span>
<span class="source-line-no">2816</span><span id="line-2816"> conf.setDouble(CompactingMemStore.IN_MEMORY_FLUSH_THRESHOLD_FACTOR_KEY, 0.005);</span>
<span class="source-line-no">2817</span><span id="line-2817"> conf.set(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, String.valueOf(flushByteSize * 200));</span>
<span class="source-line-no">2818</span><span id="line-2818"> conf.setBoolean(WALFactory.WAL_ENABLED, false);</span>
<span class="source-line-no">2819</span><span id="line-2819"></span>
<span class="source-line-no">2820</span><span id="line-2820"> init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family)</span>
<span class="source-line-no">2821</span><span id="line-2821"> .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build());</span>
<span class="source-line-no">2822</span><span id="line-2822"></span>
<span class="source-line-no">2823</span><span id="line-2823"> final CompactingMemStore myCompactingMemStore = ((CompactingMemStore) store.memstore);</span>
<span class="source-line-no">2824</span><span id="line-2824"> assertTrue((int) (myCompactingMemStore.getInmemoryFlushSize()) == flushByteSize);</span>
<span class="source-line-no">2825</span><span id="line-2825"> myCompactingMemStore.allowCompaction.set(false);</span>
<span class="source-line-no">2826</span><span id="line-2826"></span>
<span class="source-line-no">2827</span><span id="line-2827"> NonThreadSafeMemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();</span>
<span class="source-line-no">2828</span><span id="line-2828"> store.add(smallCell1, memStoreSizing);</span>
<span class="source-line-no">2829</span><span id="line-2829"> store.add(largeCell1, memStoreSizing);</span>
<span class="source-line-no">2830</span><span id="line-2830"> store.add(smallCell2, memStoreSizing);</span>
<span class="source-line-no">2831</span><span id="line-2831"> store.add(largeCell2, memStoreSizing);</span>
<span class="source-line-no">2832</span><span id="line-2832"> store.add(smallCell3, memStoreSizing);</span>
<span class="source-line-no">2833</span><span id="line-2833"> store.add(largeCell3, memStoreSizing);</span>
<span class="source-line-no">2834</span><span id="line-2834"> VersionedSegmentsList versionedSegmentsList = myCompactingMemStore.getImmutableSegments();</span>
<span class="source-line-no">2835</span><span id="line-2835"> assertTrue(versionedSegmentsList.getNumOfSegments() == 3);</span>
<span class="source-line-no">2836</span><span id="line-2836"> List&lt;ImmutableSegment&gt; segments = versionedSegmentsList.getStoreSegments();</span>
<span class="source-line-no">2837</span><span id="line-2837"> List&lt;MemStoreLABImpl&gt; memStoreLABs = new ArrayList&lt;MemStoreLABImpl&gt;(segments.size());</span>
<span class="source-line-no">2838</span><span id="line-2838"> for (ImmutableSegment segment : segments) {</span>
<span class="source-line-no">2839</span><span id="line-2839"> memStoreLABs.add((MemStoreLABImpl) segment.getMemStoreLAB());</span>
<span class="source-line-no">2840</span><span id="line-2840"> }</span>
<span class="source-line-no">2841</span><span id="line-2841"> List&lt;KeyValueScanner&gt; scanners1 = myCompactingMemStore.getScanners(Long.MAX_VALUE);</span>
<span class="source-line-no">2842</span><span id="line-2842"> for (MemStoreLABImpl memStoreLAB : memStoreLABs) {</span>
<span class="source-line-no">2843</span><span id="line-2843"> assertTrue(memStoreLAB.getRefCntValue() == 2);</span>
<span class="source-line-no">2844</span><span id="line-2844"> }</span>
<span class="source-line-no">2845</span><span id="line-2845"></span>
<span class="source-line-no">2846</span><span id="line-2846"> myCompactingMemStore.allowCompaction.set(true);</span>
<span class="source-line-no">2847</span><span id="line-2847"> myCompactingMemStore.flushInMemory();</span>
<span class="source-line-no">2848</span><span id="line-2848"></span>
<span class="source-line-no">2849</span><span id="line-2849"> versionedSegmentsList = myCompactingMemStore.getImmutableSegments();</span>
<span class="source-line-no">2850</span><span id="line-2850"> assertTrue(versionedSegmentsList.getNumOfSegments() == 1);</span>
<span class="source-line-no">2851</span><span id="line-2851"> ImmutableMemStoreLAB immutableMemStoreLAB =</span>
<span class="source-line-no">2852</span><span id="line-2852"> (ImmutableMemStoreLAB) (versionedSegmentsList.getStoreSegments().get(0).getMemStoreLAB());</span>
<span class="source-line-no">2853</span><span id="line-2853"> for (MemStoreLABImpl memStoreLAB : memStoreLABs) {</span>
<span class="source-line-no">2854</span><span id="line-2854"> assertTrue(memStoreLAB.getRefCntValue() == 2);</span>
<span class="source-line-no">2855</span><span id="line-2855"> }</span>
<span class="source-line-no">2856</span><span id="line-2856"></span>
<span class="source-line-no">2857</span><span id="line-2857"> List&lt;KeyValueScanner&gt; scanners2 = myCompactingMemStore.getScanners(Long.MAX_VALUE);</span>
<span class="source-line-no">2858</span><span id="line-2858"> for (MemStoreLABImpl memStoreLAB : memStoreLABs) {</span>
<span class="source-line-no">2859</span><span id="line-2859"> assertTrue(memStoreLAB.getRefCntValue() == 2);</span>
<span class="source-line-no">2860</span><span id="line-2860"> }</span>
<span class="source-line-no">2861</span><span id="line-2861"> assertTrue(immutableMemStoreLAB.getRefCntValue() == 2);</span>
<span class="source-line-no">2862</span><span id="line-2862"> for (KeyValueScanner scanner : scanners1) {</span>
<span class="source-line-no">2863</span><span id="line-2863"> scanner.close();</span>
<span class="source-line-no">2864</span><span id="line-2864"> }</span>
<span class="source-line-no">2865</span><span id="line-2865"> for (MemStoreLABImpl memStoreLAB : memStoreLABs) {</span>
<span class="source-line-no">2866</span><span id="line-2866"> assertTrue(memStoreLAB.getRefCntValue() == 1);</span>
<span class="source-line-no">2867</span><span id="line-2867"> }</span>
<span class="source-line-no">2868</span><span id="line-2868"> for (KeyValueScanner scanner : scanners2) {</span>
<span class="source-line-no">2869</span><span id="line-2869"> scanner.close();</span>
<span class="source-line-no">2870</span><span id="line-2870"> }</span>
<span class="source-line-no">2871</span><span id="line-2871"> for (MemStoreLABImpl memStoreLAB : memStoreLABs) {</span>
<span class="source-line-no">2872</span><span id="line-2872"> assertTrue(memStoreLAB.getRefCntValue() == 1);</span>
<span class="source-line-no">2873</span><span id="line-2873"> }</span>
<span class="source-line-no">2874</span><span id="line-2874"> assertTrue(immutableMemStoreLAB.getRefCntValue() == 1);</span>
<span class="source-line-no">2875</span><span id="line-2875"> flushStore(store, id++);</span>
<span class="source-line-no">2876</span><span id="line-2876"> for (MemStoreLABImpl memStoreLAB : memStoreLABs) {</span>
<span class="source-line-no">2877</span><span id="line-2877"> assertTrue(memStoreLAB.getRefCntValue() == 0);</span>
<span class="source-line-no">2878</span><span id="line-2878"> }</span>
<span class="source-line-no">2879</span><span id="line-2879"> assertTrue(immutableMemStoreLAB.getRefCntValue() == 0);</span>
<span class="source-line-no">2880</span><span id="line-2880"> assertTrue(immutableMemStoreLAB.isClosed());</span>
<span class="source-line-no">2881</span><span id="line-2881"> for (MemStoreLABImpl memStoreLAB : memStoreLABs) {</span>
<span class="source-line-no">2882</span><span id="line-2882"> assertTrue(memStoreLAB.isClosed());</span>
<span class="source-line-no">2883</span><span id="line-2883"> assertTrue(memStoreLAB.isReclaimed());</span>
<span class="source-line-no">2884</span><span id="line-2884"> assertTrue(memStoreLAB.chunks.isEmpty());</span>
<span class="source-line-no">2885</span><span id="line-2885"> }</span>
<span class="source-line-no">2886</span><span id="line-2886"> }</span>
<span class="source-line-no">2887</span><span id="line-2887"></span>
<span class="source-line-no">2888</span><span id="line-2888"> private HStoreFile mockStoreFileWithLength(long length) {</span>
<span class="source-line-no">2889</span><span id="line-2889"> HStoreFile sf = mock(HStoreFile.class);</span>
<span class="source-line-no">2890</span><span id="line-2890"> StoreFileReader sfr = mock(StoreFileReader.class);</span>
<span class="source-line-no">2891</span><span id="line-2891"> when(sf.isHFile()).thenReturn(true);</span>
<span class="source-line-no">2892</span><span id="line-2892"> when(sf.getReader()).thenReturn(sfr);</span>
<span class="source-line-no">2893</span><span id="line-2893"> when(sfr.length()).thenReturn(length);</span>
<span class="source-line-no">2894</span><span id="line-2894"> return sf;</span>
<span class="source-line-no">2895</span><span id="line-2895"> }</span>
<span class="source-line-no">2896</span><span id="line-2896"></span>
<span class="source-line-no">2897</span><span id="line-2897"> private static class MyThread extends Thread {</span>
<span class="source-line-no">2898</span><span id="line-2898"> private StoreScanner scanner;</span>
<span class="source-line-no">2899</span><span id="line-2899"> private KeyValueHeap heap;</span>
<span class="source-line-no">2900</span><span id="line-2900"></span>
<span class="source-line-no">2901</span><span id="line-2901"> public MyThread(StoreScanner scanner) {</span>
<span class="source-line-no">2902</span><span id="line-2902"> this.scanner = scanner;</span>
<span class="source-line-no">2903</span><span id="line-2903"> }</span>
<span class="source-line-no">2904</span><span id="line-2904"></span>
<span class="source-line-no">2905</span><span id="line-2905"> public KeyValueHeap getHeap() {</span>
<span class="source-line-no">2906</span><span id="line-2906"> return this.heap;</span>
<span class="source-line-no">2907</span><span id="line-2907"> }</span>
<span class="source-line-no">2908</span><span id="line-2908"></span>
<span class="source-line-no">2909</span><span id="line-2909"> @Override</span>
<span class="source-line-no">2910</span><span id="line-2910"> public void run() {</span>
<span class="source-line-no">2911</span><span id="line-2911"> scanner.trySwitchToStreamRead();</span>
<span class="source-line-no">2912</span><span id="line-2912"> heap = scanner.heap;</span>
<span class="source-line-no">2913</span><span id="line-2913"> }</span>
<span class="source-line-no">2914</span><span id="line-2914"> }</span>
<span class="source-line-no">2915</span><span id="line-2915"></span>
<span class="source-line-no">2916</span><span id="line-2916"> private static class MyMemStoreCompactor extends MemStoreCompactor {</span>
<span class="source-line-no">2917</span><span id="line-2917"> private static final AtomicInteger RUNNER_COUNT = new AtomicInteger(0);</span>
<span class="source-line-no">2918</span><span id="line-2918"> private static final CountDownLatch START_COMPACTOR_LATCH = new CountDownLatch(1);</span>
<span class="source-line-no">2919</span><span id="line-2919"></span>
<span class="source-line-no">2920</span><span id="line-2920"> public MyMemStoreCompactor(CompactingMemStore compactingMemStore,</span>
<span class="source-line-no">2921</span><span id="line-2921"> MemoryCompactionPolicy compactionPolicy) throws IllegalArgumentIOException {</span>
<span class="source-line-no">2922</span><span id="line-2922"> super(compactingMemStore, compactionPolicy);</span>
<span class="source-line-no">2923</span><span id="line-2923"> }</span>
<span class="source-line-no">2924</span><span id="line-2924"></span>
<span class="source-line-no">2925</span><span id="line-2925"> @Override</span>
<span class="source-line-no">2926</span><span id="line-2926"> public boolean start() throws IOException {</span>
<span class="source-line-no">2927</span><span id="line-2927"> boolean isFirst = RUNNER_COUNT.getAndIncrement() == 0;</span>
<span class="source-line-no">2928</span><span id="line-2928"> if (isFirst) {</span>
<span class="source-line-no">2929</span><span id="line-2929"> try {</span>
<span class="source-line-no">2930</span><span id="line-2930"> START_COMPACTOR_LATCH.await();</span>
<span class="source-line-no">2931</span><span id="line-2931"> return super.start();</span>
<span class="source-line-no">2932</span><span id="line-2932"> } catch (InterruptedException ex) {</span>
<span class="source-line-no">2933</span><span id="line-2933"> throw new RuntimeException(ex);</span>
<span class="source-line-no">2934</span><span id="line-2934"> }</span>
<span class="source-line-no">2935</span><span id="line-2935"> }</span>
<span class="source-line-no">2936</span><span id="line-2936"> return super.start();</span>
<span class="source-line-no">2937</span><span id="line-2937"> }</span>
<span class="source-line-no">2938</span><span id="line-2938"> }</span>
<span class="source-line-no">2939</span><span id="line-2939"></span>
<span class="source-line-no">2940</span><span id="line-2940"> public static class MyCompactingMemStoreWithCustomCompactor extends CompactingMemStore {</span>
<span class="source-line-no">2941</span><span id="line-2941"> private static final AtomicInteger RUNNER_COUNT = new AtomicInteger(0);</span>
<span class="source-line-no">2942</span><span id="line-2942"></span>
<span class="source-line-no">2943</span><span id="line-2943"> public MyCompactingMemStoreWithCustomCompactor(Configuration conf, CellComparatorImpl c,</span>
<span class="source-line-no">2944</span><span id="line-2944"> HStore store, RegionServicesForStores regionServices, MemoryCompactionPolicy compactionPolicy)</span>
<span class="source-line-no">2945</span><span id="line-2945"> throws IOException {</span>
<span class="source-line-no">2946</span><span id="line-2946"> super(conf, c, store, regionServices, compactionPolicy);</span>
<span class="source-line-no">2947</span><span id="line-2947"> }</span>
<span class="source-line-no">2948</span><span id="line-2948"></span>
<span class="source-line-no">2949</span><span id="line-2949"> @Override</span>
<span class="source-line-no">2950</span><span id="line-2950"> protected MemStoreCompactor createMemStoreCompactor(MemoryCompactionPolicy compactionPolicy)</span>
<span class="source-line-no">2951</span><span id="line-2951"> throws IllegalArgumentIOException {</span>
<span class="source-line-no">2952</span><span id="line-2952"> return new MyMemStoreCompactor(this, compactionPolicy);</span>
<span class="source-line-no">2953</span><span id="line-2953"> }</span>
<span class="source-line-no">2954</span><span id="line-2954"></span>
<span class="source-line-no">2955</span><span id="line-2955"> @Override</span>
<span class="source-line-no">2956</span><span id="line-2956"> protected boolean setInMemoryCompactionFlag() {</span>
<span class="source-line-no">2957</span><span id="line-2957"> boolean rval = super.setInMemoryCompactionFlag();</span>
<span class="source-line-no">2958</span><span id="line-2958"> if (rval) {</span>
<span class="source-line-no">2959</span><span id="line-2959"> RUNNER_COUNT.incrementAndGet();</span>
<span class="source-line-no">2960</span><span id="line-2960"> if (LOG.isDebugEnabled()) {</span>
<span class="source-line-no">2961</span><span id="line-2961"> LOG.debug("runner count: " + RUNNER_COUNT.get());</span>
<span class="source-line-no">2962</span><span id="line-2962"> }</span>
<span class="source-line-no">2963</span><span id="line-2963"> }</span>
<span class="source-line-no">2964</span><span id="line-2964"> return rval;</span>
<span class="source-line-no">2965</span><span id="line-2965"> }</span>
<span class="source-line-no">2966</span><span id="line-2966"> }</span>
<span class="source-line-no">2967</span><span id="line-2967"></span>
<span class="source-line-no">2968</span><span id="line-2968"> public static class MyCompactingMemStore extends CompactingMemStore {</span>
<span class="source-line-no">2969</span><span id="line-2969"> private static final AtomicBoolean START_TEST = new AtomicBoolean(false);</span>
<span class="source-line-no">2970</span><span id="line-2970"> private final CountDownLatch getScannerLatch = new CountDownLatch(1);</span>
<span class="source-line-no">2971</span><span id="line-2971"> private final CountDownLatch snapshotLatch = new CountDownLatch(1);</span>
<span class="source-line-no">2972</span><span id="line-2972"></span>
<span class="source-line-no">2973</span><span id="line-2973"> public MyCompactingMemStore(Configuration conf, CellComparatorImpl c, HStore store,</span>
<span class="source-line-no">2974</span><span id="line-2974"> RegionServicesForStores regionServices, MemoryCompactionPolicy compactionPolicy)</span>
<span class="source-line-no">2975</span><span id="line-2975"> throws IOException {</span>
<span class="source-line-no">2976</span><span id="line-2976"> super(conf, c, store, regionServices, compactionPolicy);</span>
<span class="source-line-no">2977</span><span id="line-2977"> }</span>
<span class="source-line-no">2978</span><span id="line-2978"></span>
<span class="source-line-no">2979</span><span id="line-2979"> @Override</span>
<span class="source-line-no">2980</span><span id="line-2980"> protected List&lt;KeyValueScanner&gt; createList(int capacity) {</span>
<span class="source-line-no">2981</span><span id="line-2981"> if (START_TEST.get()) {</span>
<span class="source-line-no">2982</span><span id="line-2982"> try {</span>
<span class="source-line-no">2983</span><span id="line-2983"> getScannerLatch.countDown();</span>
<span class="source-line-no">2984</span><span id="line-2984"> snapshotLatch.await();</span>
<span class="source-line-no">2985</span><span id="line-2985"> } catch (InterruptedException e) {</span>
<span class="source-line-no">2986</span><span id="line-2986"> throw new RuntimeException(e);</span>
<span class="source-line-no">2987</span><span id="line-2987"> }</span>
<span class="source-line-no">2988</span><span id="line-2988"> }</span>
<span class="source-line-no">2989</span><span id="line-2989"> return new ArrayList&lt;&gt;(capacity);</span>
<span class="source-line-no">2990</span><span id="line-2990"> }</span>
<span class="source-line-no">2991</span><span id="line-2991"></span>
<span class="source-line-no">2992</span><span id="line-2992"> @Override</span>
<span class="source-line-no">2993</span><span id="line-2993"> protected void pushActiveToPipeline(MutableSegment active, boolean checkEmpty) {</span>
<span class="source-line-no">2994</span><span id="line-2994"> if (START_TEST.get()) {</span>
<span class="source-line-no">2995</span><span id="line-2995"> try {</span>
<span class="source-line-no">2996</span><span id="line-2996"> getScannerLatch.await();</span>
<span class="source-line-no">2997</span><span id="line-2997"> } catch (InterruptedException e) {</span>
<span class="source-line-no">2998</span><span id="line-2998"> throw new RuntimeException(e);</span>
<span class="source-line-no">2999</span><span id="line-2999"> }</span>
<span class="source-line-no">3000</span><span id="line-3000"> }</span>
<span class="source-line-no">3001</span><span id="line-3001"></span>
<span class="source-line-no">3002</span><span id="line-3002"> super.pushActiveToPipeline(active, checkEmpty);</span>
<span class="source-line-no">3003</span><span id="line-3003"> if (START_TEST.get()) {</span>
<span class="source-line-no">3004</span><span id="line-3004"> snapshotLatch.countDown();</span>
<span class="source-line-no">3005</span><span id="line-3005"> }</span>
<span class="source-line-no">3006</span><span id="line-3006"> }</span>
<span class="source-line-no">3007</span><span id="line-3007"> }</span>
<span class="source-line-no">3008</span><span id="line-3008"></span>
<span class="source-line-no">3009</span><span id="line-3009"> interface MyListHook {</span>
<span class="source-line-no">3010</span><span id="line-3010"> void hook(int currentSize);</span>
<span class="source-line-no">3011</span><span id="line-3011"> }</span>
<span class="source-line-no">3012</span><span id="line-3012"></span>
<span class="source-line-no">3013</span><span id="line-3013"> private static class MyList&lt;T&gt; implements List&lt;T&gt; {</span>
<span class="source-line-no">3014</span><span id="line-3014"> private final List&lt;T&gt; delegatee = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">3015</span><span id="line-3015"> private final MyListHook hookAtAdd;</span>
<span class="source-line-no">3016</span><span id="line-3016"></span>
<span class="source-line-no">3017</span><span id="line-3017"> MyList(final MyListHook hookAtAdd) {</span>
<span class="source-line-no">3018</span><span id="line-3018"> this.hookAtAdd = hookAtAdd;</span>
<span class="source-line-no">3019</span><span id="line-3019"> }</span>
<span class="source-line-no">3020</span><span id="line-3020"></span>
<span class="source-line-no">3021</span><span id="line-3021"> @Override</span>
<span class="source-line-no">3022</span><span id="line-3022"> public int size() {</span>
<span class="source-line-no">3023</span><span id="line-3023"> return delegatee.size();</span>
<span class="source-line-no">3024</span><span id="line-3024"> }</span>
<span class="source-line-no">3025</span><span id="line-3025"></span>
<span class="source-line-no">3026</span><span id="line-3026"> @Override</span>
<span class="source-line-no">3027</span><span id="line-3027"> public boolean isEmpty() {</span>
<span class="source-line-no">3028</span><span id="line-3028"> return delegatee.isEmpty();</span>
<span class="source-line-no">3029</span><span id="line-3029"> }</span>
<span class="source-line-no">3030</span><span id="line-3030"></span>
<span class="source-line-no">3031</span><span id="line-3031"> @Override</span>
<span class="source-line-no">3032</span><span id="line-3032"> public boolean contains(Object o) {</span>
<span class="source-line-no">3033</span><span id="line-3033"> return delegatee.contains(o);</span>
<span class="source-line-no">3034</span><span id="line-3034"> }</span>
<span class="source-line-no">3035</span><span id="line-3035"></span>
<span class="source-line-no">3036</span><span id="line-3036"> @Override</span>
<span class="source-line-no">3037</span><span id="line-3037"> public Iterator&lt;T&gt; iterator() {</span>
<span class="source-line-no">3038</span><span id="line-3038"> return delegatee.iterator();</span>
<span class="source-line-no">3039</span><span id="line-3039"> }</span>
<span class="source-line-no">3040</span><span id="line-3040"></span>
<span class="source-line-no">3041</span><span id="line-3041"> @Override</span>
<span class="source-line-no">3042</span><span id="line-3042"> public Object[] toArray() {</span>
<span class="source-line-no">3043</span><span id="line-3043"> return delegatee.toArray();</span>
<span class="source-line-no">3044</span><span id="line-3044"> }</span>
<span class="source-line-no">3045</span><span id="line-3045"></span>
<span class="source-line-no">3046</span><span id="line-3046"> @Override</span>
<span class="source-line-no">3047</span><span id="line-3047"> public &lt;R&gt; R[] toArray(R[] a) {</span>
<span class="source-line-no">3048</span><span id="line-3048"> return delegatee.toArray(a);</span>
<span class="source-line-no">3049</span><span id="line-3049"> }</span>
<span class="source-line-no">3050</span><span id="line-3050"></span>
<span class="source-line-no">3051</span><span id="line-3051"> @Override</span>
<span class="source-line-no">3052</span><span id="line-3052"> public boolean add(T e) {</span>
<span class="source-line-no">3053</span><span id="line-3053"> hookAtAdd.hook(size());</span>
<span class="source-line-no">3054</span><span id="line-3054"> return delegatee.add(e);</span>
<span class="source-line-no">3055</span><span id="line-3055"> }</span>
<span class="source-line-no">3056</span><span id="line-3056"></span>
<span class="source-line-no">3057</span><span id="line-3057"> @Override</span>
<span class="source-line-no">3058</span><span id="line-3058"> public boolean remove(Object o) {</span>
<span class="source-line-no">3059</span><span id="line-3059"> return delegatee.remove(o);</span>
<span class="source-line-no">3060</span><span id="line-3060"> }</span>
<span class="source-line-no">3061</span><span id="line-3061"></span>
<span class="source-line-no">3062</span><span id="line-3062"> @Override</span>
<span class="source-line-no">3063</span><span id="line-3063"> public boolean containsAll(Collection&lt;?&gt; c) {</span>
<span class="source-line-no">3064</span><span id="line-3064"> return delegatee.containsAll(c);</span>
<span class="source-line-no">3065</span><span id="line-3065"> }</span>
<span class="source-line-no">3066</span><span id="line-3066"></span>
<span class="source-line-no">3067</span><span id="line-3067"> @Override</span>
<span class="source-line-no">3068</span><span id="line-3068"> public boolean addAll(Collection&lt;? extends T&gt; c) {</span>
<span class="source-line-no">3069</span><span id="line-3069"> return delegatee.addAll(c);</span>
<span class="source-line-no">3070</span><span id="line-3070"> }</span>
<span class="source-line-no">3071</span><span id="line-3071"></span>
<span class="source-line-no">3072</span><span id="line-3072"> @Override</span>
<span class="source-line-no">3073</span><span id="line-3073"> public boolean addAll(int index, Collection&lt;? extends T&gt; c) {</span>
<span class="source-line-no">3074</span><span id="line-3074"> return delegatee.addAll(index, c);</span>
<span class="source-line-no">3075</span><span id="line-3075"> }</span>
<span class="source-line-no">3076</span><span id="line-3076"></span>
<span class="source-line-no">3077</span><span id="line-3077"> @Override</span>
<span class="source-line-no">3078</span><span id="line-3078"> public boolean removeAll(Collection&lt;?&gt; c) {</span>
<span class="source-line-no">3079</span><span id="line-3079"> return delegatee.removeAll(c);</span>
<span class="source-line-no">3080</span><span id="line-3080"> }</span>
<span class="source-line-no">3081</span><span id="line-3081"></span>
<span class="source-line-no">3082</span><span id="line-3082"> @Override</span>
<span class="source-line-no">3083</span><span id="line-3083"> public boolean retainAll(Collection&lt;?&gt; c) {</span>
<span class="source-line-no">3084</span><span id="line-3084"> return delegatee.retainAll(c);</span>
<span class="source-line-no">3085</span><span id="line-3085"> }</span>
<span class="source-line-no">3086</span><span id="line-3086"></span>
<span class="source-line-no">3087</span><span id="line-3087"> @Override</span>
<span class="source-line-no">3088</span><span id="line-3088"> public void clear() {</span>
<span class="source-line-no">3089</span><span id="line-3089"> delegatee.clear();</span>
<span class="source-line-no">3090</span><span id="line-3090"> }</span>
<span class="source-line-no">3091</span><span id="line-3091"></span>
<span class="source-line-no">3092</span><span id="line-3092"> @Override</span>
<span class="source-line-no">3093</span><span id="line-3093"> public T get(int index) {</span>
<span class="source-line-no">3094</span><span id="line-3094"> return delegatee.get(index);</span>
<span class="source-line-no">3095</span><span id="line-3095"> }</span>
<span class="source-line-no">3096</span><span id="line-3096"></span>
<span class="source-line-no">3097</span><span id="line-3097"> @Override</span>
<span class="source-line-no">3098</span><span id="line-3098"> public T set(int index, T element) {</span>
<span class="source-line-no">3099</span><span id="line-3099"> return delegatee.set(index, element);</span>
<span class="source-line-no">3100</span><span id="line-3100"> }</span>
<span class="source-line-no">3101</span><span id="line-3101"></span>
<span class="source-line-no">3102</span><span id="line-3102"> @Override</span>
<span class="source-line-no">3103</span><span id="line-3103"> public void add(int index, T element) {</span>
<span class="source-line-no">3104</span><span id="line-3104"> delegatee.add(index, element);</span>
<span class="source-line-no">3105</span><span id="line-3105"> }</span>
<span class="source-line-no">3106</span><span id="line-3106"></span>
<span class="source-line-no">3107</span><span id="line-3107"> @Override</span>
<span class="source-line-no">3108</span><span id="line-3108"> public T remove(int index) {</span>
<span class="source-line-no">3109</span><span id="line-3109"> return delegatee.remove(index);</span>
<span class="source-line-no">3110</span><span id="line-3110"> }</span>
<span class="source-line-no">3111</span><span id="line-3111"></span>
<span class="source-line-no">3112</span><span id="line-3112"> @Override</span>
<span class="source-line-no">3113</span><span id="line-3113"> public int indexOf(Object o) {</span>
<span class="source-line-no">3114</span><span id="line-3114"> return delegatee.indexOf(o);</span>
<span class="source-line-no">3115</span><span id="line-3115"> }</span>
<span class="source-line-no">3116</span><span id="line-3116"></span>
<span class="source-line-no">3117</span><span id="line-3117"> @Override</span>
<span class="source-line-no">3118</span><span id="line-3118"> public int lastIndexOf(Object o) {</span>
<span class="source-line-no">3119</span><span id="line-3119"> return delegatee.lastIndexOf(o);</span>
<span class="source-line-no">3120</span><span id="line-3120"> }</span>
<span class="source-line-no">3121</span><span id="line-3121"></span>
<span class="source-line-no">3122</span><span id="line-3122"> @Override</span>
<span class="source-line-no">3123</span><span id="line-3123"> public ListIterator&lt;T&gt; listIterator() {</span>
<span class="source-line-no">3124</span><span id="line-3124"> return delegatee.listIterator();</span>
<span class="source-line-no">3125</span><span id="line-3125"> }</span>
<span class="source-line-no">3126</span><span id="line-3126"></span>
<span class="source-line-no">3127</span><span id="line-3127"> @Override</span>
<span class="source-line-no">3128</span><span id="line-3128"> public ListIterator&lt;T&gt; listIterator(int index) {</span>
<span class="source-line-no">3129</span><span id="line-3129"> return delegatee.listIterator(index);</span>
<span class="source-line-no">3130</span><span id="line-3130"> }</span>
<span class="source-line-no">3131</span><span id="line-3131"></span>
<span class="source-line-no">3132</span><span id="line-3132"> @Override</span>
<span class="source-line-no">3133</span><span id="line-3133"> public List&lt;T&gt; subList(int fromIndex, int toIndex) {</span>
<span class="source-line-no">3134</span><span id="line-3134"> return delegatee.subList(fromIndex, toIndex);</span>
<span class="source-line-no">3135</span><span id="line-3135"> }</span>
<span class="source-line-no">3136</span><span id="line-3136"> }</span>
<span class="source-line-no">3137</span><span id="line-3137"></span>
<span class="source-line-no">3138</span><span id="line-3138"> public static class MyCompactingMemStore2 extends CompactingMemStore {</span>
<span class="source-line-no">3139</span><span id="line-3139"> private static final String LARGE_CELL_THREAD_NAME = "largeCellThread";</span>
<span class="source-line-no">3140</span><span id="line-3140"> private static final String SMALL_CELL_THREAD_NAME = "smallCellThread";</span>
<span class="source-line-no">3141</span><span id="line-3141"> private final CyclicBarrier preCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3142</span><span id="line-3142"> private final CyclicBarrier postCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3143</span><span id="line-3143"> private final AtomicInteger largeCellPreUpdateCounter = new AtomicInteger(0);</span>
<span class="source-line-no">3144</span><span id="line-3144"> private final AtomicInteger smallCellPreUpdateCounter = new AtomicInteger(0);</span>
<span class="source-line-no">3145</span><span id="line-3145"></span>
<span class="source-line-no">3146</span><span id="line-3146"> public MyCompactingMemStore2(Configuration conf, CellComparatorImpl cellComparator,</span>
<span class="source-line-no">3147</span><span id="line-3147"> HStore store, RegionServicesForStores regionServices, MemoryCompactionPolicy compactionPolicy)</span>
<span class="source-line-no">3148</span><span id="line-3148"> throws IOException {</span>
<span class="source-line-no">3149</span><span id="line-3149"> super(conf, cellComparator, store, regionServices, compactionPolicy);</span>
<span class="source-line-no">3150</span><span id="line-3150"> }</span>
<span class="source-line-no">3151</span><span id="line-3151"></span>
<span class="source-line-no">3152</span><span id="line-3152"> @Override</span>
<span class="source-line-no">3153</span><span id="line-3153"> protected boolean checkAndAddToActiveSize(MutableSegment currActive, Cell cellToAdd,</span>
<span class="source-line-no">3154</span><span id="line-3154"> MemStoreSizing memstoreSizing) {</span>
<span class="source-line-no">3155</span><span id="line-3155"> if (Thread.currentThread().getName().equals(LARGE_CELL_THREAD_NAME)) {</span>
<span class="source-line-no">3156</span><span id="line-3156"> int currentCount = largeCellPreUpdateCounter.incrementAndGet();</span>
<span class="source-line-no">3157</span><span id="line-3157"> if (currentCount &lt;= 1) {</span>
<span class="source-line-no">3158</span><span id="line-3158"> try {</span>
<span class="source-line-no">3159</span><span id="line-3159"> /**</span>
<span class="source-line-no">3160</span><span id="line-3160"> * smallCellThread enters CompactingMemStore.checkAndAddToActiveSize first, then</span>
<span class="source-line-no">3161</span><span id="line-3161"> * largeCellThread enters CompactingMemStore.checkAndAddToActiveSize, and then</span>
<span class="source-line-no">3162</span><span id="line-3162"> * largeCellThread invokes flushInMemory.</span>
<span class="source-line-no">3163</span><span id="line-3163"> */</span>
<span class="source-line-no">3164</span><span id="line-3164"> preCyclicBarrier.await();</span>
<span class="source-line-no">3165</span><span id="line-3165"> } catch (Throwable e) {</span>
<span class="source-line-no">3166</span><span id="line-3166"> throw new RuntimeException(e);</span>
<span class="source-line-no">3167</span><span id="line-3167"> }</span>
<span class="source-line-no">3168</span><span id="line-3168"> }</span>
<span class="source-line-no">3169</span><span id="line-3169"> }</span>
<span class="source-line-no">3170</span><span id="line-3170"></span>
<span class="source-line-no">3171</span><span id="line-3171"> boolean returnValue = super.checkAndAddToActiveSize(currActive, cellToAdd, memstoreSizing);</span>
<span class="source-line-no">3172</span><span id="line-3172"> if (Thread.currentThread().getName().equals(SMALL_CELL_THREAD_NAME)) {</span>
<span class="source-line-no">3173</span><span id="line-3173"> try {</span>
<span class="source-line-no">3174</span><span id="line-3174"> preCyclicBarrier.await();</span>
<span class="source-line-no">3175</span><span id="line-3175"> } catch (Throwable e) {</span>
<span class="source-line-no">3176</span><span id="line-3176"> throw new RuntimeException(e);</span>
<span class="source-line-no">3177</span><span id="line-3177"> }</span>
<span class="source-line-no">3178</span><span id="line-3178"> }</span>
<span class="source-line-no">3179</span><span id="line-3179"> return returnValue;</span>
<span class="source-line-no">3180</span><span id="line-3180"> }</span>
<span class="source-line-no">3181</span><span id="line-3181"></span>
<span class="source-line-no">3182</span><span id="line-3182"> @Override</span>
<span class="source-line-no">3183</span><span id="line-3183"> protected void doAdd(MutableSegment currentActive, ExtendedCell cell,</span>
<span class="source-line-no">3184</span><span id="line-3184"> MemStoreSizing memstoreSizing) {</span>
<span class="source-line-no">3185</span><span id="line-3185"> if (Thread.currentThread().getName().equals(SMALL_CELL_THREAD_NAME)) {</span>
<span class="source-line-no">3186</span><span id="line-3186"> try {</span>
<span class="source-line-no">3187</span><span id="line-3187"> /**</span>
<span class="source-line-no">3188</span><span id="line-3188"> * After largeCellThread finished flushInMemory method, smallCellThread can add cell to</span>
<span class="source-line-no">3189</span><span id="line-3189"> * currentActive . That is to say when largeCellThread called flushInMemory method,</span>
<span class="source-line-no">3190</span><span id="line-3190"> * currentActive has no cell.</span>
<span class="source-line-no">3191</span><span id="line-3191"> */</span>
<span class="source-line-no">3192</span><span id="line-3192"> postCyclicBarrier.await();</span>
<span class="source-line-no">3193</span><span id="line-3193"> } catch (Throwable e) {</span>
<span class="source-line-no">3194</span><span id="line-3194"> throw new RuntimeException(e);</span>
<span class="source-line-no">3195</span><span id="line-3195"> }</span>
<span class="source-line-no">3196</span><span id="line-3196"> }</span>
<span class="source-line-no">3197</span><span id="line-3197"> super.doAdd(currentActive, cell, memstoreSizing);</span>
<span class="source-line-no">3198</span><span id="line-3198"> }</span>
<span class="source-line-no">3199</span><span id="line-3199"></span>
<span class="source-line-no">3200</span><span id="line-3200"> @Override</span>
<span class="source-line-no">3201</span><span id="line-3201"> protected void flushInMemory(MutableSegment currentActiveMutableSegment) {</span>
<span class="source-line-no">3202</span><span id="line-3202"> super.flushInMemory(currentActiveMutableSegment);</span>
<span class="source-line-no">3203</span><span id="line-3203"> if (Thread.currentThread().getName().equals(LARGE_CELL_THREAD_NAME)) {</span>
<span class="source-line-no">3204</span><span id="line-3204"> if (largeCellPreUpdateCounter.get() &lt;= 1) {</span>
<span class="source-line-no">3205</span><span id="line-3205"> try {</span>
<span class="source-line-no">3206</span><span id="line-3206"> postCyclicBarrier.await();</span>
<span class="source-line-no">3207</span><span id="line-3207"> } catch (Throwable e) {</span>
<span class="source-line-no">3208</span><span id="line-3208"> throw new RuntimeException(e);</span>
<span class="source-line-no">3209</span><span id="line-3209"> }</span>
<span class="source-line-no">3210</span><span id="line-3210"> }</span>
<span class="source-line-no">3211</span><span id="line-3211"> }</span>
<span class="source-line-no">3212</span><span id="line-3212"> }</span>
<span class="source-line-no">3213</span><span id="line-3213"></span>
<span class="source-line-no">3214</span><span id="line-3214"> }</span>
<span class="source-line-no">3215</span><span id="line-3215"></span>
<span class="source-line-no">3216</span><span id="line-3216"> public static class MyCompactingMemStore3 extends CompactingMemStore {</span>
<span class="source-line-no">3217</span><span id="line-3217"> private static final String LARGE_CELL_THREAD_NAME = "largeCellThread";</span>
<span class="source-line-no">3218</span><span id="line-3218"> private static final String SMALL_CELL_THREAD_NAME = "smallCellThread";</span>
<span class="source-line-no">3219</span><span id="line-3219"></span>
<span class="source-line-no">3220</span><span id="line-3220"> private final CyclicBarrier preCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3221</span><span id="line-3221"> private final CyclicBarrier postCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3222</span><span id="line-3222"> private final AtomicInteger flushCounter = new AtomicInteger(0);</span>
<span class="source-line-no">3223</span><span id="line-3223"> private static final int CELL_COUNT = 5;</span>
<span class="source-line-no">3224</span><span id="line-3224"> private boolean flushByteSizeLessThanSmallAndLargeCellSize = true;</span>
<span class="source-line-no">3225</span><span id="line-3225"></span>
<span class="source-line-no">3226</span><span id="line-3226"> public MyCompactingMemStore3(Configuration conf, CellComparatorImpl cellComparator,</span>
<span class="source-line-no">3227</span><span id="line-3227"> HStore store, RegionServicesForStores regionServices, MemoryCompactionPolicy compactionPolicy)</span>
<span class="source-line-no">3228</span><span id="line-3228"> throws IOException {</span>
<span class="source-line-no">3229</span><span id="line-3229"> super(conf, cellComparator, store, regionServices, compactionPolicy);</span>
<span class="source-line-no">3230</span><span id="line-3230"> }</span>
<span class="source-line-no">3231</span><span id="line-3231"></span>
<span class="source-line-no">3232</span><span id="line-3232"> @Override</span>
<span class="source-line-no">3233</span><span id="line-3233"> protected boolean checkAndAddToActiveSize(MutableSegment currActive, Cell cellToAdd,</span>
<span class="source-line-no">3234</span><span id="line-3234"> MemStoreSizing memstoreSizing) {</span>
<span class="source-line-no">3235</span><span id="line-3235"> if (!flushByteSizeLessThanSmallAndLargeCellSize) {</span>
<span class="source-line-no">3236</span><span id="line-3236"> return super.checkAndAddToActiveSize(currActive, cellToAdd, memstoreSizing);</span>
<span class="source-line-no">3237</span><span id="line-3237"> }</span>
<span class="source-line-no">3238</span><span id="line-3238"> if (Thread.currentThread().getName().equals(LARGE_CELL_THREAD_NAME)) {</span>
<span class="source-line-no">3239</span><span id="line-3239"> try {</span>
<span class="source-line-no">3240</span><span id="line-3240"> preCyclicBarrier.await();</span>
<span class="source-line-no">3241</span><span id="line-3241"> } catch (Throwable e) {</span>
<span class="source-line-no">3242</span><span id="line-3242"> throw new RuntimeException(e);</span>
<span class="source-line-no">3243</span><span id="line-3243"> }</span>
<span class="source-line-no">3244</span><span id="line-3244"> }</span>
<span class="source-line-no">3245</span><span id="line-3245"></span>
<span class="source-line-no">3246</span><span id="line-3246"> boolean returnValue = super.checkAndAddToActiveSize(currActive, cellToAdd, memstoreSizing);</span>
<span class="source-line-no">3247</span><span id="line-3247"> if (Thread.currentThread().getName().equals(SMALL_CELL_THREAD_NAME)) {</span>
<span class="source-line-no">3248</span><span id="line-3248"> try {</span>
<span class="source-line-no">3249</span><span id="line-3249"> preCyclicBarrier.await();</span>
<span class="source-line-no">3250</span><span id="line-3250"> } catch (Throwable e) {</span>
<span class="source-line-no">3251</span><span id="line-3251"> throw new RuntimeException(e);</span>
<span class="source-line-no">3252</span><span id="line-3252"> }</span>
<span class="source-line-no">3253</span><span id="line-3253"> }</span>
<span class="source-line-no">3254</span><span id="line-3254"> return returnValue;</span>
<span class="source-line-no">3255</span><span id="line-3255"> }</span>
<span class="source-line-no">3256</span><span id="line-3256"></span>
<span class="source-line-no">3257</span><span id="line-3257"> @Override</span>
<span class="source-line-no">3258</span><span id="line-3258"> protected void postUpdate(MutableSegment currentActiveMutableSegment) {</span>
<span class="source-line-no">3259</span><span id="line-3259"> super.postUpdate(currentActiveMutableSegment);</span>
<span class="source-line-no">3260</span><span id="line-3260"> if (!flushByteSizeLessThanSmallAndLargeCellSize) {</span>
<span class="source-line-no">3261</span><span id="line-3261"> try {</span>
<span class="source-line-no">3262</span><span id="line-3262"> postCyclicBarrier.await();</span>
<span class="source-line-no">3263</span><span id="line-3263"> } catch (Throwable e) {</span>
<span class="source-line-no">3264</span><span id="line-3264"> throw new RuntimeException(e);</span>
<span class="source-line-no">3265</span><span id="line-3265"> }</span>
<span class="source-line-no">3266</span><span id="line-3266"> return;</span>
<span class="source-line-no">3267</span><span id="line-3267"> }</span>
<span class="source-line-no">3268</span><span id="line-3268"></span>
<span class="source-line-no">3269</span><span id="line-3269"> if (Thread.currentThread().getName().equals(SMALL_CELL_THREAD_NAME)) {</span>
<span class="source-line-no">3270</span><span id="line-3270"> try {</span>
<span class="source-line-no">3271</span><span id="line-3271"> postCyclicBarrier.await();</span>
<span class="source-line-no">3272</span><span id="line-3272"> } catch (Throwable e) {</span>
<span class="source-line-no">3273</span><span id="line-3273"> throw new RuntimeException(e);</span>
<span class="source-line-no">3274</span><span id="line-3274"> }</span>
<span class="source-line-no">3275</span><span id="line-3275"> }</span>
<span class="source-line-no">3276</span><span id="line-3276"> }</span>
<span class="source-line-no">3277</span><span id="line-3277"></span>
<span class="source-line-no">3278</span><span id="line-3278"> @Override</span>
<span class="source-line-no">3279</span><span id="line-3279"> protected void flushInMemory(MutableSegment currentActiveMutableSegment) {</span>
<span class="source-line-no">3280</span><span id="line-3280"> super.flushInMemory(currentActiveMutableSegment);</span>
<span class="source-line-no">3281</span><span id="line-3281"> flushCounter.incrementAndGet();</span>
<span class="source-line-no">3282</span><span id="line-3282"> if (!flushByteSizeLessThanSmallAndLargeCellSize) {</span>
<span class="source-line-no">3283</span><span id="line-3283"> return;</span>
<span class="source-line-no">3284</span><span id="line-3284"> }</span>
<span class="source-line-no">3285</span><span id="line-3285"></span>
<span class="source-line-no">3286</span><span id="line-3286"> assertTrue(Thread.currentThread().getName().equals(LARGE_CELL_THREAD_NAME));</span>
<span class="source-line-no">3287</span><span id="line-3287"> try {</span>
<span class="source-line-no">3288</span><span id="line-3288"> postCyclicBarrier.await();</span>
<span class="source-line-no">3289</span><span id="line-3289"> } catch (Throwable e) {</span>
<span class="source-line-no">3290</span><span id="line-3290"> throw new RuntimeException(e);</span>
<span class="source-line-no">3291</span><span id="line-3291"> }</span>
<span class="source-line-no">3292</span><span id="line-3292"></span>
<span class="source-line-no">3293</span><span id="line-3293"> }</span>
<span class="source-line-no">3294</span><span id="line-3294"></span>
<span class="source-line-no">3295</span><span id="line-3295"> void disableCompaction() {</span>
<span class="source-line-no">3296</span><span id="line-3296"> allowCompaction.set(false);</span>
<span class="source-line-no">3297</span><span id="line-3297"> }</span>
<span class="source-line-no">3298</span><span id="line-3298"></span>
<span class="source-line-no">3299</span><span id="line-3299"> void enableCompaction() {</span>
<span class="source-line-no">3300</span><span id="line-3300"> allowCompaction.set(true);</span>
<span class="source-line-no">3301</span><span id="line-3301"> }</span>
<span class="source-line-no">3302</span><span id="line-3302"></span>
<span class="source-line-no">3303</span><span id="line-3303"> }</span>
<span class="source-line-no">3304</span><span id="line-3304"></span>
<span class="source-line-no">3305</span><span id="line-3305"> public static class MyCompactingMemStore4 extends CompactingMemStore {</span>
<span class="source-line-no">3306</span><span id="line-3306"> private static final String TAKE_SNAPSHOT_THREAD_NAME = "takeSnapShotThread";</span>
<span class="source-line-no">3307</span><span id="line-3307"> /**</span>
<span class="source-line-no">3308</span><span id="line-3308"> * {@link CompactingMemStore#flattenOneSegment} must execute after</span>
<span class="source-line-no">3309</span><span id="line-3309"> * {@link CompactingMemStore#getImmutableSegments}</span>
<span class="source-line-no">3310</span><span id="line-3310"> */</span>
<span class="source-line-no">3311</span><span id="line-3311"> private final CyclicBarrier flattenOneSegmentPreCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3312</span><span id="line-3312"> /**</span>
<span class="source-line-no">3313</span><span id="line-3313"> * Only after {@link CompactingMemStore#flattenOneSegment} completed,</span>
<span class="source-line-no">3314</span><span id="line-3314"> * {@link CompactingMemStore#swapPipelineWithNull} could execute.</span>
<span class="source-line-no">3315</span><span id="line-3315"> */</span>
<span class="source-line-no">3316</span><span id="line-3316"> private final CyclicBarrier flattenOneSegmentPostCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3317</span><span id="line-3317"> /**</span>
<span class="source-line-no">3318</span><span id="line-3318"> * Only the in memory compact thread enters {@link CompactingMemStore#flattenOneSegment},the</span>
<span class="source-line-no">3319</span><span id="line-3319"> * snapshot thread starts {@link CompactingMemStore#snapshot},because</span>
<span class="source-line-no">3320</span><span id="line-3320"> * {@link CompactingMemStore#snapshot} would invoke {@link CompactingMemStore#stopCompaction}.</span>
<span class="source-line-no">3321</span><span id="line-3321"> */</span>
<span class="source-line-no">3322</span><span id="line-3322"> private final CyclicBarrier snapShotStartCyclicCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3323</span><span id="line-3323"> /**</span>
<span class="source-line-no">3324</span><span id="line-3324"> * To wait for {@link CompactingMemStore.InMemoryCompactionRunnable} stopping.</span>
<span class="source-line-no">3325</span><span id="line-3325"> */</span>
<span class="source-line-no">3326</span><span id="line-3326"> private final CyclicBarrier inMemoryCompactionEndCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3327</span><span id="line-3327"> private final AtomicInteger getImmutableSegmentsListCounter = new AtomicInteger(0);</span>
<span class="source-line-no">3328</span><span id="line-3328"> private final AtomicInteger swapPipelineWithNullCounter = new AtomicInteger(0);</span>
<span class="source-line-no">3329</span><span id="line-3329"> private final AtomicInteger flattenOneSegmentCounter = new AtomicInteger(0);</span>
<span class="source-line-no">3330</span><span id="line-3330"> private final AtomicInteger setInMemoryCompactionFlagCounter = new AtomicInteger(0);</span>
<span class="source-line-no">3331</span><span id="line-3331"></span>
<span class="source-line-no">3332</span><span id="line-3332"> public MyCompactingMemStore4(Configuration conf, CellComparatorImpl cellComparator,</span>
<span class="source-line-no">3333</span><span id="line-3333"> HStore store, RegionServicesForStores regionServices, MemoryCompactionPolicy compactionPolicy)</span>
<span class="source-line-no">3334</span><span id="line-3334"> throws IOException {</span>
<span class="source-line-no">3335</span><span id="line-3335"> super(conf, cellComparator, store, regionServices, compactionPolicy);</span>
<span class="source-line-no">3336</span><span id="line-3336"> }</span>
<span class="source-line-no">3337</span><span id="line-3337"></span>
<span class="source-line-no">3338</span><span id="line-3338"> @Override</span>
<span class="source-line-no">3339</span><span id="line-3339"> public VersionedSegmentsList getImmutableSegments() {</span>
<span class="source-line-no">3340</span><span id="line-3340"> VersionedSegmentsList result = super.getImmutableSegments();</span>
<span class="source-line-no">3341</span><span id="line-3341"> if (Thread.currentThread().getName().equals(TAKE_SNAPSHOT_THREAD_NAME)) {</span>
<span class="source-line-no">3342</span><span id="line-3342"> int currentCount = getImmutableSegmentsListCounter.incrementAndGet();</span>
<span class="source-line-no">3343</span><span id="line-3343"> if (currentCount &lt;= 1) {</span>
<span class="source-line-no">3344</span><span id="line-3344"> try {</span>
<span class="source-line-no">3345</span><span id="line-3345"> flattenOneSegmentPreCyclicBarrier.await();</span>
<span class="source-line-no">3346</span><span id="line-3346"> } catch (Throwable e) {</span>
<span class="source-line-no">3347</span><span id="line-3347"> throw new RuntimeException(e);</span>
<span class="source-line-no">3348</span><span id="line-3348"> }</span>
<span class="source-line-no">3349</span><span id="line-3349"> }</span>
<span class="source-line-no">3350</span><span id="line-3350"> }</span>
<span class="source-line-no">3351</span><span id="line-3351"> return result;</span>
<span class="source-line-no">3352</span><span id="line-3352"> }</span>
<span class="source-line-no">3353</span><span id="line-3353"></span>
<span class="source-line-no">3354</span><span id="line-3354"> @Override</span>
<span class="source-line-no">3355</span><span id="line-3355"> protected boolean swapPipelineWithNull(VersionedSegmentsList segments) {</span>
<span class="source-line-no">3356</span><span id="line-3356"> if (Thread.currentThread().getName().equals(TAKE_SNAPSHOT_THREAD_NAME)) {</span>
<span class="source-line-no">3357</span><span id="line-3357"> int currentCount = swapPipelineWithNullCounter.incrementAndGet();</span>
<span class="source-line-no">3358</span><span id="line-3358"> if (currentCount &lt;= 1) {</span>
<span class="source-line-no">3359</span><span id="line-3359"> try {</span>
<span class="source-line-no">3360</span><span id="line-3360"> flattenOneSegmentPostCyclicBarrier.await();</span>
<span class="source-line-no">3361</span><span id="line-3361"> } catch (Throwable e) {</span>
<span class="source-line-no">3362</span><span id="line-3362"> throw new RuntimeException(e);</span>
<span class="source-line-no">3363</span><span id="line-3363"> }</span>
<span class="source-line-no">3364</span><span id="line-3364"> }</span>
<span class="source-line-no">3365</span><span id="line-3365"> }</span>
<span class="source-line-no">3366</span><span id="line-3366"> boolean result = super.swapPipelineWithNull(segments);</span>
<span class="source-line-no">3367</span><span id="line-3367"> if (Thread.currentThread().getName().equals(TAKE_SNAPSHOT_THREAD_NAME)) {</span>
<span class="source-line-no">3368</span><span id="line-3368"> int currentCount = swapPipelineWithNullCounter.get();</span>
<span class="source-line-no">3369</span><span id="line-3369"> if (currentCount &lt;= 1) {</span>
<span class="source-line-no">3370</span><span id="line-3370"> assertTrue(!result);</span>
<span class="source-line-no">3371</span><span id="line-3371"> }</span>
<span class="source-line-no">3372</span><span id="line-3372"> if (currentCount == 2) {</span>
<span class="source-line-no">3373</span><span id="line-3373"> assertTrue(result);</span>
<span class="source-line-no">3374</span><span id="line-3374"> }</span>
<span class="source-line-no">3375</span><span id="line-3375"> }</span>
<span class="source-line-no">3376</span><span id="line-3376"> return result;</span>
<span class="source-line-no">3377</span><span id="line-3377"></span>
<span class="source-line-no">3378</span><span id="line-3378"> }</span>
<span class="source-line-no">3379</span><span id="line-3379"></span>
<span class="source-line-no">3380</span><span id="line-3380"> @Override</span>
<span class="source-line-no">3381</span><span id="line-3381"> public void flattenOneSegment(long requesterVersion, Action action) {</span>
<span class="source-line-no">3382</span><span id="line-3382"> int currentCount = flattenOneSegmentCounter.incrementAndGet();</span>
<span class="source-line-no">3383</span><span id="line-3383"> if (currentCount &lt;= 1) {</span>
<span class="source-line-no">3384</span><span id="line-3384"> try {</span>
<span class="source-line-no">3385</span><span id="line-3385"> /**</span>
<span class="source-line-no">3386</span><span id="line-3386"> * {@link CompactingMemStore#snapshot} could start.</span>
<span class="source-line-no">3387</span><span id="line-3387"> */</span>
<span class="source-line-no">3388</span><span id="line-3388"> snapShotStartCyclicCyclicBarrier.await();</span>
<span class="source-line-no">3389</span><span id="line-3389"> flattenOneSegmentPreCyclicBarrier.await();</span>
<span class="source-line-no">3390</span><span id="line-3390"> } catch (Throwable e) {</span>
<span class="source-line-no">3391</span><span id="line-3391"> throw new RuntimeException(e);</span>
<span class="source-line-no">3392</span><span id="line-3392"> }</span>
<span class="source-line-no">3393</span><span id="line-3393"> }</span>
<span class="source-line-no">3394</span><span id="line-3394"> super.flattenOneSegment(requesterVersion, action);</span>
<span class="source-line-no">3395</span><span id="line-3395"> if (currentCount &lt;= 1) {</span>
<span class="source-line-no">3396</span><span id="line-3396"> try {</span>
<span class="source-line-no">3397</span><span id="line-3397"> flattenOneSegmentPostCyclicBarrier.await();</span>
<span class="source-line-no">3398</span><span id="line-3398"> } catch (Throwable e) {</span>
<span class="source-line-no">3399</span><span id="line-3399"> throw new RuntimeException(e);</span>
<span class="source-line-no">3400</span><span id="line-3400"> }</span>
<span class="source-line-no">3401</span><span id="line-3401"> }</span>
<span class="source-line-no">3402</span><span id="line-3402"> }</span>
<span class="source-line-no">3403</span><span id="line-3403"></span>
<span class="source-line-no">3404</span><span id="line-3404"> @Override</span>
<span class="source-line-no">3405</span><span id="line-3405"> protected boolean setInMemoryCompactionFlag() {</span>
<span class="source-line-no">3406</span><span id="line-3406"> boolean result = super.setInMemoryCompactionFlag();</span>
<span class="source-line-no">3407</span><span id="line-3407"> assertTrue(result);</span>
<span class="source-line-no">3408</span><span id="line-3408"> setInMemoryCompactionFlagCounter.incrementAndGet();</span>
<span class="source-line-no">3409</span><span id="line-3409"> return result;</span>
<span class="source-line-no">3410</span><span id="line-3410"> }</span>
<span class="source-line-no">3411</span><span id="line-3411"></span>
<span class="source-line-no">3412</span><span id="line-3412"> @Override</span>
<span class="source-line-no">3413</span><span id="line-3413"> void inMemoryCompaction() {</span>
<span class="source-line-no">3414</span><span id="line-3414"> try {</span>
<span class="source-line-no">3415</span><span id="line-3415"> super.inMemoryCompaction();</span>
<span class="source-line-no">3416</span><span id="line-3416"> } finally {</span>
<span class="source-line-no">3417</span><span id="line-3417"> try {</span>
<span class="source-line-no">3418</span><span id="line-3418"> inMemoryCompactionEndCyclicBarrier.await();</span>
<span class="source-line-no">3419</span><span id="line-3419"> } catch (Throwable e) {</span>
<span class="source-line-no">3420</span><span id="line-3420"> throw new RuntimeException(e);</span>
<span class="source-line-no">3421</span><span id="line-3421"> }</span>
<span class="source-line-no">3422</span><span id="line-3422"></span>
<span class="source-line-no">3423</span><span id="line-3423"> }</span>
<span class="source-line-no">3424</span><span id="line-3424"> }</span>
<span class="source-line-no">3425</span><span id="line-3425"></span>
<span class="source-line-no">3426</span><span id="line-3426"> }</span>
<span class="source-line-no">3427</span><span id="line-3427"></span>
<span class="source-line-no">3428</span><span id="line-3428"> public static class MyCompactingMemStore5 extends CompactingMemStore {</span>
<span class="source-line-no">3429</span><span id="line-3429"> private static final String TAKE_SNAPSHOT_THREAD_NAME = "takeSnapShotThread";</span>
<span class="source-line-no">3430</span><span id="line-3430"> private static final String WRITE_AGAIN_THREAD_NAME = "writeAgainThread";</span>
<span class="source-line-no">3431</span><span id="line-3431"> /**</span>
<span class="source-line-no">3432</span><span id="line-3432"> * {@link CompactingMemStore#flattenOneSegment} must execute after</span>
<span class="source-line-no">3433</span><span id="line-3433"> * {@link CompactingMemStore#getImmutableSegments}</span>
<span class="source-line-no">3434</span><span id="line-3434"> */</span>
<span class="source-line-no">3435</span><span id="line-3435"> private final CyclicBarrier flattenOneSegmentPreCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3436</span><span id="line-3436"> /**</span>
<span class="source-line-no">3437</span><span id="line-3437"> * Only after {@link CompactingMemStore#flattenOneSegment} completed,</span>
<span class="source-line-no">3438</span><span id="line-3438"> * {@link CompactingMemStore#swapPipelineWithNull} could execute.</span>
<span class="source-line-no">3439</span><span id="line-3439"> */</span>
<span class="source-line-no">3440</span><span id="line-3440"> private final CyclicBarrier flattenOneSegmentPostCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3441</span><span id="line-3441"> /**</span>
<span class="source-line-no">3442</span><span id="line-3442"> * Only the in memory compact thread enters {@link CompactingMemStore#flattenOneSegment},the</span>
<span class="source-line-no">3443</span><span id="line-3443"> * snapshot thread starts {@link CompactingMemStore#snapshot},because</span>
<span class="source-line-no">3444</span><span id="line-3444"> * {@link CompactingMemStore#snapshot} would invoke {@link CompactingMemStore#stopCompaction}.</span>
<span class="source-line-no">3445</span><span id="line-3445"> */</span>
<span class="source-line-no">3446</span><span id="line-3446"> private final CyclicBarrier snapShotStartCyclicCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3447</span><span id="line-3447"> /**</span>
<span class="source-line-no">3448</span><span id="line-3448"> * To wait for {@link CompactingMemStore.InMemoryCompactionRunnable} stopping.</span>
<span class="source-line-no">3449</span><span id="line-3449"> */</span>
<span class="source-line-no">3450</span><span id="line-3450"> private final CyclicBarrier inMemoryCompactionEndCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3451</span><span id="line-3451"> private final AtomicInteger getImmutableSegmentsListCounter = new AtomicInteger(0);</span>
<span class="source-line-no">3452</span><span id="line-3452"> private final AtomicInteger swapPipelineWithNullCounter = new AtomicInteger(0);</span>
<span class="source-line-no">3453</span><span id="line-3453"> private final AtomicInteger flattenOneSegmentCounter = new AtomicInteger(0);</span>
<span class="source-line-no">3454</span><span id="line-3454"> private final AtomicInteger setInMemoryCompactionFlagCounter = new AtomicInteger(0);</span>
<span class="source-line-no">3455</span><span id="line-3455"> /**</span>
<span class="source-line-no">3456</span><span id="line-3456"> * Only the snapshot thread retry {@link CompactingMemStore#swapPipelineWithNull}, writeAgain</span>
<span class="source-line-no">3457</span><span id="line-3457"> * thread could start.</span>
<span class="source-line-no">3458</span><span id="line-3458"> */</span>
<span class="source-line-no">3459</span><span id="line-3459"> private final CyclicBarrier writeMemStoreAgainStartCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3460</span><span id="line-3460"> /**</span>
<span class="source-line-no">3461</span><span id="line-3461"> * This is used for snapshot thread,writeAgain thread and in memory compact thread. Only the</span>
<span class="source-line-no">3462</span><span id="line-3462"> * writeAgain thread completes, {@link CompactingMemStore#swapPipelineWithNull} would</span>
<span class="source-line-no">3463</span><span id="line-3463"> * execute,and in memory compact thread would exit,because we expect that in memory compact</span>
<span class="source-line-no">3464</span><span id="line-3464"> * executing only once.</span>
<span class="source-line-no">3465</span><span id="line-3465"> */</span>
<span class="source-line-no">3466</span><span id="line-3466"> private final CyclicBarrier writeMemStoreAgainEndCyclicBarrier = new CyclicBarrier(3);</span>
<span class="source-line-no">3467</span><span id="line-3467"></span>
<span class="source-line-no">3468</span><span id="line-3468"> public MyCompactingMemStore5(Configuration conf, CellComparatorImpl cellComparator,</span>
<span class="source-line-no">3469</span><span id="line-3469"> HStore store, RegionServicesForStores regionServices, MemoryCompactionPolicy compactionPolicy)</span>
<span class="source-line-no">3470</span><span id="line-3470"> throws IOException {</span>
<span class="source-line-no">3471</span><span id="line-3471"> super(conf, cellComparator, store, regionServices, compactionPolicy);</span>
<span class="source-line-no">3472</span><span id="line-3472"> }</span>
<span class="source-line-no">3473</span><span id="line-3473"></span>
<span class="source-line-no">3474</span><span id="line-3474"> @Override</span>
<span class="source-line-no">3475</span><span id="line-3475"> public VersionedSegmentsList getImmutableSegments() {</span>
<span class="source-line-no">3476</span><span id="line-3476"> VersionedSegmentsList result = super.getImmutableSegments();</span>
<span class="source-line-no">3477</span><span id="line-3477"> if (Thread.currentThread().getName().equals(TAKE_SNAPSHOT_THREAD_NAME)) {</span>
<span class="source-line-no">3478</span><span id="line-3478"> int currentCount = getImmutableSegmentsListCounter.incrementAndGet();</span>
<span class="source-line-no">3479</span><span id="line-3479"> if (currentCount &lt;= 1) {</span>
<span class="source-line-no">3480</span><span id="line-3480"> try {</span>
<span class="source-line-no">3481</span><span id="line-3481"> flattenOneSegmentPreCyclicBarrier.await();</span>
<span class="source-line-no">3482</span><span id="line-3482"> } catch (Throwable e) {</span>
<span class="source-line-no">3483</span><span id="line-3483"> throw new RuntimeException(e);</span>
<span class="source-line-no">3484</span><span id="line-3484"> }</span>
<span class="source-line-no">3485</span><span id="line-3485"> }</span>
<span class="source-line-no">3486</span><span id="line-3486"></span>
<span class="source-line-no">3487</span><span id="line-3487"> }</span>
<span class="source-line-no">3488</span><span id="line-3488"></span>
<span class="source-line-no">3489</span><span id="line-3489"> return result;</span>
<span class="source-line-no">3490</span><span id="line-3490"> }</span>
<span class="source-line-no">3491</span><span id="line-3491"></span>
<span class="source-line-no">3492</span><span id="line-3492"> @Override</span>
<span class="source-line-no">3493</span><span id="line-3493"> protected boolean swapPipelineWithNull(VersionedSegmentsList segments) {</span>
<span class="source-line-no">3494</span><span id="line-3494"> if (Thread.currentThread().getName().equals(TAKE_SNAPSHOT_THREAD_NAME)) {</span>
<span class="source-line-no">3495</span><span id="line-3495"> int currentCount = swapPipelineWithNullCounter.incrementAndGet();</span>
<span class="source-line-no">3496</span><span id="line-3496"> if (currentCount &lt;= 1) {</span>
<span class="source-line-no">3497</span><span id="line-3497"> try {</span>
<span class="source-line-no">3498</span><span id="line-3498"> flattenOneSegmentPostCyclicBarrier.await();</span>
<span class="source-line-no">3499</span><span id="line-3499"> } catch (Throwable e) {</span>
<span class="source-line-no">3500</span><span id="line-3500"> throw new RuntimeException(e);</span>
<span class="source-line-no">3501</span><span id="line-3501"> }</span>
<span class="source-line-no">3502</span><span id="line-3502"> }</span>
<span class="source-line-no">3503</span><span id="line-3503"></span>
<span class="source-line-no">3504</span><span id="line-3504"> if (currentCount == 2) {</span>
<span class="source-line-no">3505</span><span id="line-3505"> try {</span>
<span class="source-line-no">3506</span><span id="line-3506"> /**</span>
<span class="source-line-no">3507</span><span id="line-3507"> * Only the snapshot thread retry {@link CompactingMemStore#swapPipelineWithNull},</span>
<span class="source-line-no">3508</span><span id="line-3508"> * writeAgain thread could start.</span>
<span class="source-line-no">3509</span><span id="line-3509"> */</span>
<span class="source-line-no">3510</span><span id="line-3510"> writeMemStoreAgainStartCyclicBarrier.await();</span>
<span class="source-line-no">3511</span><span id="line-3511"> /**</span>
<span class="source-line-no">3512</span><span id="line-3512"> * Only the writeAgain thread completes, retry</span>
<span class="source-line-no">3513</span><span id="line-3513"> * {@link CompactingMemStore#swapPipelineWithNull} would execute.</span>
<span class="source-line-no">3514</span><span id="line-3514"> */</span>
<span class="source-line-no">3515</span><span id="line-3515"> writeMemStoreAgainEndCyclicBarrier.await();</span>
<span class="source-line-no">3516</span><span id="line-3516"> } catch (Throwable e) {</span>
<span class="source-line-no">3517</span><span id="line-3517"> throw new RuntimeException(e);</span>
<span class="source-line-no">3518</span><span id="line-3518"> }</span>
<span class="source-line-no">3519</span><span id="line-3519"> }</span>
<span class="source-line-no">3520</span><span id="line-3520"></span>
<span class="source-line-no">3521</span><span id="line-3521"> }</span>
<span class="source-line-no">3522</span><span id="line-3522"> boolean result = super.swapPipelineWithNull(segments);</span>
<span class="source-line-no">3523</span><span id="line-3523"> if (Thread.currentThread().getName().equals(TAKE_SNAPSHOT_THREAD_NAME)) {</span>
<span class="source-line-no">3524</span><span id="line-3524"> int currentCount = swapPipelineWithNullCounter.get();</span>
<span class="source-line-no">3525</span><span id="line-3525"> if (currentCount &lt;= 1) {</span>
<span class="source-line-no">3526</span><span id="line-3526"> assertTrue(!result);</span>
<span class="source-line-no">3527</span><span id="line-3527"> }</span>
<span class="source-line-no">3528</span><span id="line-3528"> if (currentCount == 2) {</span>
<span class="source-line-no">3529</span><span id="line-3529"> assertTrue(result);</span>
<span class="source-line-no">3530</span><span id="line-3530"> }</span>
<span class="source-line-no">3531</span><span id="line-3531"> }</span>
<span class="source-line-no">3532</span><span id="line-3532"> return result;</span>
<span class="source-line-no">3533</span><span id="line-3533"></span>
<span class="source-line-no">3534</span><span id="line-3534"> }</span>
<span class="source-line-no">3535</span><span id="line-3535"></span>
<span class="source-line-no">3536</span><span id="line-3536"> @Override</span>
<span class="source-line-no">3537</span><span id="line-3537"> public void flattenOneSegment(long requesterVersion, Action action) {</span>
<span class="source-line-no">3538</span><span id="line-3538"> int currentCount = flattenOneSegmentCounter.incrementAndGet();</span>
<span class="source-line-no">3539</span><span id="line-3539"> if (currentCount &lt;= 1) {</span>
<span class="source-line-no">3540</span><span id="line-3540"> try {</span>
<span class="source-line-no">3541</span><span id="line-3541"> /**</span>
<span class="source-line-no">3542</span><span id="line-3542"> * {@link CompactingMemStore#snapshot} could start.</span>
<span class="source-line-no">3543</span><span id="line-3543"> */</span>
<span class="source-line-no">3544</span><span id="line-3544"> snapShotStartCyclicCyclicBarrier.await();</span>
<span class="source-line-no">3545</span><span id="line-3545"> flattenOneSegmentPreCyclicBarrier.await();</span>
<span class="source-line-no">3546</span><span id="line-3546"> } catch (Throwable e) {</span>
<span class="source-line-no">3547</span><span id="line-3547"> throw new RuntimeException(e);</span>
<span class="source-line-no">3548</span><span id="line-3548"> }</span>
<span class="source-line-no">3549</span><span id="line-3549"> }</span>
<span class="source-line-no">3550</span><span id="line-3550"> super.flattenOneSegment(requesterVersion, action);</span>
<span class="source-line-no">3551</span><span id="line-3551"> if (currentCount &lt;= 1) {</span>
<span class="source-line-no">3552</span><span id="line-3552"> try {</span>
<span class="source-line-no">3553</span><span id="line-3553"> flattenOneSegmentPostCyclicBarrier.await();</span>
<span class="source-line-no">3554</span><span id="line-3554"> /**</span>
<span class="source-line-no">3555</span><span id="line-3555"> * Only the writeAgain thread completes, in memory compact thread would exit,because we</span>
<span class="source-line-no">3556</span><span id="line-3556"> * expect that in memory compact executing only once.</span>
<span class="source-line-no">3557</span><span id="line-3557"> */</span>
<span class="source-line-no">3558</span><span id="line-3558"> writeMemStoreAgainEndCyclicBarrier.await();</span>
<span class="source-line-no">3559</span><span id="line-3559"> } catch (Throwable e) {</span>
<span class="source-line-no">3560</span><span id="line-3560"> throw new RuntimeException(e);</span>
<span class="source-line-no">3561</span><span id="line-3561"> }</span>
<span class="source-line-no">3562</span><span id="line-3562"></span>
<span class="source-line-no">3563</span><span id="line-3563"> }</span>
<span class="source-line-no">3564</span><span id="line-3564"> }</span>
<span class="source-line-no">3565</span><span id="line-3565"></span>
<span class="source-line-no">3566</span><span id="line-3566"> @Override</span>
<span class="source-line-no">3567</span><span id="line-3567"> protected boolean setInMemoryCompactionFlag() {</span>
<span class="source-line-no">3568</span><span id="line-3568"> boolean result = super.setInMemoryCompactionFlag();</span>
<span class="source-line-no">3569</span><span id="line-3569"> int count = setInMemoryCompactionFlagCounter.incrementAndGet();</span>
<span class="source-line-no">3570</span><span id="line-3570"> if (count &lt;= 1) {</span>
<span class="source-line-no">3571</span><span id="line-3571"> assertTrue(result);</span>
<span class="source-line-no">3572</span><span id="line-3572"> }</span>
<span class="source-line-no">3573</span><span id="line-3573"> if (count == 2) {</span>
<span class="source-line-no">3574</span><span id="line-3574"> assertTrue(!result);</span>
<span class="source-line-no">3575</span><span id="line-3575"> }</span>
<span class="source-line-no">3576</span><span id="line-3576"> return result;</span>
<span class="source-line-no">3577</span><span id="line-3577"> }</span>
<span class="source-line-no">3578</span><span id="line-3578"></span>
<span class="source-line-no">3579</span><span id="line-3579"> @Override</span>
<span class="source-line-no">3580</span><span id="line-3580"> void inMemoryCompaction() {</span>
<span class="source-line-no">3581</span><span id="line-3581"> try {</span>
<span class="source-line-no">3582</span><span id="line-3582"> super.inMemoryCompaction();</span>
<span class="source-line-no">3583</span><span id="line-3583"> } finally {</span>
<span class="source-line-no">3584</span><span id="line-3584"> try {</span>
<span class="source-line-no">3585</span><span id="line-3585"> inMemoryCompactionEndCyclicBarrier.await();</span>
<span class="source-line-no">3586</span><span id="line-3586"> } catch (Throwable e) {</span>
<span class="source-line-no">3587</span><span id="line-3587"> throw new RuntimeException(e);</span>
<span class="source-line-no">3588</span><span id="line-3588"> }</span>
<span class="source-line-no">3589</span><span id="line-3589"></span>
<span class="source-line-no">3590</span><span id="line-3590"> }</span>
<span class="source-line-no">3591</span><span id="line-3591"> }</span>
<span class="source-line-no">3592</span><span id="line-3592"> }</span>
<span class="source-line-no">3593</span><span id="line-3593"></span>
<span class="source-line-no">3594</span><span id="line-3594"> public static class MyCompactingMemStore6 extends CompactingMemStore {</span>
<span class="source-line-no">3595</span><span id="line-3595"> private final CyclicBarrier inMemoryCompactionEndCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3596</span><span id="line-3596"></span>
<span class="source-line-no">3597</span><span id="line-3597"> public MyCompactingMemStore6(Configuration conf, CellComparatorImpl cellComparator,</span>
<span class="source-line-no">3598</span><span id="line-3598"> HStore store, RegionServicesForStores regionServices, MemoryCompactionPolicy compactionPolicy)</span>
<span class="source-line-no">3599</span><span id="line-3599"> throws IOException {</span>
<span class="source-line-no">3600</span><span id="line-3600"> super(conf, cellComparator, store, regionServices, compactionPolicy);</span>
<span class="source-line-no">3601</span><span id="line-3601"> }</span>
<span class="source-line-no">3602</span><span id="line-3602"></span>
<span class="source-line-no">3603</span><span id="line-3603"> @Override</span>
<span class="source-line-no">3604</span><span id="line-3604"> void inMemoryCompaction() {</span>
<span class="source-line-no">3605</span><span id="line-3605"> try {</span>
<span class="source-line-no">3606</span><span id="line-3606"> super.inMemoryCompaction();</span>
<span class="source-line-no">3607</span><span id="line-3607"> } finally {</span>
<span class="source-line-no">3608</span><span id="line-3608"> try {</span>
<span class="source-line-no">3609</span><span id="line-3609"> inMemoryCompactionEndCyclicBarrier.await();</span>
<span class="source-line-no">3610</span><span id="line-3610"> } catch (Throwable e) {</span>
<span class="source-line-no">3611</span><span id="line-3611"> throw new RuntimeException(e);</span>
<span class="source-line-no">3612</span><span id="line-3612"> }</span>
<span class="source-line-no">3613</span><span id="line-3613"></span>
<span class="source-line-no">3614</span><span id="line-3614"> }</span>
<span class="source-line-no">3615</span><span id="line-3615"> }</span>
<span class="source-line-no">3616</span><span id="line-3616"> }</span>
<span class="source-line-no">3617</span><span id="line-3617"></span>
<span class="source-line-no">3618</span><span id="line-3618"> public static class MyDefaultMemStore extends DefaultMemStore {</span>
<span class="source-line-no">3619</span><span id="line-3619"> private static final String GET_SCANNER_THREAD_NAME = "getScannerMyThread";</span>
<span class="source-line-no">3620</span><span id="line-3620"> private static final String FLUSH_THREAD_NAME = "flushMyThread";</span>
<span class="source-line-no">3621</span><span id="line-3621"> /**</span>
<span class="source-line-no">3622</span><span id="line-3622"> * Only when flush thread enters {@link DefaultMemStore#doClearSnapShot}, getScanner thread</span>
<span class="source-line-no">3623</span><span id="line-3623"> * could start.</span>
<span class="source-line-no">3624</span><span id="line-3624"> */</span>
<span class="source-line-no">3625</span><span id="line-3625"> private final CyclicBarrier getScannerCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3626</span><span id="line-3626"> /**</span>
<span class="source-line-no">3627</span><span id="line-3627"> * Used by getScanner thread notifies flush thread {@link DefaultMemStore#getSnapshotSegments}</span>
<span class="source-line-no">3628</span><span id="line-3628"> * completed, {@link DefaultMemStore#doClearSnapShot} could continue.</span>
<span class="source-line-no">3629</span><span id="line-3629"> */</span>
<span class="source-line-no">3630</span><span id="line-3630"> private final CyclicBarrier preClearSnapShotCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3631</span><span id="line-3631"> /**</span>
<span class="source-line-no">3632</span><span id="line-3632"> * Used by flush thread notifies getScanner thread {@link DefaultMemStore#doClearSnapShot}</span>
<span class="source-line-no">3633</span><span id="line-3633"> * completed, {@link DefaultMemStore#getScanners} could continue.</span>
<span class="source-line-no">3634</span><span id="line-3634"> */</span>
<span class="source-line-no">3635</span><span id="line-3635"> private final CyclicBarrier postClearSnapShotCyclicBarrier = new CyclicBarrier(2);</span>
<span class="source-line-no">3636</span><span id="line-3636"> private final AtomicInteger getSnapshotSegmentsCounter = new AtomicInteger(0);</span>
<span class="source-line-no">3637</span><span id="line-3637"> private final AtomicInteger clearSnapshotCounter = new AtomicInteger(0);</span>
<span class="source-line-no">3638</span><span id="line-3638"> private volatile boolean shouldWait = true;</span>
<span class="source-line-no">3639</span><span id="line-3639"> private volatile HStore store = null;</span>
<span class="source-line-no">3640</span><span id="line-3640"></span>
<span class="source-line-no">3641</span><span id="line-3641"> public MyDefaultMemStore(Configuration conf, CellComparator cellComparator,</span>
<span class="source-line-no">3642</span><span id="line-3642"> RegionServicesForStores regionServices) throws IOException {</span>
<span class="source-line-no">3643</span><span id="line-3643"> super(conf, cellComparator, regionServices);</span>
<span class="source-line-no">3644</span><span id="line-3644"> }</span>
<span class="source-line-no">3645</span><span id="line-3645"></span>
<span class="source-line-no">3646</span><span id="line-3646"> @Override</span>
<span class="source-line-no">3647</span><span id="line-3647"> protected List&lt;Segment&gt; getSnapshotSegments() {</span>
<span class="source-line-no">3648</span><span id="line-3648"></span>
<span class="source-line-no">3649</span><span id="line-3649"> List&lt;Segment&gt; result = super.getSnapshotSegments();</span>
<span class="source-line-no">3650</span><span id="line-3650"></span>
<span class="source-line-no">3651</span><span id="line-3651"> if (Thread.currentThread().getName().equals(GET_SCANNER_THREAD_NAME)) {</span>
<span class="source-line-no">3652</span><span id="line-3652"> int currentCount = getSnapshotSegmentsCounter.incrementAndGet();</span>
<span class="source-line-no">3653</span><span id="line-3653"> if (currentCount == 1) {</span>
<span class="source-line-no">3654</span><span id="line-3654"> if (this.shouldWait) {</span>
<span class="source-line-no">3655</span><span id="line-3655"> try {</span>
<span class="source-line-no">3656</span><span id="line-3656"> /**</span>
<span class="source-line-no">3657</span><span id="line-3657"> * Notify flush thread {@link DefaultMemStore#getSnapshotSegments} completed,</span>
<span class="source-line-no">3658</span><span id="line-3658"> * {@link DefaultMemStore#doClearSnapShot} could continue.</span>
<span class="source-line-no">3659</span><span id="line-3659"> */</span>
<span class="source-line-no">3660</span><span id="line-3660"> preClearSnapShotCyclicBarrier.await();</span>
<span class="source-line-no">3661</span><span id="line-3661"> /**</span>
<span class="source-line-no">3662</span><span id="line-3662"> * Wait for {@link DefaultMemStore#doClearSnapShot} completed.</span>
<span class="source-line-no">3663</span><span id="line-3663"> */</span>
<span class="source-line-no">3664</span><span id="line-3664"> postClearSnapShotCyclicBarrier.await();</span>
<span class="source-line-no">3665</span><span id="line-3665"></span>
<span class="source-line-no">3666</span><span id="line-3666"> } catch (Throwable e) {</span>
<span class="source-line-no">3667</span><span id="line-3667"> throw new RuntimeException(e);</span>
<span class="source-line-no">3668</span><span id="line-3668"> }</span>
<span class="source-line-no">3669</span><span id="line-3669"> }</span>
<span class="source-line-no">3670</span><span id="line-3670"> }</span>
<span class="source-line-no">3671</span><span id="line-3671"> }</span>
<span class="source-line-no">3672</span><span id="line-3672"> return result;</span>
<span class="source-line-no">3673</span><span id="line-3673"> }</span>
<span class="source-line-no">3674</span><span id="line-3674"></span>
<span class="source-line-no">3675</span><span id="line-3675"> @Override</span>
<span class="source-line-no">3676</span><span id="line-3676"> protected void doClearSnapShot() {</span>
<span class="source-line-no">3677</span><span id="line-3677"> if (Thread.currentThread().getName().equals(FLUSH_THREAD_NAME)) {</span>
<span class="source-line-no">3678</span><span id="line-3678"> int currentCount = clearSnapshotCounter.incrementAndGet();</span>
<span class="source-line-no">3679</span><span id="line-3679"> if (currentCount == 1) {</span>
<span class="source-line-no">3680</span><span id="line-3680"> try {</span>
<span class="source-line-no">3681</span><span id="line-3681"> if (</span>
<span class="source-line-no">3682</span><span id="line-3682"> ((ReentrantReadWriteLock) store.getStoreEngine().getLock())</span>
<span class="source-line-no">3683</span><span id="line-3683"> .isWriteLockedByCurrentThread()</span>
<span class="source-line-no">3684</span><span id="line-3684"> ) {</span>
<span class="source-line-no">3685</span><span id="line-3685"> shouldWait = false;</span>
<span class="source-line-no">3686</span><span id="line-3686"> }</span>
<span class="source-line-no">3687</span><span id="line-3687"> /**</span>
<span class="source-line-no">3688</span><span id="line-3688"> * Only when flush thread enters {@link DefaultMemStore#doClearSnapShot}, getScanner</span>
<span class="source-line-no">3689</span><span id="line-3689"> * thread could start.</span>
<span class="source-line-no">3690</span><span id="line-3690"> */</span>
<span class="source-line-no">3691</span><span id="line-3691"> getScannerCyclicBarrier.await();</span>
<span class="source-line-no">3692</span><span id="line-3692"></span>
<span class="source-line-no">3693</span><span id="line-3693"> if (shouldWait) {</span>
<span class="source-line-no">3694</span><span id="line-3694"> /**</span>
<span class="source-line-no">3695</span><span id="line-3695"> * Wait for {@link DefaultMemStore#getSnapshotSegments} completed.</span>
<span class="source-line-no">3696</span><span id="line-3696"> */</span>
<span class="source-line-no">3697</span><span id="line-3697"> preClearSnapShotCyclicBarrier.await();</span>
<span class="source-line-no">3698</span><span id="line-3698"> }</span>
<span class="source-line-no">3699</span><span id="line-3699"> } catch (Throwable e) {</span>
<span class="source-line-no">3700</span><span id="line-3700"> throw new RuntimeException(e);</span>
<span class="source-line-no">3701</span><span id="line-3701"> }</span>
<span class="source-line-no">3702</span><span id="line-3702"> }</span>
<span class="source-line-no">3703</span><span id="line-3703"> }</span>
<span class="source-line-no">3704</span><span id="line-3704"> super.doClearSnapShot();</span>
<span class="source-line-no">3705</span><span id="line-3705"></span>
<span class="source-line-no">3706</span><span id="line-3706"> if (Thread.currentThread().getName().equals(FLUSH_THREAD_NAME)) {</span>
<span class="source-line-no">3707</span><span id="line-3707"> int currentCount = clearSnapshotCounter.get();</span>
<span class="source-line-no">3708</span><span id="line-3708"> if (currentCount == 1) {</span>
<span class="source-line-no">3709</span><span id="line-3709"> if (shouldWait) {</span>
<span class="source-line-no">3710</span><span id="line-3710"> try {</span>
<span class="source-line-no">3711</span><span id="line-3711"> /**</span>
<span class="source-line-no">3712</span><span id="line-3712"> * Notify getScanner thread {@link DefaultMemStore#doClearSnapShot} completed,</span>
<span class="source-line-no">3713</span><span id="line-3713"> * {@link DefaultMemStore#getScanners} could continue.</span>
<span class="source-line-no">3714</span><span id="line-3714"> */</span>
<span class="source-line-no">3715</span><span id="line-3715"> postClearSnapShotCyclicBarrier.await();</span>
<span class="source-line-no">3716</span><span id="line-3716"> } catch (Throwable e) {</span>
<span class="source-line-no">3717</span><span id="line-3717"> throw new RuntimeException(e);</span>
<span class="source-line-no">3718</span><span id="line-3718"> }</span>
<span class="source-line-no">3719</span><span id="line-3719"> }</span>
<span class="source-line-no">3720</span><span id="line-3720"> }</span>
<span class="source-line-no">3721</span><span id="line-3721"> }</span>
<span class="source-line-no">3722</span><span id="line-3722"> }</span>
<span class="source-line-no">3723</span><span id="line-3723"> }</span>
<span class="source-line-no">3724</span><span id="line-3724">}</span>
</pre>
</div>
</main>
</body>
</html>