| <!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.mapreduce, class: TestHFileOutputFormat2, class: RandomKVGeneratingMapper"> |
| <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.mapreduce;</span> |
| <span class="source-line-no">019</span><span id="line-19"></span> |
| <span class="source-line-no">020</span><span id="line-20">import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY;</span> |
| <span class="source-line-no">021</span><span id="line-21">import static org.junit.Assert.assertEquals;</span> |
| <span class="source-line-no">022</span><span id="line-22">import static org.junit.Assert.assertFalse;</span> |
| <span class="source-line-no">023</span><span id="line-23">import static org.junit.Assert.assertNotNull;</span> |
| <span class="source-line-no">024</span><span id="line-24">import static org.junit.Assert.assertNotSame;</span> |
| <span class="source-line-no">025</span><span id="line-25">import static org.junit.Assert.assertTrue;</span> |
| <span class="source-line-no">026</span><span id="line-26">import static org.junit.Assert.fail;</span> |
| <span class="source-line-no">027</span><span id="line-27">import static org.mockito.Mockito.verify;</span> |
| <span class="source-line-no">028</span><span id="line-28"></span> |
| <span class="source-line-no">029</span><span id="line-29">import java.io.IOException;</span> |
| <span class="source-line-no">030</span><span id="line-30">import java.lang.reflect.Field;</span> |
| <span class="source-line-no">031</span><span id="line-31">import java.security.PrivilegedAction;</span> |
| <span class="source-line-no">032</span><span id="line-32">import java.util.ArrayList;</span> |
| <span class="source-line-no">033</span><span id="line-33">import java.util.Arrays;</span> |
| <span class="source-line-no">034</span><span id="line-34">import java.util.HashMap;</span> |
| <span class="source-line-no">035</span><span id="line-35">import java.util.LinkedList;</span> |
| <span class="source-line-no">036</span><span id="line-36">import java.util.List;</span> |
| <span class="source-line-no">037</span><span id="line-37">import java.util.Map;</span> |
| <span class="source-line-no">038</span><span id="line-38">import java.util.Map.Entry;</span> |
| <span class="source-line-no">039</span><span id="line-39">import java.util.Random;</span> |
| <span class="source-line-no">040</span><span id="line-40">import java.util.Set;</span> |
| <span class="source-line-no">041</span><span id="line-41">import java.util.UUID;</span> |
| <span class="source-line-no">042</span><span id="line-42">import java.util.concurrent.Callable;</span> |
| <span class="source-line-no">043</span><span id="line-43">import java.util.concurrent.ConcurrentHashMap;</span> |
| <span class="source-line-no">044</span><span id="line-44">import java.util.concurrent.CopyOnWriteArrayList;</span> |
| <span class="source-line-no">045</span><span id="line-45">import java.util.concurrent.ExecutorService;</span> |
| <span class="source-line-no">046</span><span id="line-46">import java.util.concurrent.ThreadLocalRandom;</span> |
| <span class="source-line-no">047</span><span id="line-47">import java.util.stream.Collectors;</span> |
| <span class="source-line-no">048</span><span id="line-48">import java.util.stream.Stream;</span> |
| <span class="source-line-no">049</span><span id="line-49">import org.apache.hadoop.conf.Configuration;</span> |
| <span class="source-line-no">050</span><span id="line-50">import org.apache.hadoop.fs.FileStatus;</span> |
| <span class="source-line-no">051</span><span id="line-51">import org.apache.hadoop.fs.FileSystem;</span> |
| <span class="source-line-no">052</span><span id="line-52">import org.apache.hadoop.fs.LocatedFileStatus;</span> |
| <span class="source-line-no">053</span><span id="line-53">import org.apache.hadoop.fs.Path;</span> |
| <span class="source-line-no">054</span><span id="line-54">import org.apache.hadoop.fs.RemoteIterator;</span> |
| <span class="source-line-no">055</span><span id="line-55">import org.apache.hadoop.hbase.ArrayBackedTag;</span> |
| <span class="source-line-no">056</span><span id="line-56">import org.apache.hadoop.hbase.Cell;</span> |
| <span class="source-line-no">057</span><span id="line-57">import org.apache.hadoop.hbase.CellUtil;</span> |
| <span class="source-line-no">058</span><span id="line-58">import org.apache.hadoop.hbase.CompatibilitySingletonFactory;</span> |
| <span class="source-line-no">059</span><span id="line-59">import org.apache.hadoop.hbase.ExtendedCell;</span> |
| <span class="source-line-no">060</span><span id="line-60">import org.apache.hadoop.hbase.HBaseClassTestRule;</span> |
| <span class="source-line-no">061</span><span id="line-61">import org.apache.hadoop.hbase.HBaseConfiguration;</span> |
| <span class="source-line-no">062</span><span id="line-62">import org.apache.hadoop.hbase.HBaseTestingUtil;</span> |
| <span class="source-line-no">063</span><span id="line-63">import org.apache.hadoop.hbase.HConstants;</span> |
| <span class="source-line-no">064</span><span id="line-64">import org.apache.hadoop.hbase.HDFSBlocksDistribution;</span> |
| <span class="source-line-no">065</span><span id="line-65">import org.apache.hadoop.hbase.HadoopShims;</span> |
| <span class="source-line-no">066</span><span id="line-66">import org.apache.hadoop.hbase.KeyValue;</span> |
| <span class="source-line-no">067</span><span id="line-67">import org.apache.hadoop.hbase.PerformanceEvaluation;</span> |
| <span class="source-line-no">068</span><span id="line-68">import org.apache.hadoop.hbase.PrivateCellUtil;</span> |
| <span class="source-line-no">069</span><span id="line-69">import org.apache.hadoop.hbase.ServerName;</span> |
| <span class="source-line-no">070</span><span id="line-70">import org.apache.hadoop.hbase.StartTestingClusterOption;</span> |
| <span class="source-line-no">071</span><span id="line-71">import org.apache.hadoop.hbase.TableName;</span> |
| <span class="source-line-no">072</span><span id="line-72">import org.apache.hadoop.hbase.Tag;</span> |
| <span class="source-line-no">073</span><span id="line-73">import org.apache.hadoop.hbase.TagType;</span> |
| <span class="source-line-no">074</span><span id="line-74">import org.apache.hadoop.hbase.client.Admin;</span> |
| <span class="source-line-no">075</span><span id="line-75">import org.apache.hadoop.hbase.client.AsyncConnection;</span> |
| <span class="source-line-no">076</span><span id="line-76">import org.apache.hadoop.hbase.client.BufferedMutator;</span> |
| <span class="source-line-no">077</span><span id="line-77">import org.apache.hadoop.hbase.client.BufferedMutatorParams;</span> |
| <span class="source-line-no">078</span><span id="line-78">import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;</span> |
| <span class="source-line-no">079</span><span id="line-79">import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;</span> |
| <span class="source-line-no">080</span><span id="line-80">import org.apache.hadoop.hbase.client.Connection;</span> |
| <span class="source-line-no">081</span><span id="line-81">import org.apache.hadoop.hbase.client.ConnectionFactory;</span> |
| <span class="source-line-no">082</span><span id="line-82">import org.apache.hadoop.hbase.client.ConnectionRegistry;</span> |
| <span class="source-line-no">083</span><span id="line-83">import org.apache.hadoop.hbase.client.ConnectionUtils;</span> |
| <span class="source-line-no">084</span><span id="line-84">import org.apache.hadoop.hbase.client.Hbck;</span> |
| <span class="source-line-no">085</span><span id="line-85">import org.apache.hadoop.hbase.client.Put;</span> |
| <span class="source-line-no">086</span><span id="line-86">import org.apache.hadoop.hbase.client.RegionLocator;</span> |
| <span class="source-line-no">087</span><span id="line-87">import org.apache.hadoop.hbase.client.Result;</span> |
| <span class="source-line-no">088</span><span id="line-88">import org.apache.hadoop.hbase.client.ResultScanner;</span> |
| <span class="source-line-no">089</span><span id="line-89">import org.apache.hadoop.hbase.client.Scan;</span> |
| <span class="source-line-no">090</span><span id="line-90">import org.apache.hadoop.hbase.client.Table;</span> |
| <span class="source-line-no">091</span><span id="line-91">import org.apache.hadoop.hbase.client.TableBuilder;</span> |
| <span class="source-line-no">092</span><span id="line-92">import org.apache.hadoop.hbase.client.TableDescriptor;</span> |
| <span class="source-line-no">093</span><span id="line-93">import org.apache.hadoop.hbase.client.TableDescriptorBuilder;</span> |
| <span class="source-line-no">094</span><span id="line-94">import org.apache.hadoop.hbase.io.ImmutableBytesWritable;</span> |
| <span class="source-line-no">095</span><span id="line-95">import org.apache.hadoop.hbase.io.compress.Compression;</span> |
| <span class="source-line-no">096</span><span id="line-96">import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;</span> |
| <span class="source-line-no">097</span><span id="line-97">import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;</span> |
| <span class="source-line-no">098</span><span id="line-98">import org.apache.hadoop.hbase.io.hfile.CacheConfig;</span> |
| <span class="source-line-no">099</span><span id="line-99">import org.apache.hadoop.hbase.io.hfile.HFile;</span> |
| <span class="source-line-no">100</span><span id="line-100">import org.apache.hadoop.hbase.io.hfile.HFile.Reader;</span> |
| <span class="source-line-no">101</span><span id="line-101">import org.apache.hadoop.hbase.io.hfile.HFileScanner;</span> |
| <span class="source-line-no">102</span><span id="line-102">import org.apache.hadoop.hbase.regionserver.BloomType;</span> |
| <span class="source-line-no">103</span><span id="line-103">import org.apache.hadoop.hbase.regionserver.HRegion;</span> |
| <span class="source-line-no">104</span><span id="line-104">import org.apache.hadoop.hbase.regionserver.HStore;</span> |
| <span class="source-line-no">105</span><span id="line-105">import org.apache.hadoop.hbase.regionserver.TestHRegionFileSystem;</span> |
| <span class="source-line-no">106</span><span id="line-106">import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;</span> |
| <span class="source-line-no">107</span><span id="line-107">import org.apache.hadoop.hbase.security.User;</span> |
| <span class="source-line-no">108</span><span id="line-108">import org.apache.hadoop.hbase.testclassification.LargeTests;</span> |
| <span class="source-line-no">109</span><span id="line-109">import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;</span> |
| <span class="source-line-no">110</span><span id="line-110">import org.apache.hadoop.hbase.tool.BulkLoadHFiles;</span> |
| <span class="source-line-no">111</span><span id="line-111">import org.apache.hadoop.hbase.util.Bytes;</span> |
| <span class="source-line-no">112</span><span id="line-112">import org.apache.hadoop.hbase.util.CommonFSUtils;</span> |
| <span class="source-line-no">113</span><span id="line-113">import org.apache.hadoop.hbase.util.FSUtils;</span> |
| <span class="source-line-no">114</span><span id="line-114">import org.apache.hadoop.hbase.util.FutureUtils;</span> |
| <span class="source-line-no">115</span><span id="line-115">import org.apache.hadoop.hbase.util.ReflectionUtils;</span> |
| <span class="source-line-no">116</span><span id="line-116">import org.apache.hadoop.hdfs.DistributedFileSystem;</span> |
| <span class="source-line-no">117</span><span id="line-117">import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;</span> |
| <span class="source-line-no">118</span><span id="line-118">import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;</span> |
| <span class="source-line-no">119</span><span id="line-119">import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;</span> |
| <span class="source-line-no">120</span><span id="line-120">import org.apache.hadoop.io.NullWritable;</span> |
| <span class="source-line-no">121</span><span id="line-121">import org.apache.hadoop.mapreduce.Job;</span> |
| <span class="source-line-no">122</span><span id="line-122">import org.apache.hadoop.mapreduce.Mapper;</span> |
| <span class="source-line-no">123</span><span id="line-123">import org.apache.hadoop.mapreduce.RecordWriter;</span> |
| <span class="source-line-no">124</span><span id="line-124">import org.apache.hadoop.mapreduce.TaskAttemptContext;</span> |
| <span class="source-line-no">125</span><span id="line-125">import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;</span> |
| <span class="source-line-no">126</span><span id="line-126">import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;</span> |
| <span class="source-line-no">127</span><span id="line-127">import org.apache.hadoop.security.UserGroupInformation;</span> |
| <span class="source-line-no">128</span><span id="line-128">import org.junit.Assert;</span> |
| <span class="source-line-no">129</span><span id="line-129">import org.junit.ClassRule;</span> |
| <span class="source-line-no">130</span><span id="line-130">import org.junit.Ignore;</span> |
| <span class="source-line-no">131</span><span id="line-131">import org.junit.Test;</span> |
| <span class="source-line-no">132</span><span id="line-132">import org.junit.experimental.categories.Category;</span> |
| <span class="source-line-no">133</span><span id="line-133">import org.mockito.Mockito;</span> |
| <span class="source-line-no">134</span><span id="line-134">import org.slf4j.Logger;</span> |
| <span class="source-line-no">135</span><span id="line-135">import org.slf4j.LoggerFactory;</span> |
| <span class="source-line-no">136</span><span id="line-136"></span> |
| <span class="source-line-no">137</span><span id="line-137">/**</span> |
| <span class="source-line-no">138</span><span id="line-138"> * Simple test for {@link HFileOutputFormat2}. Sets up and runs a mapreduce job that writes hfile</span> |
| <span class="source-line-no">139</span><span id="line-139"> * output. Creates a few inner classes to implement splits and an inputformat that emits keys and</span> |
| <span class="source-line-no">140</span><span id="line-140"> * values like those of {@link PerformanceEvaluation}.</span> |
| <span class="source-line-no">141</span><span id="line-141"> */</span> |
| <span class="source-line-no">142</span><span id="line-142">@Category({ VerySlowMapReduceTests.class, LargeTests.class })</span> |
| <span class="source-line-no">143</span><span id="line-143">public class TestHFileOutputFormat2 {</span> |
| <span class="source-line-no">144</span><span id="line-144"></span> |
| <span class="source-line-no">145</span><span id="line-145"> @ClassRule</span> |
| <span class="source-line-no">146</span><span id="line-146"> public static final HBaseClassTestRule CLASS_RULE =</span> |
| <span class="source-line-no">147</span><span id="line-147"> HBaseClassTestRule.forClass(TestHFileOutputFormat2.class);</span> |
| <span class="source-line-no">148</span><span id="line-148"></span> |
| <span class="source-line-no">149</span><span id="line-149"> private final static int ROWSPERSPLIT = 1024;</span> |
| <span class="source-line-no">150</span><span id="line-150"></span> |
| <span class="source-line-no">151</span><span id="line-151"> public static final byte[] FAMILY_NAME = TestHRegionFileSystem.FAMILY_NAME;</span> |
| <span class="source-line-no">152</span><span id="line-152"> private static final byte[][] FAMILIES =</span> |
| <span class="source-line-no">153</span><span id="line-153"> { Bytes.add(FAMILY_NAME, Bytes.toBytes("-A")), Bytes.add(FAMILY_NAME, Bytes.toBytes("-B")) };</span> |
| <span class="source-line-no">154</span><span id="line-154"> private static final TableName[] TABLE_NAMES = Stream.of("TestTable", "TestTable2", "TestTable3")</span> |
| <span class="source-line-no">155</span><span id="line-155"> .map(TableName::valueOf).toArray(TableName[]::new);</span> |
| <span class="source-line-no">156</span><span id="line-156"></span> |
| <span class="source-line-no">157</span><span id="line-157"> private HBaseTestingUtil util = new HBaseTestingUtil();</span> |
| <span class="source-line-no">158</span><span id="line-158"></span> |
| <span class="source-line-no">159</span><span id="line-159"> private static final Logger LOG = LoggerFactory.getLogger(TestHFileOutputFormat2.class);</span> |
| <span class="source-line-no">160</span><span id="line-160"></span> |
| <span class="source-line-no">161</span><span id="line-161"> /**</span> |
| <span class="source-line-no">162</span><span id="line-162"> * Simple mapper that makes KeyValue output.</span> |
| <span class="source-line-no">163</span><span id="line-163"> */</span> |
| <span class="source-line-no">164</span><span id="line-164"> static class RandomKVGeneratingMapper</span> |
| <span class="source-line-no">165</span><span id="line-165"> extends Mapper<NullWritable, NullWritable, ImmutableBytesWritable, Cell> {</span> |
| <span class="source-line-no">166</span><span id="line-166"></span> |
| <span class="source-line-no">167</span><span id="line-167"> private int keyLength;</span> |
| <span class="source-line-no">168</span><span id="line-168"> private static final int KEYLEN_DEFAULT = 10;</span> |
| <span class="source-line-no">169</span><span id="line-169"> private static final String KEYLEN_CONF = "randomkv.key.length";</span> |
| <span class="source-line-no">170</span><span id="line-170"></span> |
| <span class="source-line-no">171</span><span id="line-171"> private int valLength;</span> |
| <span class="source-line-no">172</span><span id="line-172"> private static final int VALLEN_DEFAULT = 10;</span> |
| <span class="source-line-no">173</span><span id="line-173"> private static final String VALLEN_CONF = "randomkv.val.length";</span> |
| <span class="source-line-no">174</span><span id="line-174"> private static final byte[] QUALIFIER = Bytes.toBytes("data");</span> |
| <span class="source-line-no">175</span><span id="line-175"> private boolean multiTableMapper = false;</span> |
| <span class="source-line-no">176</span><span id="line-176"> private TableName[] tables = null;</span> |
| <span class="source-line-no">177</span><span id="line-177"></span> |
| <span class="source-line-no">178</span><span id="line-178"> @Override</span> |
| <span class="source-line-no">179</span><span id="line-179"> protected void setup(Context context) throws IOException, InterruptedException {</span> |
| <span class="source-line-no">180</span><span id="line-180"> super.setup(context);</span> |
| <span class="source-line-no">181</span><span id="line-181"></span> |
| <span class="source-line-no">182</span><span id="line-182"> Configuration conf = context.getConfiguration();</span> |
| <span class="source-line-no">183</span><span id="line-183"> keyLength = conf.getInt(KEYLEN_CONF, KEYLEN_DEFAULT);</span> |
| <span class="source-line-no">184</span><span id="line-184"> valLength = conf.getInt(VALLEN_CONF, VALLEN_DEFAULT);</span> |
| <span class="source-line-no">185</span><span id="line-185"> multiTableMapper =</span> |
| <span class="source-line-no">186</span><span id="line-186"> conf.getBoolean(HFileOutputFormat2.MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY, false);</span> |
| <span class="source-line-no">187</span><span id="line-187"> if (multiTableMapper) {</span> |
| <span class="source-line-no">188</span><span id="line-188"> tables = TABLE_NAMES;</span> |
| <span class="source-line-no">189</span><span id="line-189"> } else {</span> |
| <span class="source-line-no">190</span><span id="line-190"> tables = new TableName[] { TABLE_NAMES[0] };</span> |
| <span class="source-line-no">191</span><span id="line-191"> }</span> |
| <span class="source-line-no">192</span><span id="line-192"> }</span> |
| <span class="source-line-no">193</span><span id="line-193"></span> |
| <span class="source-line-no">194</span><span id="line-194"> @Override</span> |
| <span class="source-line-no">195</span><span id="line-195"> protected void map(NullWritable n1, NullWritable n2,</span> |
| <span class="source-line-no">196</span><span id="line-196"> Mapper<NullWritable, NullWritable, ImmutableBytesWritable, Cell>.Context context)</span> |
| <span class="source-line-no">197</span><span id="line-197"> throws java.io.IOException, InterruptedException {</span> |
| <span class="source-line-no">198</span><span id="line-198"></span> |
| <span class="source-line-no">199</span><span id="line-199"> byte keyBytes[] = new byte[keyLength];</span> |
| <span class="source-line-no">200</span><span id="line-200"> byte valBytes[] = new byte[valLength];</span> |
| <span class="source-line-no">201</span><span id="line-201"></span> |
| <span class="source-line-no">202</span><span id="line-202"> int taskId = context.getTaskAttemptID().getTaskID().getId();</span> |
| <span class="source-line-no">203</span><span id="line-203"> assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!";</span> |
| <span class="source-line-no">204</span><span id="line-204"> byte[] key;</span> |
| <span class="source-line-no">205</span><span id="line-205"> for (int j = 0; j < tables.length; ++j) {</span> |
| <span class="source-line-no">206</span><span id="line-206"> for (int i = 0; i < ROWSPERSPLIT; i++) {</span> |
| <span class="source-line-no">207</span><span id="line-207"> Bytes.random(keyBytes);</span> |
| <span class="source-line-no">208</span><span id="line-208"> // Ensure that unique tasks generate unique keys</span> |
| <span class="source-line-no">209</span><span id="line-209"> keyBytes[keyLength - 1] = (byte) (taskId & 0xFF);</span> |
| <span class="source-line-no">210</span><span id="line-210"> Bytes.random(valBytes);</span> |
| <span class="source-line-no">211</span><span id="line-211"> key = keyBytes;</span> |
| <span class="source-line-no">212</span><span id="line-212"> if (multiTableMapper) {</span> |
| <span class="source-line-no">213</span><span id="line-213"> key = MultiTableHFileOutputFormat.createCompositeKey(tables[j].getName(), keyBytes);</span> |
| <span class="source-line-no">214</span><span id="line-214"> }</span> |
| <span class="source-line-no">215</span><span id="line-215"></span> |
| <span class="source-line-no">216</span><span id="line-216"> for (byte[] family : TestHFileOutputFormat2.FAMILIES) {</span> |
| <span class="source-line-no">217</span><span id="line-217"> Cell kv = new KeyValue(keyBytes, family, QUALIFIER, valBytes);</span> |
| <span class="source-line-no">218</span><span id="line-218"> context.write(new ImmutableBytesWritable(key), kv);</span> |
| <span class="source-line-no">219</span><span id="line-219"> }</span> |
| <span class="source-line-no">220</span><span id="line-220"> }</span> |
| <span class="source-line-no">221</span><span id="line-221"> }</span> |
| <span class="source-line-no">222</span><span id="line-222"> }</span> |
| <span class="source-line-no">223</span><span id="line-223"> }</span> |
| <span class="source-line-no">224</span><span id="line-224"></span> |
| <span class="source-line-no">225</span><span id="line-225"> /**</span> |
| <span class="source-line-no">226</span><span id="line-226"> * Simple mapper that makes Put output.</span> |
| <span class="source-line-no">227</span><span id="line-227"> */</span> |
| <span class="source-line-no">228</span><span id="line-228"> static class RandomPutGeneratingMapper</span> |
| <span class="source-line-no">229</span><span id="line-229"> extends Mapper<NullWritable, NullWritable, ImmutableBytesWritable, Put> {</span> |
| <span class="source-line-no">230</span><span id="line-230"></span> |
| <span class="source-line-no">231</span><span id="line-231"> private int keyLength;</span> |
| <span class="source-line-no">232</span><span id="line-232"> private static final int KEYLEN_DEFAULT = 10;</span> |
| <span class="source-line-no">233</span><span id="line-233"> private static final String KEYLEN_CONF = "randomkv.key.length";</span> |
| <span class="source-line-no">234</span><span id="line-234"></span> |
| <span class="source-line-no">235</span><span id="line-235"> private int valLength;</span> |
| <span class="source-line-no">236</span><span id="line-236"> private static final int VALLEN_DEFAULT = 10;</span> |
| <span class="source-line-no">237</span><span id="line-237"> private static final String VALLEN_CONF = "randomkv.val.length";</span> |
| <span class="source-line-no">238</span><span id="line-238"> private static final byte[] QUALIFIER = Bytes.toBytes("data");</span> |
| <span class="source-line-no">239</span><span id="line-239"> private boolean multiTableMapper = false;</span> |
| <span class="source-line-no">240</span><span id="line-240"> private TableName[] tables = null;</span> |
| <span class="source-line-no">241</span><span id="line-241"></span> |
| <span class="source-line-no">242</span><span id="line-242"> @Override</span> |
| <span class="source-line-no">243</span><span id="line-243"> protected void setup(Context context) throws IOException, InterruptedException {</span> |
| <span class="source-line-no">244</span><span id="line-244"> super.setup(context);</span> |
| <span class="source-line-no">245</span><span id="line-245"></span> |
| <span class="source-line-no">246</span><span id="line-246"> Configuration conf = context.getConfiguration();</span> |
| <span class="source-line-no">247</span><span id="line-247"> keyLength = conf.getInt(KEYLEN_CONF, KEYLEN_DEFAULT);</span> |
| <span class="source-line-no">248</span><span id="line-248"> valLength = conf.getInt(VALLEN_CONF, VALLEN_DEFAULT);</span> |
| <span class="source-line-no">249</span><span id="line-249"> multiTableMapper =</span> |
| <span class="source-line-no">250</span><span id="line-250"> conf.getBoolean(HFileOutputFormat2.MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY, false);</span> |
| <span class="source-line-no">251</span><span id="line-251"> if (multiTableMapper) {</span> |
| <span class="source-line-no">252</span><span id="line-252"> tables = TABLE_NAMES;</span> |
| <span class="source-line-no">253</span><span id="line-253"> } else {</span> |
| <span class="source-line-no">254</span><span id="line-254"> tables = new TableName[] { TABLE_NAMES[0] };</span> |
| <span class="source-line-no">255</span><span id="line-255"> }</span> |
| <span class="source-line-no">256</span><span id="line-256"> }</span> |
| <span class="source-line-no">257</span><span id="line-257"></span> |
| <span class="source-line-no">258</span><span id="line-258"> @Override</span> |
| <span class="source-line-no">259</span><span id="line-259"> protected void map(NullWritable n1, NullWritable n2,</span> |
| <span class="source-line-no">260</span><span id="line-260"> Mapper<NullWritable, NullWritable, ImmutableBytesWritable, Put>.Context context)</span> |
| <span class="source-line-no">261</span><span id="line-261"> throws java.io.IOException, InterruptedException {</span> |
| <span class="source-line-no">262</span><span id="line-262"></span> |
| <span class="source-line-no">263</span><span id="line-263"> byte keyBytes[] = new byte[keyLength];</span> |
| <span class="source-line-no">264</span><span id="line-264"> byte valBytes[] = new byte[valLength];</span> |
| <span class="source-line-no">265</span><span id="line-265"></span> |
| <span class="source-line-no">266</span><span id="line-266"> int taskId = context.getTaskAttemptID().getTaskID().getId();</span> |
| <span class="source-line-no">267</span><span id="line-267"> assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!";</span> |
| <span class="source-line-no">268</span><span id="line-268"></span> |
| <span class="source-line-no">269</span><span id="line-269"> byte[] key;</span> |
| <span class="source-line-no">270</span><span id="line-270"> for (int j = 0; j < tables.length; ++j) {</span> |
| <span class="source-line-no">271</span><span id="line-271"> for (int i = 0; i < ROWSPERSPLIT; i++) {</span> |
| <span class="source-line-no">272</span><span id="line-272"> Bytes.random(keyBytes);</span> |
| <span class="source-line-no">273</span><span id="line-273"> // Ensure that unique tasks generate unique keys</span> |
| <span class="source-line-no">274</span><span id="line-274"> keyBytes[keyLength - 1] = (byte) (taskId & 0xFF);</span> |
| <span class="source-line-no">275</span><span id="line-275"> Bytes.random(valBytes);</span> |
| <span class="source-line-no">276</span><span id="line-276"> key = keyBytes;</span> |
| <span class="source-line-no">277</span><span id="line-277"> if (multiTableMapper) {</span> |
| <span class="source-line-no">278</span><span id="line-278"> key = MultiTableHFileOutputFormat.createCompositeKey(tables[j].getName(), keyBytes);</span> |
| <span class="source-line-no">279</span><span id="line-279"> }</span> |
| <span class="source-line-no">280</span><span id="line-280"></span> |
| <span class="source-line-no">281</span><span id="line-281"> for (byte[] family : TestHFileOutputFormat2.FAMILIES) {</span> |
| <span class="source-line-no">282</span><span id="line-282"> Put p = new Put(keyBytes);</span> |
| <span class="source-line-no">283</span><span id="line-283"> p.addColumn(family, QUALIFIER, valBytes);</span> |
| <span class="source-line-no">284</span><span id="line-284"> // set TTL to very low so that the scan does not return any value</span> |
| <span class="source-line-no">285</span><span id="line-285"> p.setTTL(1l);</span> |
| <span class="source-line-no">286</span><span id="line-286"> context.write(new ImmutableBytesWritable(key), p);</span> |
| <span class="source-line-no">287</span><span id="line-287"> }</span> |
| <span class="source-line-no">288</span><span id="line-288"> }</span> |
| <span class="source-line-no">289</span><span id="line-289"> }</span> |
| <span class="source-line-no">290</span><span id="line-290"> }</span> |
| <span class="source-line-no">291</span><span id="line-291"> }</span> |
| <span class="source-line-no">292</span><span id="line-292"></span> |
| <span class="source-line-no">293</span><span id="line-293"> private void setupRandomGeneratorMapper(Job job, boolean putSortReducer) {</span> |
| <span class="source-line-no">294</span><span id="line-294"> if (putSortReducer) {</span> |
| <span class="source-line-no">295</span><span id="line-295"> job.setInputFormatClass(NMapInputFormat.class);</span> |
| <span class="source-line-no">296</span><span id="line-296"> job.setMapperClass(RandomPutGeneratingMapper.class);</span> |
| <span class="source-line-no">297</span><span id="line-297"> job.setMapOutputKeyClass(ImmutableBytesWritable.class);</span> |
| <span class="source-line-no">298</span><span id="line-298"> job.setMapOutputValueClass(Put.class);</span> |
| <span class="source-line-no">299</span><span id="line-299"> } else {</span> |
| <span class="source-line-no">300</span><span id="line-300"> job.setInputFormatClass(NMapInputFormat.class);</span> |
| <span class="source-line-no">301</span><span id="line-301"> job.setMapperClass(RandomKVGeneratingMapper.class);</span> |
| <span class="source-line-no">302</span><span id="line-302"> job.setMapOutputKeyClass(ImmutableBytesWritable.class);</span> |
| <span class="source-line-no">303</span><span id="line-303"> job.setMapOutputValueClass(KeyValue.class);</span> |
| <span class="source-line-no">304</span><span id="line-304"> }</span> |
| <span class="source-line-no">305</span><span id="line-305"> }</span> |
| <span class="source-line-no">306</span><span id="line-306"></span> |
| <span class="source-line-no">307</span><span id="line-307"> /**</span> |
| <span class="source-line-no">308</span><span id="line-308"> * Test that {@link HFileOutputFormat2} RecordWriter amends timestamps if passed a keyvalue whose</span> |
| <span class="source-line-no">309</span><span id="line-309"> * timestamp is {@link HConstants#LATEST_TIMESTAMP}.</span> |
| <span class="source-line-no">310</span><span id="line-310"> * @see <a href="https://issues.apache.org/jira/browse/HBASE-2615">HBASE-2615</a></span> |
| <span class="source-line-no">311</span><span id="line-311"> */</span> |
| <span class="source-line-no">312</span><span id="line-312"> @Ignore("Goes zombie too frequently; needs work. See HBASE-14563")</span> |
| <span class="source-line-no">313</span><span id="line-313"> @Test</span> |
| <span class="source-line-no">314</span><span id="line-314"> public void test_LATEST_TIMESTAMP_isReplaced() throws Exception {</span> |
| <span class="source-line-no">315</span><span id="line-315"> Configuration conf = new Configuration(this.util.getConfiguration());</span> |
| <span class="source-line-no">316</span><span id="line-316"> RecordWriter<ImmutableBytesWritable, Cell> writer = null;</span> |
| <span class="source-line-no">317</span><span id="line-317"> TaskAttemptContext context = null;</span> |
| <span class="source-line-no">318</span><span id="line-318"> Path dir = util.getDataTestDir("test_LATEST_TIMESTAMP_isReplaced");</span> |
| <span class="source-line-no">319</span><span id="line-319"> try {</span> |
| <span class="source-line-no">320</span><span id="line-320"> Job job = new Job(conf);</span> |
| <span class="source-line-no">321</span><span id="line-321"> FileOutputFormat.setOutputPath(job, dir);</span> |
| <span class="source-line-no">322</span><span id="line-322"> context = createTestTaskAttemptContext(job);</span> |
| <span class="source-line-no">323</span><span id="line-323"> HFileOutputFormat2 hof = new HFileOutputFormat2();</span> |
| <span class="source-line-no">324</span><span id="line-324"> writer = hof.getRecordWriter(context);</span> |
| <span class="source-line-no">325</span><span id="line-325"> final byte[] b = Bytes.toBytes("b");</span> |
| <span class="source-line-no">326</span><span id="line-326"></span> |
| <span class="source-line-no">327</span><span id="line-327"> // Test 1. Pass a KV that has a ts of LATEST_TIMESTAMP. It should be</span> |
| <span class="source-line-no">328</span><span id="line-328"> // changed by call to write. Check all in kv is same but ts.</span> |
| <span class="source-line-no">329</span><span id="line-329"> KeyValue kv = new KeyValue(b, b, b);</span> |
| <span class="source-line-no">330</span><span id="line-330"> KeyValue original = kv.clone();</span> |
| <span class="source-line-no">331</span><span id="line-331"> writer.write(new ImmutableBytesWritable(), kv);</span> |
| <span class="source-line-no">332</span><span id="line-332"> assertFalse(original.equals(kv));</span> |
| <span class="source-line-no">333</span><span id="line-333"> assertTrue(Bytes.equals(CellUtil.cloneRow(original), CellUtil.cloneRow(kv)));</span> |
| <span class="source-line-no">334</span><span id="line-334"> assertTrue(Bytes.equals(CellUtil.cloneFamily(original), CellUtil.cloneFamily(kv)));</span> |
| <span class="source-line-no">335</span><span id="line-335"> assertTrue(Bytes.equals(CellUtil.cloneQualifier(original), CellUtil.cloneQualifier(kv)));</span> |
| <span class="source-line-no">336</span><span id="line-336"> assertNotSame(original.getTimestamp(), kv.getTimestamp());</span> |
| <span class="source-line-no">337</span><span id="line-337"> assertNotSame(HConstants.LATEST_TIMESTAMP, kv.getTimestamp());</span> |
| <span class="source-line-no">338</span><span id="line-338"></span> |
| <span class="source-line-no">339</span><span id="line-339"> // Test 2. Now test passing a kv that has explicit ts. It should not be</span> |
| <span class="source-line-no">340</span><span id="line-340"> // changed by call to record write.</span> |
| <span class="source-line-no">341</span><span id="line-341"> kv = new KeyValue(b, b, b, kv.getTimestamp() - 1, b);</span> |
| <span class="source-line-no">342</span><span id="line-342"> original = kv.clone();</span> |
| <span class="source-line-no">343</span><span id="line-343"> writer.write(new ImmutableBytesWritable(), kv);</span> |
| <span class="source-line-no">344</span><span id="line-344"> assertTrue(original.equals(kv));</span> |
| <span class="source-line-no">345</span><span id="line-345"> } finally {</span> |
| <span class="source-line-no">346</span><span id="line-346"> if (writer != null && context != null) writer.close(context);</span> |
| <span class="source-line-no">347</span><span id="line-347"> dir.getFileSystem(conf).delete(dir, true);</span> |
| <span class="source-line-no">348</span><span id="line-348"> }</span> |
| <span class="source-line-no">349</span><span id="line-349"> }</span> |
| <span class="source-line-no">350</span><span id="line-350"></span> |
| <span class="source-line-no">351</span><span id="line-351"> private TaskAttemptContext createTestTaskAttemptContext(final Job job) throws Exception {</span> |
| <span class="source-line-no">352</span><span id="line-352"> HadoopShims hadoop = CompatibilitySingletonFactory.getInstance(HadoopShims.class);</span> |
| <span class="source-line-no">353</span><span id="line-353"> TaskAttemptContext context =</span> |
| <span class="source-line-no">354</span><span id="line-354"> hadoop.createTestTaskAttemptContext(job, "attempt_201402131733_0001_m_000000_0");</span> |
| <span class="source-line-no">355</span><span id="line-355"> return context;</span> |
| <span class="source-line-no">356</span><span id="line-356"> }</span> |
| <span class="source-line-no">357</span><span id="line-357"></span> |
| <span class="source-line-no">358</span><span id="line-358"> /*</span> |
| <span class="source-line-no">359</span><span id="line-359"> * Test that {@link HFileOutputFormat2} creates an HFile with TIMERANGE metadata used by</span> |
| <span class="source-line-no">360</span><span id="line-360"> * time-restricted scans.</span> |
| <span class="source-line-no">361</span><span id="line-361"> */</span> |
| <span class="source-line-no">362</span><span id="line-362"> @Ignore("Goes zombie too frequently; needs work. See HBASE-14563")</span> |
| <span class="source-line-no">363</span><span id="line-363"> @Test</span> |
| <span class="source-line-no">364</span><span id="line-364"> public void test_TIMERANGE() throws Exception {</span> |
| <span class="source-line-no">365</span><span id="line-365"> Configuration conf = new Configuration(this.util.getConfiguration());</span> |
| <span class="source-line-no">366</span><span id="line-366"> RecordWriter<ImmutableBytesWritable, Cell> writer = null;</span> |
| <span class="source-line-no">367</span><span id="line-367"> TaskAttemptContext context = null;</span> |
| <span class="source-line-no">368</span><span id="line-368"> Path dir = util.getDataTestDir("test_TIMERANGE_present");</span> |
| <span class="source-line-no">369</span><span id="line-369"> LOG.info("Timerange dir writing to dir: " + dir);</span> |
| <span class="source-line-no">370</span><span id="line-370"> try {</span> |
| <span class="source-line-no">371</span><span id="line-371"> // build a record writer using HFileOutputFormat2</span> |
| <span class="source-line-no">372</span><span id="line-372"> Job job = new Job(conf);</span> |
| <span class="source-line-no">373</span><span id="line-373"> FileOutputFormat.setOutputPath(job, dir);</span> |
| <span class="source-line-no">374</span><span id="line-374"> context = createTestTaskAttemptContext(job);</span> |
| <span class="source-line-no">375</span><span id="line-375"> HFileOutputFormat2 hof = new HFileOutputFormat2();</span> |
| <span class="source-line-no">376</span><span id="line-376"> writer = hof.getRecordWriter(context);</span> |
| <span class="source-line-no">377</span><span id="line-377"></span> |
| <span class="source-line-no">378</span><span id="line-378"> // Pass two key values with explicit times stamps</span> |
| <span class="source-line-no">379</span><span id="line-379"> final byte[] b = Bytes.toBytes("b");</span> |
| <span class="source-line-no">380</span><span id="line-380"></span> |
| <span class="source-line-no">381</span><span id="line-381"> // value 1 with timestamp 2000</span> |
| <span class="source-line-no">382</span><span id="line-382"> KeyValue kv = new KeyValue(b, b, b, 2000, b);</span> |
| <span class="source-line-no">383</span><span id="line-383"> KeyValue original = kv.clone();</span> |
| <span class="source-line-no">384</span><span id="line-384"> writer.write(new ImmutableBytesWritable(), kv);</span> |
| <span class="source-line-no">385</span><span id="line-385"> assertEquals(original, kv);</span> |
| <span class="source-line-no">386</span><span id="line-386"></span> |
| <span class="source-line-no">387</span><span id="line-387"> // value 2 with timestamp 1000</span> |
| <span class="source-line-no">388</span><span id="line-388"> kv = new KeyValue(b, b, b, 1000, b);</span> |
| <span class="source-line-no">389</span><span id="line-389"> original = kv.clone();</span> |
| <span class="source-line-no">390</span><span id="line-390"> writer.write(new ImmutableBytesWritable(), kv);</span> |
| <span class="source-line-no">391</span><span id="line-391"> assertEquals(original, kv);</span> |
| <span class="source-line-no">392</span><span id="line-392"></span> |
| <span class="source-line-no">393</span><span id="line-393"> // verify that the file has the proper FileInfo.</span> |
| <span class="source-line-no">394</span><span id="line-394"> writer.close(context);</span> |
| <span class="source-line-no">395</span><span id="line-395"></span> |
| <span class="source-line-no">396</span><span id="line-396"> // the generated file lives 1 directory down from the attempt directory</span> |
| <span class="source-line-no">397</span><span id="line-397"> // and is the only file, e.g.</span> |
| <span class="source-line-no">398</span><span id="line-398"> // _attempt__0000_r_000000_0/b/1979617994050536795</span> |
| <span class="source-line-no">399</span><span id="line-399"> FileSystem fs = FileSystem.get(conf);</span> |
| <span class="source-line-no">400</span><span id="line-400"> Path attemptDirectory = hof.getDefaultWorkFile(context, "").getParent();</span> |
| <span class="source-line-no">401</span><span id="line-401"> FileStatus[] sub1 = fs.listStatus(attemptDirectory);</span> |
| <span class="source-line-no">402</span><span id="line-402"> FileStatus[] file = fs.listStatus(sub1[0].getPath());</span> |
| <span class="source-line-no">403</span><span id="line-403"></span> |
| <span class="source-line-no">404</span><span id="line-404"> // open as HFile Reader and pull out TIMERANGE FileInfo.</span> |
| <span class="source-line-no">405</span><span id="line-405"> HFile.Reader rd =</span> |
| <span class="source-line-no">406</span><span id="line-406"> HFile.createReader(fs, file[0].getPath(), new CacheConfig(conf), true, conf);</span> |
| <span class="source-line-no">407</span><span id="line-407"> Map<byte[], byte[]> finfo = rd.getHFileInfo();</span> |
| <span class="source-line-no">408</span><span id="line-408"> byte[] range = finfo.get(Bytes.toBytes("TIMERANGE"));</span> |
| <span class="source-line-no">409</span><span id="line-409"> assertNotNull(range);</span> |
| <span class="source-line-no">410</span><span id="line-410"></span> |
| <span class="source-line-no">411</span><span id="line-411"> // unmarshall and check values.</span> |
| <span class="source-line-no">412</span><span id="line-412"> TimeRangeTracker timeRangeTracker = TimeRangeTracker.parseFrom(range);</span> |
| <span class="source-line-no">413</span><span id="line-413"> LOG.info(timeRangeTracker.getMin() + "...." + timeRangeTracker.getMax());</span> |
| <span class="source-line-no">414</span><span id="line-414"> assertEquals(1000, timeRangeTracker.getMin());</span> |
| <span class="source-line-no">415</span><span id="line-415"> assertEquals(2000, timeRangeTracker.getMax());</span> |
| <span class="source-line-no">416</span><span id="line-416"> rd.close();</span> |
| <span class="source-line-no">417</span><span id="line-417"> } finally {</span> |
| <span class="source-line-no">418</span><span id="line-418"> if (writer != null && context != null) writer.close(context);</span> |
| <span class="source-line-no">419</span><span id="line-419"> dir.getFileSystem(conf).delete(dir, true);</span> |
| <span class="source-line-no">420</span><span id="line-420"> }</span> |
| <span class="source-line-no">421</span><span id="line-421"> }</span> |
| <span class="source-line-no">422</span><span id="line-422"></span> |
| <span class="source-line-no">423</span><span id="line-423"> /**</span> |
| <span class="source-line-no">424</span><span id="line-424"> * Run small MR job.</span> |
| <span class="source-line-no">425</span><span id="line-425"> */</span> |
| <span class="source-line-no">426</span><span id="line-426"> @Ignore("Goes zombie too frequently; needs work. See HBASE-14563")</span> |
| <span class="source-line-no">427</span><span id="line-427"> @Test</span> |
| <span class="source-line-no">428</span><span id="line-428"> public void testWritingPEData() throws Exception {</span> |
| <span class="source-line-no">429</span><span id="line-429"> Configuration conf = util.getConfiguration();</span> |
| <span class="source-line-no">430</span><span id="line-430"> Path testDir = util.getDataTestDirOnTestFS("testWritingPEData");</span> |
| <span class="source-line-no">431</span><span id="line-431"> FileSystem fs = testDir.getFileSystem(conf);</span> |
| <span class="source-line-no">432</span><span id="line-432"></span> |
| <span class="source-line-no">433</span><span id="line-433"> // Set down this value or we OOME in eclipse.</span> |
| <span class="source-line-no">434</span><span id="line-434"> conf.setInt("mapreduce.task.io.sort.mb", 20);</span> |
| <span class="source-line-no">435</span><span id="line-435"> // Write a few files.</span> |
| <span class="source-line-no">436</span><span id="line-436"> long hregionMaxFilesize = 10 * 1024;</span> |
| <span class="source-line-no">437</span><span id="line-437"> conf.setLong(HConstants.HREGION_MAX_FILESIZE, hregionMaxFilesize);</span> |
| <span class="source-line-no">438</span><span id="line-438"></span> |
| <span class="source-line-no">439</span><span id="line-439"> Job job = new Job(conf, "testWritingPEData");</span> |
| <span class="source-line-no">440</span><span id="line-440"> setupRandomGeneratorMapper(job, false);</span> |
| <span class="source-line-no">441</span><span id="line-441"> // This partitioner doesn't work well for number keys but using it anyways</span> |
| <span class="source-line-no">442</span><span id="line-442"> // just to demonstrate how to configure it.</span> |
| <span class="source-line-no">443</span><span id="line-443"> byte[] startKey = new byte[RandomKVGeneratingMapper.KEYLEN_DEFAULT];</span> |
| <span class="source-line-no">444</span><span id="line-444"> byte[] endKey = new byte[RandomKVGeneratingMapper.KEYLEN_DEFAULT];</span> |
| <span class="source-line-no">445</span><span id="line-445"></span> |
| <span class="source-line-no">446</span><span id="line-446"> Arrays.fill(startKey, (byte) 0);</span> |
| <span class="source-line-no">447</span><span id="line-447"> Arrays.fill(endKey, (byte) 0xff);</span> |
| <span class="source-line-no">448</span><span id="line-448"></span> |
| <span class="source-line-no">449</span><span id="line-449"> job.setPartitionerClass(SimpleTotalOrderPartitioner.class);</span> |
| <span class="source-line-no">450</span><span id="line-450"> // Set start and end rows for partitioner.</span> |
| <span class="source-line-no">451</span><span id="line-451"> SimpleTotalOrderPartitioner.setStartKey(job.getConfiguration(), startKey);</span> |
| <span class="source-line-no">452</span><span id="line-452"> SimpleTotalOrderPartitioner.setEndKey(job.getConfiguration(), endKey);</span> |
| <span class="source-line-no">453</span><span id="line-453"> job.setReducerClass(CellSortReducer.class);</span> |
| <span class="source-line-no">454</span><span id="line-454"> job.setOutputFormatClass(HFileOutputFormat2.class);</span> |
| <span class="source-line-no">455</span><span id="line-455"> job.setNumReduceTasks(4);</span> |
| <span class="source-line-no">456</span><span id="line-456"> job.getConfiguration().setStrings("io.serializations", conf.get("io.serializations"),</span> |
| <span class="source-line-no">457</span><span id="line-457"> MutationSerialization.class.getName(), ResultSerialization.class.getName(),</span> |
| <span class="source-line-no">458</span><span id="line-458"> CellSerialization.class.getName());</span> |
| <span class="source-line-no">459</span><span id="line-459"></span> |
| <span class="source-line-no">460</span><span id="line-460"> FileOutputFormat.setOutputPath(job, testDir);</span> |
| <span class="source-line-no">461</span><span id="line-461"> assertTrue(job.waitForCompletion(false));</span> |
| <span class="source-line-no">462</span><span id="line-462"> FileStatus[] files = fs.listStatus(testDir);</span> |
| <span class="source-line-no">463</span><span id="line-463"> assertTrue(files.length > 0);</span> |
| <span class="source-line-no">464</span><span id="line-464"></span> |
| <span class="source-line-no">465</span><span id="line-465"> // check output file num and size.</span> |
| <span class="source-line-no">466</span><span id="line-466"> for (byte[] family : FAMILIES) {</span> |
| <span class="source-line-no">467</span><span id="line-467"> long kvCount = 0;</span> |
| <span class="source-line-no">468</span><span id="line-468"> RemoteIterator<LocatedFileStatus> iterator =</span> |
| <span class="source-line-no">469</span><span id="line-469"> fs.listFiles(testDir.suffix("/" + new String(family)), true);</span> |
| <span class="source-line-no">470</span><span id="line-470"> while (iterator.hasNext()) {</span> |
| <span class="source-line-no">471</span><span id="line-471"> LocatedFileStatus keyFileStatus = iterator.next();</span> |
| <span class="source-line-no">472</span><span id="line-472"> HFile.Reader reader =</span> |
| <span class="source-line-no">473</span><span id="line-473"> HFile.createReader(fs, keyFileStatus.getPath(), new CacheConfig(conf), true, conf);</span> |
| <span class="source-line-no">474</span><span id="line-474"> HFileScanner scanner = reader.getScanner(conf, false, false, false);</span> |
| <span class="source-line-no">475</span><span id="line-475"></span> |
| <span class="source-line-no">476</span><span id="line-476"> kvCount += reader.getEntries();</span> |
| <span class="source-line-no">477</span><span id="line-477"> scanner.seekTo();</span> |
| <span class="source-line-no">478</span><span id="line-478"> long perKVSize = scanner.getCell().getSerializedSize();</span> |
| <span class="source-line-no">479</span><span id="line-479"> assertTrue("Data size of each file should not be too large.",</span> |
| <span class="source-line-no">480</span><span id="line-480"> perKVSize * reader.getEntries() <= hregionMaxFilesize);</span> |
| <span class="source-line-no">481</span><span id="line-481"> }</span> |
| <span class="source-line-no">482</span><span id="line-482"> assertEquals("Should write expected data in output file.", ROWSPERSPLIT, kvCount);</span> |
| <span class="source-line-no">483</span><span id="line-483"> }</span> |
| <span class="source-line-no">484</span><span id="line-484"> }</span> |
| <span class="source-line-no">485</span><span id="line-485"></span> |
| <span class="source-line-no">486</span><span id="line-486"> /**</span> |
| <span class="source-line-no">487</span><span id="line-487"> * Test that {@link HFileOutputFormat2} RecordWriter writes tags such as ttl into hfile.</span> |
| <span class="source-line-no">488</span><span id="line-488"> */</span> |
| <span class="source-line-no">489</span><span id="line-489"> @Test</span> |
| <span class="source-line-no">490</span><span id="line-490"> public void test_WritingTagData() throws Exception {</span> |
| <span class="source-line-no">491</span><span id="line-491"> Configuration conf = new Configuration(this.util.getConfiguration());</span> |
| <span class="source-line-no">492</span><span id="line-492"> final String HFILE_FORMAT_VERSION_CONF_KEY = "hfile.format.version";</span> |
| <span class="source-line-no">493</span><span id="line-493"> conf.setInt(HFILE_FORMAT_VERSION_CONF_KEY, HFile.MIN_FORMAT_VERSION_WITH_TAGS);</span> |
| <span class="source-line-no">494</span><span id="line-494"> RecordWriter<ImmutableBytesWritable, Cell> writer = null;</span> |
| <span class="source-line-no">495</span><span id="line-495"> TaskAttemptContext context = null;</span> |
| <span class="source-line-no">496</span><span id="line-496"> Path dir = util.getDataTestDir("WritingTagData");</span> |
| <span class="source-line-no">497</span><span id="line-497"> try {</span> |
| <span class="source-line-no">498</span><span id="line-498"> conf.set(HFileOutputFormat2.OUTPUT_TABLE_NAME_CONF_KEY, TABLE_NAMES[0].getNameAsString());</span> |
| <span class="source-line-no">499</span><span id="line-499"> // turn locality off to eliminate getRegionLocation fail-and-retry time when writing kvs</span> |
| <span class="source-line-no">500</span><span id="line-500"> conf.setBoolean(HFileOutputFormat2.LOCALITY_SENSITIVE_CONF_KEY, false);</span> |
| <span class="source-line-no">501</span><span id="line-501"> Job job = new Job(conf);</span> |
| <span class="source-line-no">502</span><span id="line-502"> FileOutputFormat.setOutputPath(job, dir);</span> |
| <span class="source-line-no">503</span><span id="line-503"> context = createTestTaskAttemptContext(job);</span> |
| <span class="source-line-no">504</span><span id="line-504"> HFileOutputFormat2 hof = new HFileOutputFormat2();</span> |
| <span class="source-line-no">505</span><span id="line-505"> writer = hof.getRecordWriter(context);</span> |
| <span class="source-line-no">506</span><span id="line-506"> final byte[] b = Bytes.toBytes("b");</span> |
| <span class="source-line-no">507</span><span id="line-507"></span> |
| <span class="source-line-no">508</span><span id="line-508"> List<Tag> tags = new ArrayList<>();</span> |
| <span class="source-line-no">509</span><span id="line-509"> tags.add(new ArrayBackedTag(TagType.TTL_TAG_TYPE, Bytes.toBytes(978670)));</span> |
| <span class="source-line-no">510</span><span id="line-510"> KeyValue kv = new KeyValue(b, b, b, HConstants.LATEST_TIMESTAMP, b, tags);</span> |
| <span class="source-line-no">511</span><span id="line-511"> writer.write(new ImmutableBytesWritable(), kv);</span> |
| <span class="source-line-no">512</span><span id="line-512"> writer.close(context);</span> |
| <span class="source-line-no">513</span><span id="line-513"> writer = null;</span> |
| <span class="source-line-no">514</span><span id="line-514"> FileSystem fs = dir.getFileSystem(conf);</span> |
| <span class="source-line-no">515</span><span id="line-515"> RemoteIterator<LocatedFileStatus> iterator = fs.listFiles(dir, true);</span> |
| <span class="source-line-no">516</span><span id="line-516"> while (iterator.hasNext()) {</span> |
| <span class="source-line-no">517</span><span id="line-517"> LocatedFileStatus keyFileStatus = iterator.next();</span> |
| <span class="source-line-no">518</span><span id="line-518"> HFile.Reader reader =</span> |
| <span class="source-line-no">519</span><span id="line-519"> HFile.createReader(fs, keyFileStatus.getPath(), new CacheConfig(conf), true, conf);</span> |
| <span class="source-line-no">520</span><span id="line-520"> HFileScanner scanner = reader.getScanner(conf, false, false, false);</span> |
| <span class="source-line-no">521</span><span id="line-521"> scanner.seekTo();</span> |
| <span class="source-line-no">522</span><span id="line-522"> ExtendedCell cell = scanner.getCell();</span> |
| <span class="source-line-no">523</span><span id="line-523"> List<Tag> tagsFromCell = PrivateCellUtil.getTags(cell);</span> |
| <span class="source-line-no">524</span><span id="line-524"> assertTrue(tagsFromCell.size() > 0);</span> |
| <span class="source-line-no">525</span><span id="line-525"> for (Tag tag : tagsFromCell) {</span> |
| <span class="source-line-no">526</span><span id="line-526"> assertTrue(tag.getType() == TagType.TTL_TAG_TYPE);</span> |
| <span class="source-line-no">527</span><span id="line-527"> }</span> |
| <span class="source-line-no">528</span><span id="line-528"> }</span> |
| <span class="source-line-no">529</span><span id="line-529"> } finally {</span> |
| <span class="source-line-no">530</span><span id="line-530"> if (writer != null && context != null) writer.close(context);</span> |
| <span class="source-line-no">531</span><span id="line-531"> dir.getFileSystem(conf).delete(dir, true);</span> |
| <span class="source-line-no">532</span><span id="line-532"> }</span> |
| <span class="source-line-no">533</span><span id="line-533"> }</span> |
| <span class="source-line-no">534</span><span id="line-534"></span> |
| <span class="source-line-no">535</span><span id="line-535"> @Ignore("Goes zombie too frequently; needs work. See HBASE-14563")</span> |
| <span class="source-line-no">536</span><span id="line-536"> @Test</span> |
| <span class="source-line-no">537</span><span id="line-537"> public void testJobConfiguration() throws Exception {</span> |
| <span class="source-line-no">538</span><span id="line-538"> Configuration conf = new Configuration(this.util.getConfiguration());</span> |
| <span class="source-line-no">539</span><span id="line-539"> conf.set(HConstants.TEMPORARY_FS_DIRECTORY_KEY,</span> |
| <span class="source-line-no">540</span><span id="line-540"> util.getDataTestDir("testJobConfiguration").toString());</span> |
| <span class="source-line-no">541</span><span id="line-541"> Job job = new Job(conf);</span> |
| <span class="source-line-no">542</span><span id="line-542"> job.setWorkingDirectory(util.getDataTestDir("testJobConfiguration"));</span> |
| <span class="source-line-no">543</span><span id="line-543"> Table table = Mockito.mock(Table.class);</span> |
| <span class="source-line-no">544</span><span id="line-544"> RegionLocator regionLocator = Mockito.mock(RegionLocator.class);</span> |
| <span class="source-line-no">545</span><span id="line-545"> setupMockStartKeys(regionLocator);</span> |
| <span class="source-line-no">546</span><span id="line-546"> setupMockTableName(regionLocator);</span> |
| <span class="source-line-no">547</span><span id="line-547"> HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);</span> |
| <span class="source-line-no">548</span><span id="line-548"> assertEquals(job.getNumReduceTasks(), 4);</span> |
| <span class="source-line-no">549</span><span id="line-549"> }</span> |
| <span class="source-line-no">550</span><span id="line-550"></span> |
| <span class="source-line-no">551</span><span id="line-551"> private byte[][] generateRandomStartKeys(int numKeys) {</span> |
| <span class="source-line-no">552</span><span id="line-552"> Random random = ThreadLocalRandom.current();</span> |
| <span class="source-line-no">553</span><span id="line-553"> byte[][] ret = new byte[numKeys][];</span> |
| <span class="source-line-no">554</span><span id="line-554"> // first region start key is always empty</span> |
| <span class="source-line-no">555</span><span id="line-555"> ret[0] = HConstants.EMPTY_BYTE_ARRAY;</span> |
| <span class="source-line-no">556</span><span id="line-556"> for (int i = 1; i < numKeys; i++) {</span> |
| <span class="source-line-no">557</span><span id="line-557"> ret[i] =</span> |
| <span class="source-line-no">558</span><span id="line-558"> PerformanceEvaluation.generateData(random, PerformanceEvaluation.DEFAULT_VALUE_LENGTH);</span> |
| <span class="source-line-no">559</span><span id="line-559"> }</span> |
| <span class="source-line-no">560</span><span id="line-560"> return ret;</span> |
| <span class="source-line-no">561</span><span id="line-561"> }</span> |
| <span class="source-line-no">562</span><span id="line-562"></span> |
| <span class="source-line-no">563</span><span id="line-563"> private byte[][] generateRandomSplitKeys(int numKeys) {</span> |
| <span class="source-line-no">564</span><span id="line-564"> Random random = ThreadLocalRandom.current();</span> |
| <span class="source-line-no">565</span><span id="line-565"> byte[][] ret = new byte[numKeys][];</span> |
| <span class="source-line-no">566</span><span id="line-566"> for (int i = 0; i < numKeys; i++) {</span> |
| <span class="source-line-no">567</span><span id="line-567"> ret[i] =</span> |
| <span class="source-line-no">568</span><span id="line-568"> PerformanceEvaluation.generateData(random, PerformanceEvaluation.DEFAULT_VALUE_LENGTH);</span> |
| <span class="source-line-no">569</span><span id="line-569"> }</span> |
| <span class="source-line-no">570</span><span id="line-570"> return ret;</span> |
| <span class="source-line-no">571</span><span id="line-571"> }</span> |
| <span class="source-line-no">572</span><span id="line-572"></span> |
| <span class="source-line-no">573</span><span id="line-573"> @Ignore("Goes zombie too frequently; needs work. See HBASE-14563")</span> |
| <span class="source-line-no">574</span><span id="line-574"> @Test</span> |
| <span class="source-line-no">575</span><span id="line-575"> public void testMRIncrementalLoad() throws Exception {</span> |
| <span class="source-line-no">576</span><span id="line-576"> LOG.info("\nStarting test testMRIncrementalLoad\n");</span> |
| <span class="source-line-no">577</span><span id="line-577"> doIncrementalLoadTest(false, false, false, "testMRIncrementalLoad");</span> |
| <span class="source-line-no">578</span><span id="line-578"> }</span> |
| <span class="source-line-no">579</span><span id="line-579"></span> |
| <span class="source-line-no">580</span><span id="line-580"> @Ignore("Goes zombie too frequently; needs work. See HBASE-14563")</span> |
| <span class="source-line-no">581</span><span id="line-581"> @Test</span> |
| <span class="source-line-no">582</span><span id="line-582"> public void testMRIncrementalLoadWithSplit() throws Exception {</span> |
| <span class="source-line-no">583</span><span id="line-583"> LOG.info("\nStarting test testMRIncrementalLoadWithSplit\n");</span> |
| <span class="source-line-no">584</span><span id="line-584"> doIncrementalLoadTest(true, false, false, "testMRIncrementalLoadWithSplit");</span> |
| <span class="source-line-no">585</span><span id="line-585"> }</span> |
| <span class="source-line-no">586</span><span id="line-586"></span> |
| <span class="source-line-no">587</span><span id="line-587"> /**</span> |
| <span class="source-line-no">588</span><span id="line-588"> * Test for HFileOutputFormat2.LOCALITY_SENSITIVE_CONF_KEY = true This test could only check the</span> |
| <span class="source-line-no">589</span><span id="line-589"> * correctness of original logic if LOCALITY_SENSITIVE_CONF_KEY is set to true. Because</span> |
| <span class="source-line-no">590</span><span id="line-590"> * MiniHBaseCluster always run with single hostname (and different ports), it's not possible to</span> |
| <span class="source-line-no">591</span><span id="line-591"> * check the region locality by comparing region locations and DN hostnames. When MiniHBaseCluster</span> |
| <span class="source-line-no">592</span><span id="line-592"> * supports explicit hostnames parameter (just like MiniDFSCluster does), we could test region</span> |
| <span class="source-line-no">593</span><span id="line-593"> * locality features more easily.</span> |
| <span class="source-line-no">594</span><span id="line-594"> */</span> |
| <span class="source-line-no">595</span><span id="line-595"> @Ignore("Goes zombie too frequently; needs work. See HBASE-14563")</span> |
| <span class="source-line-no">596</span><span id="line-596"> @Test</span> |
| <span class="source-line-no">597</span><span id="line-597"> public void testMRIncrementalLoadWithLocality() throws Exception {</span> |
| <span class="source-line-no">598</span><span id="line-598"> LOG.info("\nStarting test testMRIncrementalLoadWithLocality\n");</span> |
| <span class="source-line-no">599</span><span id="line-599"> doIncrementalLoadTest(false, true, false, "testMRIncrementalLoadWithLocality1");</span> |
| <span class="source-line-no">600</span><span id="line-600"> doIncrementalLoadTest(true, true, false, "testMRIncrementalLoadWithLocality2");</span> |
| <span class="source-line-no">601</span><span id="line-601"> }</span> |
| <span class="source-line-no">602</span><span id="line-602"></span> |
| <span class="source-line-no">603</span><span id="line-603"> // @Ignore("Wahtevs")</span> |
| <span class="source-line-no">604</span><span id="line-604"> @Test</span> |
| <span class="source-line-no">605</span><span id="line-605"> public void testMRIncrementalLoadWithPutSortReducer() throws Exception {</span> |
| <span class="source-line-no">606</span><span id="line-606"> LOG.info("\nStarting test testMRIncrementalLoadWithPutSortReducer\n");</span> |
| <span class="source-line-no">607</span><span id="line-607"> doIncrementalLoadTest(false, false, true, "testMRIncrementalLoadWithPutSortReducer");</span> |
| <span class="source-line-no">608</span><span id="line-608"> }</span> |
| <span class="source-line-no">609</span><span id="line-609"></span> |
| <span class="source-line-no">610</span><span id="line-610"> private void doIncrementalLoadTest(boolean shouldChangeRegions, boolean shouldKeepLocality,</span> |
| <span class="source-line-no">611</span><span id="line-611"> boolean putSortReducer, String tableStr) throws Exception {</span> |
| <span class="source-line-no">612</span><span id="line-612"> doIncrementalLoadTest(shouldChangeRegions, shouldKeepLocality, putSortReducer,</span> |
| <span class="source-line-no">613</span><span id="line-613"> Arrays.asList(tableStr));</span> |
| <span class="source-line-no">614</span><span id="line-614"> }</span> |
| <span class="source-line-no">615</span><span id="line-615"></span> |
| <span class="source-line-no">616</span><span id="line-616"> @Test</span> |
| <span class="source-line-no">617</span><span id="line-617"> public void testMultiMRIncrementalLoadWithPutSortReducer() throws Exception {</span> |
| <span class="source-line-no">618</span><span id="line-618"> LOG.info("\nStarting test testMultiMRIncrementalLoadWithPutSortReducer\n");</span> |
| <span class="source-line-no">619</span><span id="line-619"> doIncrementalLoadTest(false, false, true,</span> |
| <span class="source-line-no">620</span><span id="line-620"> Arrays.stream(TABLE_NAMES).map(TableName::getNameAsString).collect(Collectors.toList()));</span> |
| <span class="source-line-no">621</span><span id="line-621"> }</span> |
| <span class="source-line-no">622</span><span id="line-622"></span> |
| <span class="source-line-no">623</span><span id="line-623"> private void doIncrementalLoadTest(boolean shouldChangeRegions, boolean shouldKeepLocality,</span> |
| <span class="source-line-no">624</span><span id="line-624"> boolean putSortReducer, List<String> tableStr) throws Exception {</span> |
| <span class="source-line-no">625</span><span id="line-625"> util = new HBaseTestingUtil();</span> |
| <span class="source-line-no">626</span><span id="line-626"> Configuration conf = util.getConfiguration();</span> |
| <span class="source-line-no">627</span><span id="line-627"> conf.setBoolean(MultiTableHFileOutputFormat.LOCALITY_SENSITIVE_CONF_KEY, shouldKeepLocality);</span> |
| <span class="source-line-no">628</span><span id="line-628"> int hostCount = 1;</span> |
| <span class="source-line-no">629</span><span id="line-629"> int regionNum = 5;</span> |
| <span class="source-line-no">630</span><span id="line-630"> if (shouldKeepLocality) {</span> |
| <span class="source-line-no">631</span><span id="line-631"> // We should change host count higher than hdfs replica count when MiniHBaseCluster supports</span> |
| <span class="source-line-no">632</span><span id="line-632"> // explicit hostnames parameter just like MiniDFSCluster does.</span> |
| <span class="source-line-no">633</span><span id="line-633"> hostCount = 3;</span> |
| <span class="source-line-no">634</span><span id="line-634"> regionNum = 20;</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"> String[] hostnames = new String[hostCount];</span> |
| <span class="source-line-no">638</span><span id="line-638"> for (int i = 0; i < hostCount; ++i) {</span> |
| <span class="source-line-no">639</span><span id="line-639"> hostnames[i] = "datanode_" + i;</span> |
| <span class="source-line-no">640</span><span id="line-640"> }</span> |
| <span class="source-line-no">641</span><span id="line-641"> StartTestingClusterOption option = StartTestingClusterOption.builder()</span> |
| <span class="source-line-no">642</span><span id="line-642"> .numRegionServers(hostCount).dataNodeHosts(hostnames).build();</span> |
| <span class="source-line-no">643</span><span id="line-643"> util.startMiniCluster(option);</span> |
| <span class="source-line-no">644</span><span id="line-644"></span> |
| <span class="source-line-no">645</span><span id="line-645"> Map<String, Table> allTables = new HashMap<>(tableStr.size());</span> |
| <span class="source-line-no">646</span><span id="line-646"> List<HFileOutputFormat2.TableInfo> tableInfo = new ArrayList<>(tableStr.size());</span> |
| <span class="source-line-no">647</span><span id="line-647"> boolean writeMultipleTables = tableStr.size() > 1;</span> |
| <span class="source-line-no">648</span><span id="line-648"> for (String tableStrSingle : tableStr) {</span> |
| <span class="source-line-no">649</span><span id="line-649"> byte[][] splitKeys = generateRandomSplitKeys(regionNum - 1);</span> |
| <span class="source-line-no">650</span><span id="line-650"> TableName tableName = TableName.valueOf(tableStrSingle);</span> |
| <span class="source-line-no">651</span><span id="line-651"> Table table = util.createTable(tableName, FAMILIES, splitKeys);</span> |
| <span class="source-line-no">652</span><span id="line-652"></span> |
| <span class="source-line-no">653</span><span id="line-653"> RegionLocator r = util.getConnection().getRegionLocator(tableName);</span> |
| <span class="source-line-no">654</span><span id="line-654"> assertEquals("Should start with empty table", 0, util.countRows(table));</span> |
| <span class="source-line-no">655</span><span id="line-655"> int numRegions = r.getStartKeys().length;</span> |
| <span class="source-line-no">656</span><span id="line-656"> assertEquals("Should make " + regionNum + " regions", numRegions, regionNum);</span> |
| <span class="source-line-no">657</span><span id="line-657"></span> |
| <span class="source-line-no">658</span><span id="line-658"> allTables.put(tableStrSingle, table);</span> |
| <span class="source-line-no">659</span><span id="line-659"> tableInfo.add(new HFileOutputFormat2.TableInfo(table.getDescriptor(), r));</span> |
| <span class="source-line-no">660</span><span id="line-660"> }</span> |
| <span class="source-line-no">661</span><span id="line-661"> Path testDir = util.getDataTestDirOnTestFS("testLocalMRIncrementalLoad");</span> |
| <span class="source-line-no">662</span><span id="line-662"> // Generate the bulk load files</span> |
| <span class="source-line-no">663</span><span id="line-663"> runIncrementalPELoad(conf, tableInfo, testDir, putSortReducer);</span> |
| <span class="source-line-no">664</span><span id="line-664"> if (writeMultipleTables) {</span> |
| <span class="source-line-no">665</span><span id="line-665"> testDir = new Path(testDir, "default");</span> |
| <span class="source-line-no">666</span><span id="line-666"> }</span> |
| <span class="source-line-no">667</span><span id="line-667"></span> |
| <span class="source-line-no">668</span><span id="line-668"> for (Table tableSingle : allTables.values()) {</span> |
| <span class="source-line-no">669</span><span id="line-669"> // This doesn't write into the table, just makes files</span> |
| <span class="source-line-no">670</span><span id="line-670"> assertEquals("HFOF should not touch actual table", 0, util.countRows(tableSingle));</span> |
| <span class="source-line-no">671</span><span id="line-671"> }</span> |
| <span class="source-line-no">672</span><span id="line-672"> int numTableDirs = 0;</span> |
| <span class="source-line-no">673</span><span id="line-673"> FileStatus[] fss = testDir.getFileSystem(conf).listStatus(testDir);</span> |
| <span class="source-line-no">674</span><span id="line-674"> for (FileStatus tf : fss) {</span> |
| <span class="source-line-no">675</span><span id="line-675"> Path tablePath = testDir;</span> |
| <span class="source-line-no">676</span><span id="line-676"> if (writeMultipleTables) {</span> |
| <span class="source-line-no">677</span><span id="line-677"> if (allTables.containsKey(tf.getPath().getName())) {</span> |
| <span class="source-line-no">678</span><span id="line-678"> ++numTableDirs;</span> |
| <span class="source-line-no">679</span><span id="line-679"> tablePath = tf.getPath();</span> |
| <span class="source-line-no">680</span><span id="line-680"> } else {</span> |
| <span class="source-line-no">681</span><span id="line-681"> continue;</span> |
| <span class="source-line-no">682</span><span id="line-682"> }</span> |
| <span class="source-line-no">683</span><span id="line-683"> }</span> |
| <span class="source-line-no">684</span><span id="line-684"></span> |
| <span class="source-line-no">685</span><span id="line-685"> // Make sure that a directory was created for every CF</span> |
| <span class="source-line-no">686</span><span id="line-686"> int dir = 0;</span> |
| <span class="source-line-no">687</span><span id="line-687"> fss = tablePath.getFileSystem(conf).listStatus(tablePath);</span> |
| <span class="source-line-no">688</span><span id="line-688"> for (FileStatus f : fss) {</span> |
| <span class="source-line-no">689</span><span id="line-689"> for (byte[] family : FAMILIES) {</span> |
| <span class="source-line-no">690</span><span id="line-690"> if (Bytes.toString(family).equals(f.getPath().getName())) {</span> |
| <span class="source-line-no">691</span><span id="line-691"> ++dir;</span> |
| <span class="source-line-no">692</span><span id="line-692"> }</span> |
| <span class="source-line-no">693</span><span id="line-693"> }</span> |
| <span class="source-line-no">694</span><span id="line-694"> }</span> |
| <span class="source-line-no">695</span><span id="line-695"> assertEquals("Column family not found in FS.", FAMILIES.length, dir);</span> |
| <span class="source-line-no">696</span><span id="line-696"> }</span> |
| <span class="source-line-no">697</span><span id="line-697"> if (writeMultipleTables) {</span> |
| <span class="source-line-no">698</span><span id="line-698"> assertEquals("Dir for all input tables not created", numTableDirs, allTables.size());</span> |
| <span class="source-line-no">699</span><span id="line-699"> }</span> |
| <span class="source-line-no">700</span><span id="line-700"></span> |
| <span class="source-line-no">701</span><span id="line-701"> Admin admin = util.getConnection().getAdmin();</span> |
| <span class="source-line-no">702</span><span id="line-702"> try {</span> |
| <span class="source-line-no">703</span><span id="line-703"> // handle the split case</span> |
| <span class="source-line-no">704</span><span id="line-704"> if (shouldChangeRegions) {</span> |
| <span class="source-line-no">705</span><span id="line-705"> Table chosenTable = allTables.values().iterator().next();</span> |
| <span class="source-line-no">706</span><span id="line-706"> // Choose a semi-random table if multiple tables are available</span> |
| <span class="source-line-no">707</span><span id="line-707"> LOG.info("Changing regions in table " + chosenTable.getName().getNameAsString());</span> |
| <span class="source-line-no">708</span><span id="line-708"> admin.disableTable(chosenTable.getName());</span> |
| <span class="source-line-no">709</span><span id="line-709"> util.waitUntilNoRegionsInTransition();</span> |
| <span class="source-line-no">710</span><span id="line-710"></span> |
| <span class="source-line-no">711</span><span id="line-711"> util.deleteTable(chosenTable.getName());</span> |
| <span class="source-line-no">712</span><span id="line-712"> byte[][] newSplitKeys = generateRandomSplitKeys(14);</span> |
| <span class="source-line-no">713</span><span id="line-713"> Table table = util.createTable(chosenTable.getName(), FAMILIES, newSplitKeys);</span> |
| <span class="source-line-no">714</span><span id="line-714"></span> |
| <span class="source-line-no">715</span><span id="line-715"> while (</span> |
| <span class="source-line-no">716</span><span id="line-716"> util.getConnection().getRegionLocator(chosenTable.getName()).getAllRegionLocations()</span> |
| <span class="source-line-no">717</span><span id="line-717"> .size() != 15 || !admin.isTableAvailable(table.getName())</span> |
| <span class="source-line-no">718</span><span id="line-718"> ) {</span> |
| <span class="source-line-no">719</span><span id="line-719"> Thread.sleep(200);</span> |
| <span class="source-line-no">720</span><span id="line-720"> LOG.info("Waiting for new region assignment to happen");</span> |
| <span class="source-line-no">721</span><span id="line-721"> }</span> |
| <span class="source-line-no">722</span><span id="line-722"> }</span> |
| <span class="source-line-no">723</span><span id="line-723"></span> |
| <span class="source-line-no">724</span><span id="line-724"> // Perform the actual load</span> |
| <span class="source-line-no">725</span><span id="line-725"> for (HFileOutputFormat2.TableInfo singleTableInfo : tableInfo) {</span> |
| <span class="source-line-no">726</span><span id="line-726"> Path tableDir = testDir;</span> |
| <span class="source-line-no">727</span><span id="line-727"> String tableNameStr = singleTableInfo.getTableDescriptor().getTableName().getNameAsString();</span> |
| <span class="source-line-no">728</span><span id="line-728"> LOG.info("Running BulkLoadHFiles on table" + tableNameStr);</span> |
| <span class="source-line-no">729</span><span id="line-729"> if (writeMultipleTables) {</span> |
| <span class="source-line-no">730</span><span id="line-730"> tableDir = new Path(testDir, tableNameStr);</span> |
| <span class="source-line-no">731</span><span id="line-731"> }</span> |
| <span class="source-line-no">732</span><span id="line-732"> Table currentTable = allTables.get(tableNameStr);</span> |
| <span class="source-line-no">733</span><span id="line-733"> TableName currentTableName = currentTable.getName();</span> |
| <span class="source-line-no">734</span><span id="line-734"> BulkLoadHFiles.create(conf).bulkLoad(currentTableName, tableDir);</span> |
| <span class="source-line-no">735</span><span id="line-735"></span> |
| <span class="source-line-no">736</span><span id="line-736"> // Ensure data shows up</span> |
| <span class="source-line-no">737</span><span id="line-737"> int expectedRows = 0;</span> |
| <span class="source-line-no">738</span><span id="line-738"> if (putSortReducer) {</span> |
| <span class="source-line-no">739</span><span id="line-739"> // no rows should be extracted</span> |
| <span class="source-line-no">740</span><span id="line-740"> assertEquals("BulkLoadHFiles should put expected data in table", expectedRows,</span> |
| <span class="source-line-no">741</span><span id="line-741"> util.countRows(currentTable));</span> |
| <span class="source-line-no">742</span><span id="line-742"> } else {</span> |
| <span class="source-line-no">743</span><span id="line-743"> expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;</span> |
| <span class="source-line-no">744</span><span id="line-744"> assertEquals("BulkLoadHFiles should put expected data in table", expectedRows,</span> |
| <span class="source-line-no">745</span><span id="line-745"> util.countRows(currentTable));</span> |
| <span class="source-line-no">746</span><span id="line-746"> Scan scan = new Scan();</span> |
| <span class="source-line-no">747</span><span id="line-747"> ResultScanner results = currentTable.getScanner(scan);</span> |
| <span class="source-line-no">748</span><span id="line-748"> for (Result res : results) {</span> |
| <span class="source-line-no">749</span><span id="line-749"> assertEquals(FAMILIES.length, res.rawCells().length);</span> |
| <span class="source-line-no">750</span><span id="line-750"> Cell first = res.rawCells()[0];</span> |
| <span class="source-line-no">751</span><span id="line-751"> for (Cell kv : res.rawCells()) {</span> |
| <span class="source-line-no">752</span><span id="line-752"> assertTrue(CellUtil.matchingRows(first, kv));</span> |
| <span class="source-line-no">753</span><span id="line-753"> assertTrue(Bytes.equals(CellUtil.cloneValue(first), CellUtil.cloneValue(kv)));</span> |
| <span class="source-line-no">754</span><span id="line-754"> }</span> |
| <span class="source-line-no">755</span><span id="line-755"> }</span> |
| <span class="source-line-no">756</span><span id="line-756"> results.close();</span> |
| <span class="source-line-no">757</span><span id="line-757"> }</span> |
| <span class="source-line-no">758</span><span id="line-758"> String tableDigestBefore = util.checksumRows(currentTable);</span> |
| <span class="source-line-no">759</span><span id="line-759"> // Check region locality</span> |
| <span class="source-line-no">760</span><span id="line-760"> HDFSBlocksDistribution hbd = new HDFSBlocksDistribution();</span> |
| <span class="source-line-no">761</span><span id="line-761"> for (HRegion region : util.getHBaseCluster().getRegions(currentTableName)) {</span> |
| <span class="source-line-no">762</span><span id="line-762"> hbd.add(region.getHDFSBlocksDistribution());</span> |
| <span class="source-line-no">763</span><span id="line-763"> }</span> |
| <span class="source-line-no">764</span><span id="line-764"> for (String hostname : hostnames) {</span> |
| <span class="source-line-no">765</span><span id="line-765"> float locality = hbd.getBlockLocalityIndex(hostname);</span> |
| <span class="source-line-no">766</span><span id="line-766"> LOG.info("locality of [" + hostname + "]: " + locality);</span> |
| <span class="source-line-no">767</span><span id="line-767"> assertEquals(100, (int) (locality * 100));</span> |
| <span class="source-line-no">768</span><span id="line-768"> }</span> |
| <span class="source-line-no">769</span><span id="line-769"></span> |
| <span class="source-line-no">770</span><span id="line-770"> // Cause regions to reopen</span> |
| <span class="source-line-no">771</span><span id="line-771"> admin.disableTable(currentTableName);</span> |
| <span class="source-line-no">772</span><span id="line-772"> while (!admin.isTableDisabled(currentTableName)) {</span> |
| <span class="source-line-no">773</span><span id="line-773"> Thread.sleep(200);</span> |
| <span class="source-line-no">774</span><span id="line-774"> LOG.info("Waiting for table to disable");</span> |
| <span class="source-line-no">775</span><span id="line-775"> }</span> |
| <span class="source-line-no">776</span><span id="line-776"> admin.enableTable(currentTableName);</span> |
| <span class="source-line-no">777</span><span id="line-777"> util.waitTableAvailable(currentTableName);</span> |
| <span class="source-line-no">778</span><span id="line-778"> assertEquals("Data should remain after reopening of regions", tableDigestBefore,</span> |
| <span class="source-line-no">779</span><span id="line-779"> util.checksumRows(currentTable));</span> |
| <span class="source-line-no">780</span><span id="line-780"> }</span> |
| <span class="source-line-no">781</span><span id="line-781"> } finally {</span> |
| <span class="source-line-no">782</span><span id="line-782"> for (HFileOutputFormat2.TableInfo tableInfoSingle : tableInfo) {</span> |
| <span class="source-line-no">783</span><span id="line-783"> tableInfoSingle.getRegionLocator().close();</span> |
| <span class="source-line-no">784</span><span id="line-784"> }</span> |
| <span class="source-line-no">785</span><span id="line-785"> for (Entry<String, Table> singleTable : allTables.entrySet()) {</span> |
| <span class="source-line-no">786</span><span id="line-786"> singleTable.getValue().close();</span> |
| <span class="source-line-no">787</span><span id="line-787"> util.deleteTable(singleTable.getValue().getName());</span> |
| <span class="source-line-no">788</span><span id="line-788"> }</span> |
| <span class="source-line-no">789</span><span id="line-789"> testDir.getFileSystem(conf).delete(testDir, true);</span> |
| <span class="source-line-no">790</span><span id="line-790"> util.shutdownMiniCluster();</span> |
| <span class="source-line-no">791</span><span id="line-791"> }</span> |
| <span class="source-line-no">792</span><span id="line-792"> }</span> |
| <span class="source-line-no">793</span><span id="line-793"></span> |
| <span class="source-line-no">794</span><span id="line-794"> private void runIncrementalPELoad(Configuration conf,</span> |
| <span class="source-line-no">795</span><span id="line-795"> List<HFileOutputFormat2.TableInfo> tableInfo, Path outDir, boolean putSortReducer)</span> |
| <span class="source-line-no">796</span><span id="line-796"> throws IOException, InterruptedException, ClassNotFoundException {</span> |
| <span class="source-line-no">797</span><span id="line-797"> Job job = new Job(conf, "testLocalMRIncrementalLoad");</span> |
| <span class="source-line-no">798</span><span id="line-798"> job.setWorkingDirectory(util.getDataTestDirOnTestFS("runIncrementalPELoad"));</span> |
| <span class="source-line-no">799</span><span id="line-799"> job.getConfiguration().setStrings("io.serializations", conf.get("io.serializations"),</span> |
| <span class="source-line-no">800</span><span id="line-800"> MutationSerialization.class.getName(), ResultSerialization.class.getName(),</span> |
| <span class="source-line-no">801</span><span id="line-801"> CellSerialization.class.getName());</span> |
| <span class="source-line-no">802</span><span id="line-802"> setupRandomGeneratorMapper(job, putSortReducer);</span> |
| <span class="source-line-no">803</span><span id="line-803"> if (tableInfo.size() > 1) {</span> |
| <span class="source-line-no">804</span><span id="line-804"> MultiTableHFileOutputFormat.configureIncrementalLoad(job, tableInfo);</span> |
| <span class="source-line-no">805</span><span id="line-805"> int sum = 0;</span> |
| <span class="source-line-no">806</span><span id="line-806"> for (HFileOutputFormat2.TableInfo tableInfoSingle : tableInfo) {</span> |
| <span class="source-line-no">807</span><span id="line-807"> sum += tableInfoSingle.getRegionLocator().getAllRegionLocations().size();</span> |
| <span class="source-line-no">808</span><span id="line-808"> }</span> |
| <span class="source-line-no">809</span><span id="line-809"> assertEquals(sum, job.getNumReduceTasks());</span> |
| <span class="source-line-no">810</span><span id="line-810"> } else {</span> |
| <span class="source-line-no">811</span><span id="line-811"> RegionLocator regionLocator = tableInfo.get(0).getRegionLocator();</span> |
| <span class="source-line-no">812</span><span id="line-812"> HFileOutputFormat2.configureIncrementalLoad(job, tableInfo.get(0).getTableDescriptor(),</span> |
| <span class="source-line-no">813</span><span id="line-813"> regionLocator);</span> |
| <span class="source-line-no">814</span><span id="line-814"> assertEquals(regionLocator.getAllRegionLocations().size(), job.getNumReduceTasks());</span> |
| <span class="source-line-no">815</span><span id="line-815"> }</span> |
| <span class="source-line-no">816</span><span id="line-816"></span> |
| <span class="source-line-no">817</span><span id="line-817"> FileOutputFormat.setOutputPath(job, outDir);</span> |
| <span class="source-line-no">818</span><span id="line-818"></span> |
| <span class="source-line-no">819</span><span id="line-819"> assertFalse(util.getTestFileSystem().exists(outDir));</span> |
| <span class="source-line-no">820</span><span id="line-820"></span> |
| <span class="source-line-no">821</span><span id="line-821"> assertTrue(job.waitForCompletion(true));</span> |
| <span class="source-line-no">822</span><span id="line-822"> }</span> |
| <span class="source-line-no">823</span><span id="line-823"></span> |
| <span class="source-line-no">824</span><span id="line-824"> /**</span> |
| <span class="source-line-no">825</span><span id="line-825"> * Test for {@link HFileOutputFormat2#createFamilyCompressionMap(Configuration)}. Tests that the</span> |
| <span class="source-line-no">826</span><span id="line-826"> * family compression map is correctly serialized into and deserialized from configuration</span> |
| <span class="source-line-no">827</span><span id="line-827"> */</span> |
| <span class="source-line-no">828</span><span id="line-828"> @Ignore("Goes zombie too frequently; needs work. See HBASE-14563")</span> |
| <span class="source-line-no">829</span><span id="line-829"> @Test</span> |
| <span class="source-line-no">830</span><span id="line-830"> public void testSerializeDeserializeFamilyCompressionMap() throws IOException {</span> |
| <span class="source-line-no">831</span><span id="line-831"> for (int numCfs = 0; numCfs <= 3; numCfs++) {</span> |
| <span class="source-line-no">832</span><span id="line-832"> Configuration conf = new Configuration(this.util.getConfiguration());</span> |
| <span class="source-line-no">833</span><span id="line-833"> Map<String, Compression.Algorithm> familyToCompression =</span> |
| <span class="source-line-no">834</span><span id="line-834"> getMockColumnFamiliesForCompression(numCfs);</span> |
| <span class="source-line-no">835</span><span id="line-835"> Table table = Mockito.mock(Table.class);</span> |
| <span class="source-line-no">836</span><span id="line-836"> setupMockColumnFamiliesForCompression(table, familyToCompression);</span> |
| <span class="source-line-no">837</span><span id="line-837"> conf.set(HFileOutputFormat2.COMPRESSION_FAMILIES_CONF_KEY,</span> |
| <span class="source-line-no">838</span><span id="line-838"> HFileOutputFormat2.serializeColumnFamilyAttribute(HFileOutputFormat2.compressionDetails,</span> |
| <span class="source-line-no">839</span><span id="line-839"> Arrays.asList(table.getDescriptor())));</span> |
| <span class="source-line-no">840</span><span id="line-840"></span> |
| <span class="source-line-no">841</span><span id="line-841"> // read back family specific compression setting from the configuration</span> |
| <span class="source-line-no">842</span><span id="line-842"> Map<byte[], Algorithm> retrievedFamilyToCompressionMap =</span> |
| <span class="source-line-no">843</span><span id="line-843"> HFileOutputFormat2.createFamilyCompressionMap(conf);</span> |
| <span class="source-line-no">844</span><span id="line-844"></span> |
| <span class="source-line-no">845</span><span id="line-845"> // test that we have a value for all column families that matches with the</span> |
| <span class="source-line-no">846</span><span id="line-846"> // used mock values</span> |
| <span class="source-line-no">847</span><span id="line-847"> for (Entry<String, Algorithm> entry : familyToCompression.entrySet()) {</span> |
| <span class="source-line-no">848</span><span id="line-848"> assertEquals("Compression configuration incorrect for column family:" + entry.getKey(),</span> |
| <span class="source-line-no">849</span><span id="line-849"> entry.getValue(), retrievedFamilyToCompressionMap.get(Bytes.toBytes(entry.getKey())));</span> |
| <span class="source-line-no">850</span><span id="line-850"> }</span> |
| <span class="source-line-no">851</span><span id="line-851"> }</span> |
| <span class="source-line-no">852</span><span id="line-852"> }</span> |
| <span class="source-line-no">853</span><span id="line-853"></span> |
| <span class="source-line-no">854</span><span id="line-854"> private void setupMockColumnFamiliesForCompression(Table table,</span> |
| <span class="source-line-no">855</span><span id="line-855"> Map<String, Compression.Algorithm> familyToCompression) throws IOException {</span> |
| <span class="source-line-no">856</span><span id="line-856"></span> |
| <span class="source-line-no">857</span><span id="line-857"> TableDescriptorBuilder mockTableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAMES[0]);</span> |
| <span class="source-line-no">858</span><span id="line-858"> for (Entry<String, Compression.Algorithm> entry : familyToCompression.entrySet()) {</span> |
| <span class="source-line-no">859</span><span id="line-859"> ColumnFamilyDescriptor columnFamilyDescriptor = ColumnFamilyDescriptorBuilder</span> |
| <span class="source-line-no">860</span><span id="line-860"> .newBuilder(Bytes.toBytes(entry.getKey())).setMaxVersions(1)</span> |
| <span class="source-line-no">861</span><span id="line-861"> .setCompressionType(entry.getValue()).setBlockCacheEnabled(false).setTimeToLive(0).build();</span> |
| <span class="source-line-no">862</span><span id="line-862"></span> |
| <span class="source-line-no">863</span><span id="line-863"> mockTableDescriptor.setColumnFamily(columnFamilyDescriptor);</span> |
| <span class="source-line-no">864</span><span id="line-864"> }</span> |
| <span class="source-line-no">865</span><span id="line-865"> Mockito.doReturn(mockTableDescriptor.build()).when(table).getDescriptor();</span> |
| <span class="source-line-no">866</span><span id="line-866"> }</span> |
| <span class="source-line-no">867</span><span id="line-867"></span> |
| <span class="source-line-no">868</span><span id="line-868"> /**</span> |
| <span class="source-line-no">869</span><span id="line-869"> * @return a map from column family names to compression algorithms for testing column family</span> |
| <span class="source-line-no">870</span><span id="line-870"> * compression. Column family names have special characters</span> |
| <span class="source-line-no">871</span><span id="line-871"> */</span> |
| <span class="source-line-no">872</span><span id="line-872"> private Map<String, Compression.Algorithm> getMockColumnFamiliesForCompression(int numCfs) {</span> |
| <span class="source-line-no">873</span><span id="line-873"> Map<String, Compression.Algorithm> familyToCompression = new HashMap<>();</span> |
| <span class="source-line-no">874</span><span id="line-874"> // use column family names having special characters</span> |
| <span class="source-line-no">875</span><span id="line-875"> if (numCfs-- > 0) {</span> |
| <span class="source-line-no">876</span><span id="line-876"> familyToCompression.put("Family1!@#!@#&", Compression.Algorithm.LZO);</span> |
| <span class="source-line-no">877</span><span id="line-877"> }</span> |
| <span class="source-line-no">878</span><span id="line-878"> if (numCfs-- > 0) {</span> |
| <span class="source-line-no">879</span><span id="line-879"> familyToCompression.put("Family2=asdads&!AASD", Compression.Algorithm.SNAPPY);</span> |
| <span class="source-line-no">880</span><span id="line-880"> }</span> |
| <span class="source-line-no">881</span><span id="line-881"> if (numCfs-- > 0) {</span> |
| <span class="source-line-no">882</span><span id="line-882"> familyToCompression.put("Family2=asdads&!AASD", Compression.Algorithm.GZ);</span> |
| <span class="source-line-no">883</span><span id="line-883"> }</span> |
| <span class="source-line-no">884</span><span id="line-884"> if (numCfs-- > 0) {</span> |
| <span class="source-line-no">885</span><span id="line-885"> familyToCompression.put("Family3", Compression.Algorithm.NONE);</span> |
| <span class="source-line-no">886</span><span id="line-886"> }</span> |
| <span class="source-line-no">887</span><span id="line-887"> return familyToCompression;</span> |
| <span class="source-line-no">888</span><span id="line-888"> }</span> |
| <span class="source-line-no">889</span><span id="line-889"></span> |
| <span class="source-line-no">890</span><span id="line-890"> /**</span> |
| <span class="source-line-no">891</span><span id="line-891"> * Test for {@link HFileOutputFormat2#createFamilyBloomTypeMap(Configuration)}. Tests that the</span> |
| <span class="source-line-no">892</span><span id="line-892"> * family bloom type map is correctly serialized into and deserialized from configuration</span> |
| <span class="source-line-no">893</span><span id="line-893"> */</span> |
| <span class="source-line-no">894</span><span id="line-894"> @Ignore("Goes zombie too frequently; needs work. See HBASE-14563")</span> |
| <span class="source-line-no">895</span><span id="line-895"> @Test</span> |
| <span class="source-line-no">896</span><span id="line-896"> public void testSerializeDeserializeFamilyBloomTypeMap() throws IOException {</span> |
| <span class="source-line-no">897</span><span id="line-897"> for (int numCfs = 0; numCfs <= 2; numCfs++) {</span> |
| <span class="source-line-no">898</span><span id="line-898"> Configuration conf = new Configuration(this.util.getConfiguration());</span> |
| <span class="source-line-no">899</span><span id="line-899"> Map<String, BloomType> familyToBloomType = getMockColumnFamiliesForBloomType(numCfs);</span> |
| <span class="source-line-no">900</span><span id="line-900"> Table table = Mockito.mock(Table.class);</span> |
| <span class="source-line-no">901</span><span id="line-901"> setupMockColumnFamiliesForBloomType(table, familyToBloomType);</span> |
| <span class="source-line-no">902</span><span id="line-902"> conf.set(HFileOutputFormat2.BLOOM_TYPE_FAMILIES_CONF_KEY,</span> |
| <span class="source-line-no">903</span><span id="line-903"> HFileOutputFormat2.serializeColumnFamilyAttribute(HFileOutputFormat2.bloomTypeDetails,</span> |
| <span class="source-line-no">904</span><span id="line-904"> Arrays.asList(table.getDescriptor())));</span> |
| <span class="source-line-no">905</span><span id="line-905"></span> |
| <span class="source-line-no">906</span><span id="line-906"> // read back family specific data block encoding settings from the</span> |
| <span class="source-line-no">907</span><span id="line-907"> // configuration</span> |
| <span class="source-line-no">908</span><span id="line-908"> Map<byte[], BloomType> retrievedFamilyToBloomTypeMap =</span> |
| <span class="source-line-no">909</span><span id="line-909"> HFileOutputFormat2.createFamilyBloomTypeMap(conf);</span> |
| <span class="source-line-no">910</span><span id="line-910"></span> |
| <span class="source-line-no">911</span><span id="line-911"> // test that we have a value for all column families that matches with the</span> |
| <span class="source-line-no">912</span><span id="line-912"> // used mock values</span> |
| <span class="source-line-no">913</span><span id="line-913"> for (Entry<String, BloomType> entry : familyToBloomType.entrySet()) {</span> |
| <span class="source-line-no">914</span><span id="line-914"> assertEquals("BloomType configuration incorrect for column family:" + entry.getKey(),</span> |
| <span class="source-line-no">915</span><span id="line-915"> entry.getValue(), retrievedFamilyToBloomTypeMap.get(Bytes.toBytes(entry.getKey())));</span> |
| <span class="source-line-no">916</span><span id="line-916"> }</span> |
| <span class="source-line-no">917</span><span id="line-917"> }</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"> private void setupMockColumnFamiliesForBloomType(Table table,</span> |
| <span class="source-line-no">921</span><span id="line-921"> Map<String, BloomType> familyToDataBlockEncoding) throws IOException {</span> |
| <span class="source-line-no">922</span><span id="line-922"> TableDescriptorBuilder mockTableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAMES[0]);</span> |
| <span class="source-line-no">923</span><span id="line-923"> for (Entry<String, BloomType> entry : familyToDataBlockEncoding.entrySet()) {</span> |
| <span class="source-line-no">924</span><span id="line-924"> ColumnFamilyDescriptor columnFamilyDescriptor = ColumnFamilyDescriptorBuilder</span> |
| <span class="source-line-no">925</span><span id="line-925"> .newBuilder(Bytes.toBytes(entry.getKey())).setMaxVersions(1)</span> |
| <span class="source-line-no">926</span><span id="line-926"> .setBloomFilterType(entry.getValue()).setBlockCacheEnabled(false).setTimeToLive(0).build();</span> |
| <span class="source-line-no">927</span><span id="line-927"> mockTableDescriptor.setColumnFamily(columnFamilyDescriptor);</span> |
| <span class="source-line-no">928</span><span id="line-928"> }</span> |
| <span class="source-line-no">929</span><span id="line-929"> Mockito.doReturn(mockTableDescriptor).when(table).getDescriptor();</span> |
| <span class="source-line-no">930</span><span id="line-930"> }</span> |
| <span class="source-line-no">931</span><span id="line-931"></span> |
| <span class="source-line-no">932</span><span id="line-932"> /**</span> |
| <span class="source-line-no">933</span><span id="line-933"> * @return a map from column family names to compression algorithms for testing column family</span> |
| <span class="source-line-no">934</span><span id="line-934"> * compression. Column family names have special characters</span> |
| <span class="source-line-no">935</span><span id="line-935"> */</span> |
| <span class="source-line-no">936</span><span id="line-936"> private Map<String, BloomType> getMockColumnFamiliesForBloomType(int numCfs) {</span> |
| <span class="source-line-no">937</span><span id="line-937"> Map<String, BloomType> familyToBloomType = new HashMap<>();</span> |
| <span class="source-line-no">938</span><span id="line-938"> // use column family names having special characters</span> |
| <span class="source-line-no">939</span><span id="line-939"> if (numCfs-- > 0) {</span> |
| <span class="source-line-no">940</span><span id="line-940"> familyToBloomType.put("Family1!@#!@#&", BloomType.ROW);</span> |
| <span class="source-line-no">941</span><span id="line-941"> }</span> |
| <span class="source-line-no">942</span><span id="line-942"> if (numCfs-- > 0) {</span> |
| <span class="source-line-no">943</span><span id="line-943"> familyToBloomType.put("Family2=asdads&!AASD", BloomType.ROWCOL);</span> |
| <span class="source-line-no">944</span><span id="line-944"> }</span> |
| <span class="source-line-no">945</span><span id="line-945"> if (numCfs-- > 0) {</span> |
| <span class="source-line-no">946</span><span id="line-946"> familyToBloomType.put("Family3", BloomType.NONE);</span> |
| <span class="source-line-no">947</span><span id="line-947"> }</span> |
| <span class="source-line-no">948</span><span id="line-948"> return familyToBloomType;</span> |
| <span class="source-line-no">949</span><span id="line-949"> }</span> |
| <span class="source-line-no">950</span><span id="line-950"></span> |
| <span class="source-line-no">951</span><span id="line-951"> /**</span> |
| <span class="source-line-no">952</span><span id="line-952"> * Test for {@link HFileOutputFormat2#createFamilyBlockSizeMap(Configuration)}. Tests that the</span> |
| <span class="source-line-no">953</span><span id="line-953"> * family block size map is correctly serialized into and deserialized from configuration</span> |
| <span class="source-line-no">954</span><span id="line-954"> */</span> |
| <span class="source-line-no">955</span><span id="line-955"> @Ignore("Goes zombie too frequently; needs work. See HBASE-14563")</span> |
| <span class="source-line-no">956</span><span id="line-956"> @Test</span> |
| <span class="source-line-no">957</span><span id="line-957"> public void testSerializeDeserializeFamilyBlockSizeMap() throws IOException {</span> |
| <span class="source-line-no">958</span><span id="line-958"> for (int numCfs = 0; numCfs <= 3; numCfs++) {</span> |
| <span class="source-line-no">959</span><span id="line-959"> Configuration conf = new Configuration(this.util.getConfiguration());</span> |
| <span class="source-line-no">960</span><span id="line-960"> Map<String, Integer> familyToBlockSize = getMockColumnFamiliesForBlockSize(numCfs);</span> |
| <span class="source-line-no">961</span><span id="line-961"> Table table = Mockito.mock(Table.class);</span> |
| <span class="source-line-no">962</span><span id="line-962"> setupMockColumnFamiliesForBlockSize(table, familyToBlockSize);</span> |
| <span class="source-line-no">963</span><span id="line-963"> conf.set(HFileOutputFormat2.BLOCK_SIZE_FAMILIES_CONF_KEY,</span> |
| <span class="source-line-no">964</span><span id="line-964"> HFileOutputFormat2.serializeColumnFamilyAttribute(HFileOutputFormat2.blockSizeDetails,</span> |
| <span class="source-line-no">965</span><span id="line-965"> Arrays.asList(table.getDescriptor())));</span> |
| <span class="source-line-no">966</span><span id="line-966"></span> |
| <span class="source-line-no">967</span><span id="line-967"> // read back family specific data block encoding settings from the</span> |
| <span class="source-line-no">968</span><span id="line-968"> // configuration</span> |
| <span class="source-line-no">969</span><span id="line-969"> Map<byte[], Integer> retrievedFamilyToBlockSizeMap =</span> |
| <span class="source-line-no">970</span><span id="line-970"> HFileOutputFormat2.createFamilyBlockSizeMap(conf);</span> |
| <span class="source-line-no">971</span><span id="line-971"></span> |
| <span class="source-line-no">972</span><span id="line-972"> // test that we have a value for all column families that matches with the</span> |
| <span class="source-line-no">973</span><span id="line-973"> // used mock values</span> |
| <span class="source-line-no">974</span><span id="line-974"> for (Entry<String, Integer> entry : familyToBlockSize.entrySet()) {</span> |
| <span class="source-line-no">975</span><span id="line-975"> assertEquals("BlockSize configuration incorrect for column family:" + entry.getKey(),</span> |
| <span class="source-line-no">976</span><span id="line-976"> entry.getValue(), retrievedFamilyToBlockSizeMap.get(Bytes.toBytes(entry.getKey())));</span> |
| <span class="source-line-no">977</span><span id="line-977"> }</span> |
| <span class="source-line-no">978</span><span id="line-978"> }</span> |
| <span class="source-line-no">979</span><span id="line-979"> }</span> |
| <span class="source-line-no">980</span><span id="line-980"></span> |
| <span class="source-line-no">981</span><span id="line-981"> private void setupMockColumnFamiliesForBlockSize(Table table,</span> |
| <span class="source-line-no">982</span><span id="line-982"> Map<String, Integer> familyToDataBlockEncoding) throws IOException {</span> |
| <span class="source-line-no">983</span><span id="line-983"> TableDescriptorBuilder mockTableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAMES[0]);</span> |
| <span class="source-line-no">984</span><span id="line-984"> for (Entry<String, Integer> entry : familyToDataBlockEncoding.entrySet()) {</span> |
| <span class="source-line-no">985</span><span id="line-985"> ColumnFamilyDescriptor columnFamilyDescriptor =</span> |
| <span class="source-line-no">986</span><span id="line-986"> ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(entry.getKey())).setMaxVersions(1)</span> |
| <span class="source-line-no">987</span><span id="line-987"> .setBlocksize(entry.getValue()).setBlockCacheEnabled(false).setTimeToLive(0).build();</span> |
| <span class="source-line-no">988</span><span id="line-988"> mockTableDescriptor.setColumnFamily(columnFamilyDescriptor);</span> |
| <span class="source-line-no">989</span><span id="line-989"> }</span> |
| <span class="source-line-no">990</span><span id="line-990"> Mockito.doReturn(mockTableDescriptor).when(table).getDescriptor();</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"> /**</span> |
| <span class="source-line-no">994</span><span id="line-994"> * @return a map from column family names to compression algorithms for testing column family</span> |
| <span class="source-line-no">995</span><span id="line-995"> * compression. Column family names have special characters</span> |
| <span class="source-line-no">996</span><span id="line-996"> */</span> |
| <span class="source-line-no">997</span><span id="line-997"> private Map<String, Integer> getMockColumnFamiliesForBlockSize(int numCfs) {</span> |
| <span class="source-line-no">998</span><span id="line-998"> Map<String, Integer> familyToBlockSize = new HashMap<>();</span> |
| <span class="source-line-no">999</span><span id="line-999"> // use column family names having special characters</span> |
| <span class="source-line-no">1000</span><span id="line-1000"> if (numCfs-- > 0) {</span> |
| <span class="source-line-no">1001</span><span id="line-1001"> familyToBlockSize.put("Family1!@#!@#&", 1234);</span> |
| <span class="source-line-no">1002</span><span id="line-1002"> }</span> |
| <span class="source-line-no">1003</span><span id="line-1003"> if (numCfs-- > 0) {</span> |
| <span class="source-line-no">1004</span><span id="line-1004"> familyToBlockSize.put("Family2=asdads&!AASD", Integer.MAX_VALUE);</span> |
| <span class="source-line-no">1005</span><span id="line-1005"> }</span> |
| <span class="source-line-no">1006</span><span id="line-1006"> if (numCfs-- > 0) {</span> |
| <span class="source-line-no">1007</span><span id="line-1007"> familyToBlockSize.put("Family2=asdads&!AASD", Integer.MAX_VALUE);</span> |
| <span class="source-line-no">1008</span><span id="line-1008"> }</span> |
| <span class="source-line-no">1009</span><span id="line-1009"> if (numCfs-- > 0) {</span> |
| <span class="source-line-no">1010</span><span id="line-1010"> familyToBlockSize.put("Family3", 0);</span> |
| <span class="source-line-no">1011</span><span id="line-1011"> }</span> |
| <span class="source-line-no">1012</span><span id="line-1012"> return familyToBlockSize;</span> |
| <span class="source-line-no">1013</span><span id="line-1013"> }</span> |
| <span class="source-line-no">1014</span><span id="line-1014"></span> |
| <span class="source-line-no">1015</span><span id="line-1015"> /**</span> |
| <span class="source-line-no">1016</span><span id="line-1016"> * Test for {@link HFileOutputFormat2#createFamilyDataBlockEncodingMap(Configuration)}. Tests that</span> |
| <span class="source-line-no">1017</span><span id="line-1017"> * the family data block encoding map is correctly serialized into and deserialized from</span> |
| <span class="source-line-no">1018</span><span id="line-1018"> * configuration</span> |
| <span class="source-line-no">1019</span><span id="line-1019"> */</span> |
| <span class="source-line-no">1020</span><span id="line-1020"> @Ignore("Goes zombie too frequently; needs work. See HBASE-14563")</span> |
| <span class="source-line-no">1021</span><span id="line-1021"> @Test</span> |
| <span class="source-line-no">1022</span><span id="line-1022"> public void testSerializeDeserializeFamilyDataBlockEncodingMap() throws IOException {</span> |
| <span class="source-line-no">1023</span><span id="line-1023"> for (int numCfs = 0; numCfs <= 3; numCfs++) {</span> |
| <span class="source-line-no">1024</span><span id="line-1024"> Configuration conf = new Configuration(this.util.getConfiguration());</span> |
| <span class="source-line-no">1025</span><span id="line-1025"> Map<String, DataBlockEncoding> familyToDataBlockEncoding =</span> |
| <span class="source-line-no">1026</span><span id="line-1026"> getMockColumnFamiliesForDataBlockEncoding(numCfs);</span> |
| <span class="source-line-no">1027</span><span id="line-1027"> Table table = Mockito.mock(Table.class);</span> |
| <span class="source-line-no">1028</span><span id="line-1028"> setupMockColumnFamiliesForDataBlockEncoding(table, familyToDataBlockEncoding);</span> |
| <span class="source-line-no">1029</span><span id="line-1029"> TableDescriptor tableDescriptor = table.getDescriptor();</span> |
| <span class="source-line-no">1030</span><span id="line-1030"> conf.set(HFileOutputFormat2.DATABLOCK_ENCODING_FAMILIES_CONF_KEY,</span> |
| <span class="source-line-no">1031</span><span id="line-1031"> HFileOutputFormat2.serializeColumnFamilyAttribute(</span> |
| <span class="source-line-no">1032</span><span id="line-1032"> HFileOutputFormat2.dataBlockEncodingDetails, Arrays.asList(tableDescriptor)));</span> |
| <span class="source-line-no">1033</span><span id="line-1033"></span> |
| <span class="source-line-no">1034</span><span id="line-1034"> // read back family specific data block encoding settings from the</span> |
| <span class="source-line-no">1035</span><span id="line-1035"> // configuration</span> |
| <span class="source-line-no">1036</span><span id="line-1036"> Map<byte[], DataBlockEncoding> retrievedFamilyToDataBlockEncodingMap =</span> |
| <span class="source-line-no">1037</span><span id="line-1037"> HFileOutputFormat2.createFamilyDataBlockEncodingMap(conf);</span> |
| <span class="source-line-no">1038</span><span id="line-1038"></span> |
| <span class="source-line-no">1039</span><span id="line-1039"> // test that we have a value for all column families that matches with the</span> |
| <span class="source-line-no">1040</span><span id="line-1040"> // used mock values</span> |
| <span class="source-line-no">1041</span><span id="line-1041"> for (Entry<String, DataBlockEncoding> entry : familyToDataBlockEncoding.entrySet()) {</span> |
| <span class="source-line-no">1042</span><span id="line-1042"> assertEquals(</span> |
| <span class="source-line-no">1043</span><span id="line-1043"> "DataBlockEncoding configuration incorrect for column family:" + entry.getKey(),</span> |
| <span class="source-line-no">1044</span><span id="line-1044"> entry.getValue(),</span> |
| <span class="source-line-no">1045</span><span id="line-1045"> retrievedFamilyToDataBlockEncodingMap.get(Bytes.toBytes(entry.getKey())));</span> |
| <span class="source-line-no">1046</span><span id="line-1046"> }</span> |
| <span class="source-line-no">1047</span><span id="line-1047"> }</span> |
| <span class="source-line-no">1048</span><span id="line-1048"> }</span> |
| <span class="source-line-no">1049</span><span id="line-1049"></span> |
| <span class="source-line-no">1050</span><span id="line-1050"> private void setupMockColumnFamiliesForDataBlockEncoding(Table table,</span> |
| <span class="source-line-no">1051</span><span id="line-1051"> Map<String, DataBlockEncoding> familyToDataBlockEncoding) throws IOException {</span> |
| <span class="source-line-no">1052</span><span id="line-1052"> TableDescriptorBuilder mockTableDescriptor = TableDescriptorBuilder.newBuilder(TABLE_NAMES[0]);</span> |
| <span class="source-line-no">1053</span><span id="line-1053"> for (Entry<String, DataBlockEncoding> entry : familyToDataBlockEncoding.entrySet()) {</span> |
| <span class="source-line-no">1054</span><span id="line-1054"> ColumnFamilyDescriptor columnFamilyDescriptor =</span> |
| <span class="source-line-no">1055</span><span id="line-1055"> ColumnFamilyDescriptorBuilder.newBuilder(Bytes.toBytes(entry.getKey())).setMaxVersions(1)</span> |
| <span class="source-line-no">1056</span><span id="line-1056"> .setDataBlockEncoding(entry.getValue()).setBlockCacheEnabled(false).setTimeToLive(0)</span> |
| <span class="source-line-no">1057</span><span id="line-1057"> .build();</span> |
| <span class="source-line-no">1058</span><span id="line-1058"> mockTableDescriptor.setColumnFamily(columnFamilyDescriptor);</span> |
| <span class="source-line-no">1059</span><span id="line-1059"> }</span> |
| <span class="source-line-no">1060</span><span id="line-1060"> Mockito.doReturn(mockTableDescriptor).when(table).getDescriptor();</span> |
| <span class="source-line-no">1061</span><span id="line-1061"> }</span> |
| <span class="source-line-no">1062</span><span id="line-1062"></span> |
| <span class="source-line-no">1063</span><span id="line-1063"> /**</span> |
| <span class="source-line-no">1064</span><span id="line-1064"> * @return a map from column family names to compression algorithms for testing column family</span> |
| <span class="source-line-no">1065</span><span id="line-1065"> * compression. Column family names have special characters</span> |
| <span class="source-line-no">1066</span><span id="line-1066"> */</span> |
| <span class="source-line-no">1067</span><span id="line-1067"> private Map<String, DataBlockEncoding> getMockColumnFamiliesForDataBlockEncoding(int numCfs) {</span> |
| <span class="source-line-no">1068</span><span id="line-1068"> Map<String, DataBlockEncoding> familyToDataBlockEncoding = new HashMap<>();</span> |
| <span class="source-line-no">1069</span><span id="line-1069"> // use column family names having special characters</span> |
| <span class="source-line-no">1070</span><span id="line-1070"> if (numCfs-- > 0) {</span> |
| <span class="source-line-no">1071</span><span id="line-1071"> familyToDataBlockEncoding.put("Family1!@#!@#&", DataBlockEncoding.DIFF);</span> |
| <span class="source-line-no">1072</span><span id="line-1072"> }</span> |
| <span class="source-line-no">1073</span><span id="line-1073"> if (numCfs-- > 0) {</span> |
| <span class="source-line-no">1074</span><span id="line-1074"> familyToDataBlockEncoding.put("Family2=asdads&!AASD", DataBlockEncoding.FAST_DIFF);</span> |
| <span class="source-line-no">1075</span><span id="line-1075"> }</span> |
| <span class="source-line-no">1076</span><span id="line-1076"> if (numCfs-- > 0) {</span> |
| <span class="source-line-no">1077</span><span id="line-1077"> familyToDataBlockEncoding.put("Family2=asdads&!AASD", DataBlockEncoding.PREFIX);</span> |
| <span class="source-line-no">1078</span><span id="line-1078"> }</span> |
| <span class="source-line-no">1079</span><span id="line-1079"> if (numCfs-- > 0) {</span> |
| <span class="source-line-no">1080</span><span id="line-1080"> familyToDataBlockEncoding.put("Family3", DataBlockEncoding.NONE);</span> |
| <span class="source-line-no">1081</span><span id="line-1081"> }</span> |
| <span class="source-line-no">1082</span><span id="line-1082"> return familyToDataBlockEncoding;</span> |
| <span class="source-line-no">1083</span><span id="line-1083"> }</span> |
| <span class="source-line-no">1084</span><span id="line-1084"></span> |
| <span class="source-line-no">1085</span><span id="line-1085"> private void setupMockStartKeys(RegionLocator table) throws IOException {</span> |
| <span class="source-line-no">1086</span><span id="line-1086"> byte[][] mockKeys = new byte[][] { HConstants.EMPTY_BYTE_ARRAY, Bytes.toBytes("aaa"),</span> |
| <span class="source-line-no">1087</span><span id="line-1087"> Bytes.toBytes("ggg"), Bytes.toBytes("zzz") };</span> |
| <span class="source-line-no">1088</span><span id="line-1088"> Mockito.doReturn(mockKeys).when(table).getStartKeys();</span> |
| <span class="source-line-no">1089</span><span id="line-1089"> }</span> |
| <span class="source-line-no">1090</span><span id="line-1090"></span> |
| <span class="source-line-no">1091</span><span id="line-1091"> private void setupMockTableName(RegionLocator table) throws IOException {</span> |
| <span class="source-line-no">1092</span><span id="line-1092"> TableName mockTableName = TableName.valueOf("mock_table");</span> |
| <span class="source-line-no">1093</span><span id="line-1093"> Mockito.doReturn(mockTableName).when(table).getName();</span> |
| <span class="source-line-no">1094</span><span id="line-1094"> }</span> |
| <span class="source-line-no">1095</span><span id="line-1095"></span> |
| <span class="source-line-no">1096</span><span id="line-1096"> /**</span> |
| <span class="source-line-no">1097</span><span id="line-1097"> * Test that {@link HFileOutputFormat2} RecordWriter uses compression and bloom filter settings</span> |
| <span class="source-line-no">1098</span><span id="line-1098"> * from the column family descriptor</span> |
| <span class="source-line-no">1099</span><span id="line-1099"> */</span> |
| <span class="source-line-no">1100</span><span id="line-1100"> @Ignore("Goes zombie too frequently; needs work. See HBASE-14563")</span> |
| <span class="source-line-no">1101</span><span id="line-1101"> @Test</span> |
| <span class="source-line-no">1102</span><span id="line-1102"> public void testColumnFamilySettings() throws Exception {</span> |
| <span class="source-line-no">1103</span><span id="line-1103"> Configuration conf = new Configuration(this.util.getConfiguration());</span> |
| <span class="source-line-no">1104</span><span id="line-1104"> RecordWriter<ImmutableBytesWritable, Cell> writer = null;</span> |
| <span class="source-line-no">1105</span><span id="line-1105"> TaskAttemptContext context = null;</span> |
| <span class="source-line-no">1106</span><span id="line-1106"> Path dir = util.getDataTestDir("testColumnFamilySettings");</span> |
| <span class="source-line-no">1107</span><span id="line-1107"></span> |
| <span class="source-line-no">1108</span><span id="line-1108"> // Setup table descriptor</span> |
| <span class="source-line-no">1109</span><span id="line-1109"> Table table = Mockito.mock(Table.class);</span> |
| <span class="source-line-no">1110</span><span id="line-1110"> RegionLocator regionLocator = Mockito.mock(RegionLocator.class);</span> |
| <span class="source-line-no">1111</span><span id="line-1111"> TableDescriptorBuilder tableDescriptorBuilder =</span> |
| <span class="source-line-no">1112</span><span id="line-1112"> TableDescriptorBuilder.newBuilder(TABLE_NAMES[0]);</span> |
| <span class="source-line-no">1113</span><span id="line-1113"></span> |
| <span class="source-line-no">1114</span><span id="line-1114"> Mockito.doReturn(tableDescriptorBuilder.build()).when(table).getDescriptor();</span> |
| <span class="source-line-no">1115</span><span id="line-1115"> for (ColumnFamilyDescriptor hcd : HBaseTestingUtil.generateColumnDescriptors()) {</span> |
| <span class="source-line-no">1116</span><span id="line-1116"> tableDescriptorBuilder.setColumnFamily(hcd);</span> |
| <span class="source-line-no">1117</span><span id="line-1117"> }</span> |
| <span class="source-line-no">1118</span><span id="line-1118"></span> |
| <span class="source-line-no">1119</span><span id="line-1119"> // set up the table to return some mock keys</span> |
| <span class="source-line-no">1120</span><span id="line-1120"> setupMockStartKeys(regionLocator);</span> |
| <span class="source-line-no">1121</span><span id="line-1121"></span> |
| <span class="source-line-no">1122</span><span id="line-1122"> try {</span> |
| <span class="source-line-no">1123</span><span id="line-1123"> // partial map red setup to get an operational writer for testing</span> |
| <span class="source-line-no">1124</span><span id="line-1124"> // We turn off the sequence file compression, because DefaultCodec</span> |
| <span class="source-line-no">1125</span><span id="line-1125"> // pollutes the GZip codec pool with an incompatible compressor.</span> |
| <span class="source-line-no">1126</span><span id="line-1126"> conf.set("io.seqfile.compression.type", "NONE");</span> |
| <span class="source-line-no">1127</span><span id="line-1127"> conf.set("hbase.fs.tmp.dir", dir.toString());</span> |
| <span class="source-line-no">1128</span><span id="line-1128"> // turn locality off to eliminate getRegionLocation fail-and-retry time when writing kvs</span> |
| <span class="source-line-no">1129</span><span id="line-1129"> conf.setBoolean(HFileOutputFormat2.LOCALITY_SENSITIVE_CONF_KEY, false);</span> |
| <span class="source-line-no">1130</span><span id="line-1130"></span> |
| <span class="source-line-no">1131</span><span id="line-1131"> Job job = new Job(conf, "testLocalMRIncrementalLoad");</span> |
| <span class="source-line-no">1132</span><span id="line-1132"> job.setWorkingDirectory(util.getDataTestDirOnTestFS("testColumnFamilySettings"));</span> |
| <span class="source-line-no">1133</span><span id="line-1133"> setupRandomGeneratorMapper(job, false);</span> |
| <span class="source-line-no">1134</span><span id="line-1134"> HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);</span> |
| <span class="source-line-no">1135</span><span id="line-1135"> FileOutputFormat.setOutputPath(job, dir);</span> |
| <span class="source-line-no">1136</span><span id="line-1136"> context = createTestTaskAttemptContext(job);</span> |
| <span class="source-line-no">1137</span><span id="line-1137"> HFileOutputFormat2 hof = new HFileOutputFormat2();</span> |
| <span class="source-line-no">1138</span><span id="line-1138"> writer = hof.getRecordWriter(context);</span> |
| <span class="source-line-no">1139</span><span id="line-1139"></span> |
| <span class="source-line-no">1140</span><span id="line-1140"> // write out random rows</span> |
| <span class="source-line-no">1141</span><span id="line-1141"> writeRandomKeyValues(writer, context, tableDescriptorBuilder.build().getColumnFamilyNames(),</span> |
| <span class="source-line-no">1142</span><span id="line-1142"> ROWSPERSPLIT);</span> |
| <span class="source-line-no">1143</span><span id="line-1143"> writer.close(context);</span> |
| <span class="source-line-no">1144</span><span id="line-1144"></span> |
| <span class="source-line-no">1145</span><span id="line-1145"> // Make sure that a directory was created for every CF</span> |
| <span class="source-line-no">1146</span><span id="line-1146"> FileSystem fs = dir.getFileSystem(conf);</span> |
| <span class="source-line-no">1147</span><span id="line-1147"></span> |
| <span class="source-line-no">1148</span><span id="line-1148"> // commit so that the filesystem has one directory per column family</span> |
| <span class="source-line-no">1149</span><span id="line-1149"> hof.getOutputCommitter(context).commitTask(context);</span> |
| <span class="source-line-no">1150</span><span id="line-1150"> hof.getOutputCommitter(context).commitJob(context);</span> |
| <span class="source-line-no">1151</span><span id="line-1151"> FileStatus[] families = CommonFSUtils.listStatus(fs, dir, new FSUtils.FamilyDirFilter(fs));</span> |
| <span class="source-line-no">1152</span><span id="line-1152"> assertEquals(tableDescriptorBuilder.build().getColumnFamilies().length, families.length);</span> |
| <span class="source-line-no">1153</span><span id="line-1153"> for (FileStatus f : families) {</span> |
| <span class="source-line-no">1154</span><span id="line-1154"> String familyStr = f.getPath().getName();</span> |
| <span class="source-line-no">1155</span><span id="line-1155"> ColumnFamilyDescriptor hcd =</span> |
| <span class="source-line-no">1156</span><span id="line-1156"> tableDescriptorBuilder.build().getColumnFamily(Bytes.toBytes(familyStr));</span> |
| <span class="source-line-no">1157</span><span id="line-1157"> // verify that the compression on this file matches the configured</span> |
| <span class="source-line-no">1158</span><span id="line-1158"> // compression</span> |
| <span class="source-line-no">1159</span><span id="line-1159"> Path dataFilePath = fs.listStatus(f.getPath())[0].getPath();</span> |
| <span class="source-line-no">1160</span><span id="line-1160"> Reader reader = HFile.createReader(fs, dataFilePath, new CacheConfig(conf), true, conf);</span> |
| <span class="source-line-no">1161</span><span id="line-1161"> Map<byte[], byte[]> fileInfo = reader.getHFileInfo();</span> |
| <span class="source-line-no">1162</span><span id="line-1162"></span> |
| <span class="source-line-no">1163</span><span id="line-1163"> byte[] bloomFilter = fileInfo.get(BLOOM_FILTER_TYPE_KEY);</span> |
| <span class="source-line-no">1164</span><span id="line-1164"> if (bloomFilter == null) bloomFilter = Bytes.toBytes("NONE");</span> |
| <span class="source-line-no">1165</span><span id="line-1165"> assertEquals(</span> |
| <span class="source-line-no">1166</span><span id="line-1166"> "Incorrect bloom filter used for column family " + familyStr + "(reader: " + reader + ")",</span> |
| <span class="source-line-no">1167</span><span id="line-1167"> hcd.getBloomFilterType(), BloomType.valueOf(Bytes.toString(bloomFilter)));</span> |
| <span class="source-line-no">1168</span><span id="line-1168"> assertEquals(</span> |
| <span class="source-line-no">1169</span><span id="line-1169"> "Incorrect compression used for column family " + familyStr + "(reader: " + reader + ")",</span> |
| <span class="source-line-no">1170</span><span id="line-1170"> hcd.getCompressionType(), reader.getFileContext().getCompression());</span> |
| <span class="source-line-no">1171</span><span id="line-1171"> }</span> |
| <span class="source-line-no">1172</span><span id="line-1172"> } finally {</span> |
| <span class="source-line-no">1173</span><span id="line-1173"> dir.getFileSystem(conf).delete(dir, true);</span> |
| <span class="source-line-no">1174</span><span id="line-1174"> }</span> |
| <span class="source-line-no">1175</span><span id="line-1175"> }</span> |
| <span class="source-line-no">1176</span><span id="line-1176"></span> |
| <span class="source-line-no">1177</span><span id="line-1177"> /**</span> |
| <span class="source-line-no">1178</span><span id="line-1178"> * Write random values to the writer assuming a table created using {@link #FAMILIES} as column</span> |
| <span class="source-line-no">1179</span><span id="line-1179"> * family descriptors</span> |
| <span class="source-line-no">1180</span><span id="line-1180"> */</span> |
| <span class="source-line-no">1181</span><span id="line-1181"> private void writeRandomKeyValues(RecordWriter<ImmutableBytesWritable, Cell> writer,</span> |
| <span class="source-line-no">1182</span><span id="line-1182"> TaskAttemptContext context, Set<byte[]> families, int numRows)</span> |
| <span class="source-line-no">1183</span><span id="line-1183"> throws IOException, InterruptedException {</span> |
| <span class="source-line-no">1184</span><span id="line-1184"> byte keyBytes[] = new byte[Bytes.SIZEOF_INT];</span> |
| <span class="source-line-no">1185</span><span id="line-1185"> int valLength = 10;</span> |
| <span class="source-line-no">1186</span><span id="line-1186"> byte valBytes[] = new byte[valLength];</span> |
| <span class="source-line-no">1187</span><span id="line-1187"></span> |
| <span class="source-line-no">1188</span><span id="line-1188"> int taskId = context.getTaskAttemptID().getTaskID().getId();</span> |
| <span class="source-line-no">1189</span><span id="line-1189"> assert taskId < Byte.MAX_VALUE : "Unit tests dont support > 127 tasks!";</span> |
| <span class="source-line-no">1190</span><span id="line-1190"> final byte[] qualifier = Bytes.toBytes("data");</span> |
| <span class="source-line-no">1191</span><span id="line-1191"> for (int i = 0; i < numRows; i++) {</span> |
| <span class="source-line-no">1192</span><span id="line-1192"> Bytes.putInt(keyBytes, 0, i);</span> |
| <span class="source-line-no">1193</span><span id="line-1193"> Bytes.random(valBytes);</span> |
| <span class="source-line-no">1194</span><span id="line-1194"> ImmutableBytesWritable key = new ImmutableBytesWritable(keyBytes);</span> |
| <span class="source-line-no">1195</span><span id="line-1195"> for (byte[] family : families) {</span> |
| <span class="source-line-no">1196</span><span id="line-1196"> Cell kv = new KeyValue(keyBytes, family, qualifier, valBytes);</span> |
| <span class="source-line-no">1197</span><span id="line-1197"> writer.write(key, kv);</span> |
| <span class="source-line-no">1198</span><span id="line-1198"> }</span> |
| <span class="source-line-no">1199</span><span id="line-1199"> }</span> |
| <span class="source-line-no">1200</span><span id="line-1200"> }</span> |
| <span class="source-line-no">1201</span><span id="line-1201"></span> |
| <span class="source-line-no">1202</span><span id="line-1202"> /**</span> |
| <span class="source-line-no">1203</span><span id="line-1203"> * This test is to test the scenario happened in HBASE-6901. All files are bulk loaded and</span> |
| <span class="source-line-no">1204</span><span id="line-1204"> * excluded from minor compaction. Without the fix of HBASE-6901, an</span> |
| <span class="source-line-no">1205</span><span id="line-1205"> * ArrayIndexOutOfBoundsException will be thrown.</span> |
| <span class="source-line-no">1206</span><span id="line-1206"> */</span> |
| <span class="source-line-no">1207</span><span id="line-1207"> @Ignore("Flakey: See HBASE-9051")</span> |
| <span class="source-line-no">1208</span><span id="line-1208"> @Test</span> |
| <span class="source-line-no">1209</span><span id="line-1209"> public void testExcludeAllFromMinorCompaction() throws Exception {</span> |
| <span class="source-line-no">1210</span><span id="line-1210"> Configuration conf = util.getConfiguration();</span> |
| <span class="source-line-no">1211</span><span id="line-1211"> conf.setInt("hbase.hstore.compaction.min", 2);</span> |
| <span class="source-line-no">1212</span><span id="line-1212"> generateRandomStartKeys(5);</span> |
| <span class="source-line-no">1213</span><span id="line-1213"></span> |
| <span class="source-line-no">1214</span><span id="line-1214"> util.startMiniCluster();</span> |
| <span class="source-line-no">1215</span><span id="line-1215"> try (Connection conn = ConnectionFactory.createConnection(); Admin admin = conn.getAdmin();</span> |
| <span class="source-line-no">1216</span><span id="line-1216"> Table table = util.createTable(TABLE_NAMES[0], FAMILIES);</span> |
| <span class="source-line-no">1217</span><span id="line-1217"> RegionLocator locator = conn.getRegionLocator(TABLE_NAMES[0])) {</span> |
| <span class="source-line-no">1218</span><span id="line-1218"> final FileSystem fs = util.getDFSCluster().getFileSystem();</span> |
| <span class="source-line-no">1219</span><span id="line-1219"> assertEquals("Should start with empty table", 0, util.countRows(table));</span> |
| <span class="source-line-no">1220</span><span id="line-1220"></span> |
| <span class="source-line-no">1221</span><span id="line-1221"> // deep inspection: get the StoreFile dir</span> |
| <span class="source-line-no">1222</span><span id="line-1222"> final Path storePath =</span> |
| <span class="source-line-no">1223</span><span id="line-1223"> new Path(CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), TABLE_NAMES[0]),</span> |
| <span class="source-line-no">1224</span><span id="line-1224"> new Path(admin.getRegions(TABLE_NAMES[0]).get(0).getEncodedName(),</span> |
| <span class="source-line-no">1225</span><span id="line-1225"> Bytes.toString(FAMILIES[0])));</span> |
| <span class="source-line-no">1226</span><span id="line-1226"> assertEquals(0, fs.listStatus(storePath).length);</span> |
| <span class="source-line-no">1227</span><span id="line-1227"></span> |
| <span class="source-line-no">1228</span><span id="line-1228"> // Generate two bulk load files</span> |
| <span class="source-line-no">1229</span><span id="line-1229"> conf.setBoolean("hbase.mapreduce.hfileoutputformat.compaction.exclude", true);</span> |
| <span class="source-line-no">1230</span><span id="line-1230"></span> |
| <span class="source-line-no">1231</span><span id="line-1231"> for (int i = 0; i < 2; i++) {</span> |
| <span class="source-line-no">1232</span><span id="line-1232"> Path testDir = util.getDataTestDirOnTestFS("testExcludeAllFromMinorCompaction_" + i);</span> |
| <span class="source-line-no">1233</span><span id="line-1233"> runIncrementalPELoad(conf,</span> |
| <span class="source-line-no">1234</span><span id="line-1234"> Arrays.asList(new HFileOutputFormat2.TableInfo(table.getDescriptor(),</span> |
| <span class="source-line-no">1235</span><span id="line-1235"> conn.getRegionLocator(TABLE_NAMES[0]))),</span> |
| <span class="source-line-no">1236</span><span id="line-1236"> testDir, false);</span> |
| <span class="source-line-no">1237</span><span id="line-1237"> // Perform the actual load</span> |
| <span class="source-line-no">1238</span><span id="line-1238"> BulkLoadHFiles.create(conf).bulkLoad(table.getName(), testDir);</span> |
| <span class="source-line-no">1239</span><span id="line-1239"> }</span> |
| <span class="source-line-no">1240</span><span id="line-1240"></span> |
| <span class="source-line-no">1241</span><span id="line-1241"> // Ensure data shows up</span> |
| <span class="source-line-no">1242</span><span id="line-1242"> int expectedRows = 2 * NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;</span> |
| <span class="source-line-no">1243</span><span id="line-1243"> assertEquals("BulkLoadHFiles should put expected data in table", expectedRows,</span> |
| <span class="source-line-no">1244</span><span id="line-1244"> util.countRows(table));</span> |
| <span class="source-line-no">1245</span><span id="line-1245"></span> |
| <span class="source-line-no">1246</span><span id="line-1246"> // should have a second StoreFile now</span> |
| <span class="source-line-no">1247</span><span id="line-1247"> assertEquals(2, fs.listStatus(storePath).length);</span> |
| <span class="source-line-no">1248</span><span id="line-1248"></span> |
| <span class="source-line-no">1249</span><span id="line-1249"> // minor compactions shouldn't get rid of the file</span> |
| <span class="source-line-no">1250</span><span id="line-1250"> admin.compact(TABLE_NAMES[0]);</span> |
| <span class="source-line-no">1251</span><span id="line-1251"> try {</span> |
| <span class="source-line-no">1252</span><span id="line-1252"> quickPoll(new Callable<Boolean>() {</span> |
| <span class="source-line-no">1253</span><span id="line-1253"> @Override</span> |
| <span class="source-line-no">1254</span><span id="line-1254"> public Boolean call() throws Exception {</span> |
| <span class="source-line-no">1255</span><span id="line-1255"> List<HRegion> regions = util.getMiniHBaseCluster().getRegions(TABLE_NAMES[0]);</span> |
| <span class="source-line-no">1256</span><span id="line-1256"> for (HRegion region : regions) {</span> |
| <span class="source-line-no">1257</span><span id="line-1257"> for (HStore store : region.getStores()) {</span> |
| <span class="source-line-no">1258</span><span id="line-1258"> store.closeAndArchiveCompactedFiles();</span> |
| <span class="source-line-no">1259</span><span id="line-1259"> }</span> |
| <span class="source-line-no">1260</span><span id="line-1260"> }</span> |
| <span class="source-line-no">1261</span><span id="line-1261"> return fs.listStatus(storePath).length == 1;</span> |
| <span class="source-line-no">1262</span><span id="line-1262"> }</span> |
| <span class="source-line-no">1263</span><span id="line-1263"> }, 5000);</span> |
| <span class="source-line-no">1264</span><span id="line-1264"> throw new IOException("SF# = " + fs.listStatus(storePath).length);</span> |
| <span class="source-line-no">1265</span><span id="line-1265"> } catch (AssertionError ae) {</span> |
| <span class="source-line-no">1266</span><span id="line-1266"> // this is expected behavior</span> |
| <span class="source-line-no">1267</span><span id="line-1267"> }</span> |
| <span class="source-line-no">1268</span><span id="line-1268"></span> |
| <span class="source-line-no">1269</span><span id="line-1269"> // a major compaction should work though</span> |
| <span class="source-line-no">1270</span><span id="line-1270"> admin.majorCompact(TABLE_NAMES[0]);</span> |
| <span class="source-line-no">1271</span><span id="line-1271"> quickPoll(new Callable<Boolean>() {</span> |
| <span class="source-line-no">1272</span><span id="line-1272"> @Override</span> |
| <span class="source-line-no">1273</span><span id="line-1273"> public Boolean call() throws Exception {</span> |
| <span class="source-line-no">1274</span><span id="line-1274"> List<HRegion> regions = util.getMiniHBaseCluster().getRegions(TABLE_NAMES[0]);</span> |
| <span class="source-line-no">1275</span><span id="line-1275"> for (HRegion region : regions) {</span> |
| <span class="source-line-no">1276</span><span id="line-1276"> for (HStore store : region.getStores()) {</span> |
| <span class="source-line-no">1277</span><span id="line-1277"> store.closeAndArchiveCompactedFiles();</span> |
| <span class="source-line-no">1278</span><span id="line-1278"> }</span> |
| <span class="source-line-no">1279</span><span id="line-1279"> }</span> |
| <span class="source-line-no">1280</span><span id="line-1280"> return fs.listStatus(storePath).length == 1;</span> |
| <span class="source-line-no">1281</span><span id="line-1281"> }</span> |
| <span class="source-line-no">1282</span><span id="line-1282"> }, 5000);</span> |
| <span class="source-line-no">1283</span><span id="line-1283"></span> |
| <span class="source-line-no">1284</span><span id="line-1284"> } finally {</span> |
| <span class="source-line-no">1285</span><span id="line-1285"> util.shutdownMiniCluster();</span> |
| <span class="source-line-no">1286</span><span id="line-1286"> }</span> |
| <span class="source-line-no">1287</span><span id="line-1287"> }</span> |
| <span class="source-line-no">1288</span><span id="line-1288"></span> |
| <span class="source-line-no">1289</span><span id="line-1289"> @Ignore("Goes zombie too frequently; needs work. See HBASE-14563")</span> |
| <span class="source-line-no">1290</span><span id="line-1290"> @Test</span> |
| <span class="source-line-no">1291</span><span id="line-1291"> public void testExcludeMinorCompaction() throws Exception {</span> |
| <span class="source-line-no">1292</span><span id="line-1292"> Configuration conf = util.getConfiguration();</span> |
| <span class="source-line-no">1293</span><span id="line-1293"> conf.setInt("hbase.hstore.compaction.min", 2);</span> |
| <span class="source-line-no">1294</span><span id="line-1294"> generateRandomStartKeys(5);</span> |
| <span class="source-line-no">1295</span><span id="line-1295"></span> |
| <span class="source-line-no">1296</span><span id="line-1296"> util.startMiniCluster();</span> |
| <span class="source-line-no">1297</span><span id="line-1297"> try (Connection conn = ConnectionFactory.createConnection(conf);</span> |
| <span class="source-line-no">1298</span><span id="line-1298"> Admin admin = conn.getAdmin()) {</span> |
| <span class="source-line-no">1299</span><span id="line-1299"> Path testDir = util.getDataTestDirOnTestFS("testExcludeMinorCompaction");</span> |
| <span class="source-line-no">1300</span><span id="line-1300"> final FileSystem fs = util.getDFSCluster().getFileSystem();</span> |
| <span class="source-line-no">1301</span><span id="line-1301"> Table table = util.createTable(TABLE_NAMES[0], FAMILIES);</span> |
| <span class="source-line-no">1302</span><span id="line-1302"> assertEquals("Should start with empty table", 0, util.countRows(table));</span> |
| <span class="source-line-no">1303</span><span id="line-1303"></span> |
| <span class="source-line-no">1304</span><span id="line-1304"> // deep inspection: get the StoreFile dir</span> |
| <span class="source-line-no">1305</span><span id="line-1305"> final Path storePath =</span> |
| <span class="source-line-no">1306</span><span id="line-1306"> new Path(CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), TABLE_NAMES[0]),</span> |
| <span class="source-line-no">1307</span><span id="line-1307"> new Path(admin.getRegions(TABLE_NAMES[0]).get(0).getEncodedName(),</span> |
| <span class="source-line-no">1308</span><span id="line-1308"> Bytes.toString(FAMILIES[0])));</span> |
| <span class="source-line-no">1309</span><span id="line-1309"> assertEquals(0, fs.listStatus(storePath).length);</span> |
| <span class="source-line-no">1310</span><span id="line-1310"></span> |
| <span class="source-line-no">1311</span><span id="line-1311"> // put some data in it and flush to create a storefile</span> |
| <span class="source-line-no">1312</span><span id="line-1312"> Put p = new Put(Bytes.toBytes("test"));</span> |
| <span class="source-line-no">1313</span><span id="line-1313"> p.addColumn(FAMILIES[0], Bytes.toBytes("1"), Bytes.toBytes("1"));</span> |
| <span class="source-line-no">1314</span><span id="line-1314"> table.put(p);</span> |
| <span class="source-line-no">1315</span><span id="line-1315"> admin.flush(TABLE_NAMES[0]);</span> |
| <span class="source-line-no">1316</span><span id="line-1316"> assertEquals(1, util.countRows(table));</span> |
| <span class="source-line-no">1317</span><span id="line-1317"> quickPoll(new Callable<Boolean>() {</span> |
| <span class="source-line-no">1318</span><span id="line-1318"> @Override</span> |
| <span class="source-line-no">1319</span><span id="line-1319"> public Boolean call() throws Exception {</span> |
| <span class="source-line-no">1320</span><span id="line-1320"> return fs.listStatus(storePath).length == 1;</span> |
| <span class="source-line-no">1321</span><span id="line-1321"> }</span> |
| <span class="source-line-no">1322</span><span id="line-1322"> }, 5000);</span> |
| <span class="source-line-no">1323</span><span id="line-1323"></span> |
| <span class="source-line-no">1324</span><span id="line-1324"> // Generate a bulk load file with more rows</span> |
| <span class="source-line-no">1325</span><span id="line-1325"> conf.setBoolean("hbase.mapreduce.hfileoutputformat.compaction.exclude", true);</span> |
| <span class="source-line-no">1326</span><span id="line-1326"></span> |
| <span class="source-line-no">1327</span><span id="line-1327"> RegionLocator regionLocator = conn.getRegionLocator(TABLE_NAMES[0]);</span> |
| <span class="source-line-no">1328</span><span id="line-1328"> runIncrementalPELoad(conf,</span> |
| <span class="source-line-no">1329</span><span id="line-1329"> Arrays.asList(new HFileOutputFormat2.TableInfo(table.getDescriptor(), regionLocator)),</span> |
| <span class="source-line-no">1330</span><span id="line-1330"> testDir, false);</span> |
| <span class="source-line-no">1331</span><span id="line-1331"></span> |
| <span class="source-line-no">1332</span><span id="line-1332"> // Perform the actual load</span> |
| <span class="source-line-no">1333</span><span id="line-1333"> BulkLoadHFiles.create(conf).bulkLoad(table.getName(), testDir);</span> |
| <span class="source-line-no">1334</span><span id="line-1334"></span> |
| <span class="source-line-no">1335</span><span id="line-1335"> // Ensure data shows up</span> |
| <span class="source-line-no">1336</span><span id="line-1336"> int expectedRows = NMapInputFormat.getNumMapTasks(conf) * ROWSPERSPLIT;</span> |
| <span class="source-line-no">1337</span><span id="line-1337"> assertEquals("BulkLoadHFiles should put expected data in table", expectedRows + 1,</span> |
| <span class="source-line-no">1338</span><span id="line-1338"> util.countRows(table));</span> |
| <span class="source-line-no">1339</span><span id="line-1339"></span> |
| <span class="source-line-no">1340</span><span id="line-1340"> // should have a second StoreFile now</span> |
| <span class="source-line-no">1341</span><span id="line-1341"> assertEquals(2, fs.listStatus(storePath).length);</span> |
| <span class="source-line-no">1342</span><span id="line-1342"></span> |
| <span class="source-line-no">1343</span><span id="line-1343"> // minor compactions shouldn't get rid of the file</span> |
| <span class="source-line-no">1344</span><span id="line-1344"> admin.compact(TABLE_NAMES[0]);</span> |
| <span class="source-line-no">1345</span><span id="line-1345"> try {</span> |
| <span class="source-line-no">1346</span><span id="line-1346"> quickPoll(new Callable<Boolean>() {</span> |
| <span class="source-line-no">1347</span><span id="line-1347"> @Override</span> |
| <span class="source-line-no">1348</span><span id="line-1348"> public Boolean call() throws Exception {</span> |
| <span class="source-line-no">1349</span><span id="line-1349"> return fs.listStatus(storePath).length == 1;</span> |
| <span class="source-line-no">1350</span><span id="line-1350"> }</span> |
| <span class="source-line-no">1351</span><span id="line-1351"> }, 5000);</span> |
| <span class="source-line-no">1352</span><span id="line-1352"> throw new IOException("SF# = " + fs.listStatus(storePath).length);</span> |
| <span class="source-line-no">1353</span><span id="line-1353"> } catch (AssertionError ae) {</span> |
| <span class="source-line-no">1354</span><span id="line-1354"> // this is expected behavior</span> |
| <span class="source-line-no">1355</span><span id="line-1355"> }</span> |
| <span class="source-line-no">1356</span><span id="line-1356"></span> |
| <span class="source-line-no">1357</span><span id="line-1357"> // a major compaction should work though</span> |
| <span class="source-line-no">1358</span><span id="line-1358"> admin.majorCompact(TABLE_NAMES[0]);</span> |
| <span class="source-line-no">1359</span><span id="line-1359"> quickPoll(new Callable<Boolean>() {</span> |
| <span class="source-line-no">1360</span><span id="line-1360"> @Override</span> |
| <span class="source-line-no">1361</span><span id="line-1361"> public Boolean call() throws Exception {</span> |
| <span class="source-line-no">1362</span><span id="line-1362"> return fs.listStatus(storePath).length == 1;</span> |
| <span class="source-line-no">1363</span><span id="line-1363"> }</span> |
| <span class="source-line-no">1364</span><span id="line-1364"> }, 5000);</span> |
| <span class="source-line-no">1365</span><span id="line-1365"></span> |
| <span class="source-line-no">1366</span><span id="line-1366"> } finally {</span> |
| <span class="source-line-no">1367</span><span id="line-1367"> util.shutdownMiniCluster();</span> |
| <span class="source-line-no">1368</span><span id="line-1368"> }</span> |
| <span class="source-line-no">1369</span><span id="line-1369"> }</span> |
| <span class="source-line-no">1370</span><span id="line-1370"></span> |
| <span class="source-line-no">1371</span><span id="line-1371"> private void quickPoll(Callable<Boolean> c, int waitMs) throws Exception {</span> |
| <span class="source-line-no">1372</span><span id="line-1372"> int sleepMs = 10;</span> |
| <span class="source-line-no">1373</span><span id="line-1373"> int retries = (int) Math.ceil(((double) waitMs) / sleepMs);</span> |
| <span class="source-line-no">1374</span><span id="line-1374"> while (retries-- > 0) {</span> |
| <span class="source-line-no">1375</span><span id="line-1375"> if (c.call().booleanValue()) {</span> |
| <span class="source-line-no">1376</span><span id="line-1376"> return;</span> |
| <span class="source-line-no">1377</span><span id="line-1377"> }</span> |
| <span class="source-line-no">1378</span><span id="line-1378"> Thread.sleep(sleepMs);</span> |
| <span class="source-line-no">1379</span><span id="line-1379"> }</span> |
| <span class="source-line-no">1380</span><span id="line-1380"> fail();</span> |
| <span class="source-line-no">1381</span><span id="line-1381"> }</span> |
| <span class="source-line-no">1382</span><span id="line-1382"></span> |
| <span class="source-line-no">1383</span><span id="line-1383"> public static void main(String args[]) throws Exception {</span> |
| <span class="source-line-no">1384</span><span id="line-1384"> new TestHFileOutputFormat2().manualTest(args);</span> |
| <span class="source-line-no">1385</span><span id="line-1385"> }</span> |
| <span class="source-line-no">1386</span><span id="line-1386"></span> |
| <span class="source-line-no">1387</span><span id="line-1387"> public void manualTest(String args[]) throws Exception {</span> |
| <span class="source-line-no">1388</span><span id="line-1388"> Configuration conf = HBaseConfiguration.create();</span> |
| <span class="source-line-no">1389</span><span id="line-1389"> util = new HBaseTestingUtil(conf);</span> |
| <span class="source-line-no">1390</span><span id="line-1390"> if ("newtable".equals(args[0])) {</span> |
| <span class="source-line-no">1391</span><span id="line-1391"> TableName tname = TableName.valueOf(args[1]);</span> |
| <span class="source-line-no">1392</span><span id="line-1392"> byte[][] splitKeys = generateRandomSplitKeys(4);</span> |
| <span class="source-line-no">1393</span><span id="line-1393"> Table table = util.createTable(tname, FAMILIES, splitKeys);</span> |
| <span class="source-line-no">1394</span><span id="line-1394"> } else if ("incremental".equals(args[0])) {</span> |
| <span class="source-line-no">1395</span><span id="line-1395"> TableName tname = TableName.valueOf(args[1]);</span> |
| <span class="source-line-no">1396</span><span id="line-1396"> try (Connection c = ConnectionFactory.createConnection(conf); Admin admin = c.getAdmin();</span> |
| <span class="source-line-no">1397</span><span id="line-1397"> RegionLocator regionLocator = c.getRegionLocator(tname)) {</span> |
| <span class="source-line-no">1398</span><span id="line-1398"> Path outDir = new Path("incremental-out");</span> |
| <span class="source-line-no">1399</span><span id="line-1399"> runIncrementalPELoad(conf,</span> |
| <span class="source-line-no">1400</span><span id="line-1400"> Arrays</span> |
| <span class="source-line-no">1401</span><span id="line-1401"> .asList(new HFileOutputFormat2.TableInfo(admin.getDescriptor(tname), regionLocator)),</span> |
| <span class="source-line-no">1402</span><span id="line-1402"> outDir, false);</span> |
| <span class="source-line-no">1403</span><span id="line-1403"> }</span> |
| <span class="source-line-no">1404</span><span id="line-1404"> } else {</span> |
| <span class="source-line-no">1405</span><span id="line-1405"> throw new RuntimeException("usage: TestHFileOutputFormat2 newtable | incremental");</span> |
| <span class="source-line-no">1406</span><span id="line-1406"> }</span> |
| <span class="source-line-no">1407</span><span id="line-1407"> }</span> |
| <span class="source-line-no">1408</span><span id="line-1408"></span> |
| <span class="source-line-no">1409</span><span id="line-1409"> @Test</span> |
| <span class="source-line-no">1410</span><span id="line-1410"> public void testBlockStoragePolicy() throws Exception {</span> |
| <span class="source-line-no">1411</span><span id="line-1411"> util = new HBaseTestingUtil();</span> |
| <span class="source-line-no">1412</span><span id="line-1412"> Configuration conf = util.getConfiguration();</span> |
| <span class="source-line-no">1413</span><span id="line-1413"> conf.set(HFileOutputFormat2.STORAGE_POLICY_PROPERTY, "ALL_SSD");</span> |
| <span class="source-line-no">1414</span><span id="line-1414"></span> |
| <span class="source-line-no">1415</span><span id="line-1415"> conf.set(</span> |
| <span class="source-line-no">1416</span><span id="line-1416"> HFileOutputFormat2.STORAGE_POLICY_PROPERTY_CF_PREFIX + Bytes</span> |
| <span class="source-line-no">1417</span><span id="line-1417"> .toString(HFileOutputFormat2.combineTableNameSuffix(TABLE_NAMES[0].getName(), FAMILIES[0])),</span> |
| <span class="source-line-no">1418</span><span id="line-1418"> "ONE_SSD");</span> |
| <span class="source-line-no">1419</span><span id="line-1419"> Path cf1Dir = new Path(util.getDataTestDir(), Bytes.toString(FAMILIES[0]));</span> |
| <span class="source-line-no">1420</span><span id="line-1420"> Path cf2Dir = new Path(util.getDataTestDir(), Bytes.toString(FAMILIES[1]));</span> |
| <span class="source-line-no">1421</span><span id="line-1421"> util.startMiniDFSCluster(3);</span> |
| <span class="source-line-no">1422</span><span id="line-1422"> FileSystem fs = util.getDFSCluster().getFileSystem();</span> |
| <span class="source-line-no">1423</span><span id="line-1423"> try {</span> |
| <span class="source-line-no">1424</span><span id="line-1424"> fs.mkdirs(cf1Dir);</span> |
| <span class="source-line-no">1425</span><span id="line-1425"> fs.mkdirs(cf2Dir);</span> |
| <span class="source-line-no">1426</span><span id="line-1426"></span> |
| <span class="source-line-no">1427</span><span id="line-1427"> // the original block storage policy would be HOT</span> |
| <span class="source-line-no">1428</span><span id="line-1428"> String spA = getStoragePolicyName(fs, cf1Dir);</span> |
| <span class="source-line-no">1429</span><span id="line-1429"> String spB = getStoragePolicyName(fs, cf2Dir);</span> |
| <span class="source-line-no">1430</span><span id="line-1430"> LOG.debug("Storage policy of cf 0: [" + spA + "].");</span> |
| <span class="source-line-no">1431</span><span id="line-1431"> LOG.debug("Storage policy of cf 1: [" + spB + "].");</span> |
| <span class="source-line-no">1432</span><span id="line-1432"> assertEquals("HOT", spA);</span> |
| <span class="source-line-no">1433</span><span id="line-1433"> assertEquals("HOT", spB);</span> |
| <span class="source-line-no">1434</span><span id="line-1434"></span> |
| <span class="source-line-no">1435</span><span id="line-1435"> // alter table cf schema to change storage policies</span> |
| <span class="source-line-no">1436</span><span id="line-1436"> HFileOutputFormat2.configureStoragePolicy(conf, fs,</span> |
| <span class="source-line-no">1437</span><span id="line-1437"> HFileOutputFormat2.combineTableNameSuffix(TABLE_NAMES[0].getName(), FAMILIES[0]), cf1Dir);</span> |
| <span class="source-line-no">1438</span><span id="line-1438"> HFileOutputFormat2.configureStoragePolicy(conf, fs,</span> |
| <span class="source-line-no">1439</span><span id="line-1439"> HFileOutputFormat2.combineTableNameSuffix(TABLE_NAMES[0].getName(), FAMILIES[1]), cf2Dir);</span> |
| <span class="source-line-no">1440</span><span id="line-1440"> spA = getStoragePolicyName(fs, cf1Dir);</span> |
| <span class="source-line-no">1441</span><span id="line-1441"> spB = getStoragePolicyName(fs, cf2Dir);</span> |
| <span class="source-line-no">1442</span><span id="line-1442"> LOG.debug("Storage policy of cf 0: [" + spA + "].");</span> |
| <span class="source-line-no">1443</span><span id="line-1443"> LOG.debug("Storage policy of cf 1: [" + spB + "].");</span> |
| <span class="source-line-no">1444</span><span id="line-1444"> assertNotNull(spA);</span> |
| <span class="source-line-no">1445</span><span id="line-1445"> assertEquals("ONE_SSD", spA);</span> |
| <span class="source-line-no">1446</span><span id="line-1446"> assertNotNull(spB);</span> |
| <span class="source-line-no">1447</span><span id="line-1447"> assertEquals("ALL_SSD", spB);</span> |
| <span class="source-line-no">1448</span><span id="line-1448"> } finally {</span> |
| <span class="source-line-no">1449</span><span id="line-1449"> fs.delete(cf1Dir, true);</span> |
| <span class="source-line-no">1450</span><span id="line-1450"> fs.delete(cf2Dir, true);</span> |
| <span class="source-line-no">1451</span><span id="line-1451"> util.shutdownMiniDFSCluster();</span> |
| <span class="source-line-no">1452</span><span id="line-1452"> }</span> |
| <span class="source-line-no">1453</span><span id="line-1453"> }</span> |
| <span class="source-line-no">1454</span><span id="line-1454"></span> |
| <span class="source-line-no">1455</span><span id="line-1455"> private String getStoragePolicyName(FileSystem fs, Path path) {</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"> Object blockStoragePolicySpi = ReflectionUtils.invokeMethod(fs, "getStoragePolicy", path);</span> |
| <span class="source-line-no">1458</span><span id="line-1458"> return (String) ReflectionUtils.invokeMethod(blockStoragePolicySpi, "getName");</span> |
| <span class="source-line-no">1459</span><span id="line-1459"> } catch (Exception e) {</span> |
| <span class="source-line-no">1460</span><span id="line-1460"> // Maybe fail because of using old HDFS version, try the old way</span> |
| <span class="source-line-no">1461</span><span id="line-1461"> if (LOG.isTraceEnabled()) {</span> |
| <span class="source-line-no">1462</span><span id="line-1462"> LOG.trace("Failed to get policy directly", e);</span> |
| <span class="source-line-no">1463</span><span id="line-1463"> }</span> |
| <span class="source-line-no">1464</span><span id="line-1464"> String policy = getStoragePolicyNameForOldHDFSVersion(fs, path);</span> |
| <span class="source-line-no">1465</span><span id="line-1465"> return policy == null ? "HOT" : policy;// HOT by default</span> |
| <span class="source-line-no">1466</span><span id="line-1466"> }</span> |
| <span class="source-line-no">1467</span><span id="line-1467"> }</span> |
| <span class="source-line-no">1468</span><span id="line-1468"></span> |
| <span class="source-line-no">1469</span><span id="line-1469"> private String getStoragePolicyNameForOldHDFSVersion(FileSystem fs, Path path) {</span> |
| <span class="source-line-no">1470</span><span id="line-1470"> try {</span> |
| <span class="source-line-no">1471</span><span id="line-1471"> if (fs instanceof DistributedFileSystem) {</span> |
| <span class="source-line-no">1472</span><span id="line-1472"> DistributedFileSystem dfs = (DistributedFileSystem) fs;</span> |
| <span class="source-line-no">1473</span><span id="line-1473"> HdfsFileStatus status = dfs.getClient().getFileInfo(path.toUri().getPath());</span> |
| <span class="source-line-no">1474</span><span id="line-1474"> if (null != status) {</span> |
| <span class="source-line-no">1475</span><span id="line-1475"> byte storagePolicyId = status.getStoragePolicy();</span> |
| <span class="source-line-no">1476</span><span id="line-1476"> Field idUnspecified = BlockStoragePolicySuite.class.getField("ID_UNSPECIFIED");</span> |
| <span class="source-line-no">1477</span><span id="line-1477"> if (storagePolicyId != idUnspecified.getByte(BlockStoragePolicySuite.class)) {</span> |
| <span class="source-line-no">1478</span><span id="line-1478"> BlockStoragePolicy[] policies = dfs.getStoragePolicies();</span> |
| <span class="source-line-no">1479</span><span id="line-1479"> for (BlockStoragePolicy policy : policies) {</span> |
| <span class="source-line-no">1480</span><span id="line-1480"> if (policy.getId() == storagePolicyId) {</span> |
| <span class="source-line-no">1481</span><span id="line-1481"> return policy.getName();</span> |
| <span class="source-line-no">1482</span><span id="line-1482"> }</span> |
| <span class="source-line-no">1483</span><span id="line-1483"> }</span> |
| <span class="source-line-no">1484</span><span id="line-1484"> }</span> |
| <span class="source-line-no">1485</span><span id="line-1485"> }</span> |
| <span class="source-line-no">1486</span><span id="line-1486"> }</span> |
| <span class="source-line-no">1487</span><span id="line-1487"> } catch (Throwable e) {</span> |
| <span class="source-line-no">1488</span><span id="line-1488"> LOG.warn("failed to get block storage policy of [" + path + "]", e);</span> |
| <span class="source-line-no">1489</span><span id="line-1489"> }</span> |
| <span class="source-line-no">1490</span><span id="line-1490"></span> |
| <span class="source-line-no">1491</span><span id="line-1491"> return null;</span> |
| <span class="source-line-no">1492</span><span id="line-1492"> }</span> |
| <span class="source-line-no">1493</span><span id="line-1493"></span> |
| <span class="source-line-no">1494</span><span id="line-1494"> @Test</span> |
| <span class="source-line-no">1495</span><span id="line-1495"> public void TestConfigurePartitioner() throws IOException {</span> |
| <span class="source-line-no">1496</span><span id="line-1496"> Configuration conf = util.getConfiguration();</span> |
| <span class="source-line-no">1497</span><span id="line-1497"> // Create a user who is not the current user</span> |
| <span class="source-line-no">1498</span><span id="line-1498"> String fooUserName = "foo1234";</span> |
| <span class="source-line-no">1499</span><span id="line-1499"> String fooGroupName = "group1";</span> |
| <span class="source-line-no">1500</span><span id="line-1500"> UserGroupInformation ugi =</span> |
| <span class="source-line-no">1501</span><span id="line-1501"> UserGroupInformation.createUserForTesting(fooUserName, new String[] { fooGroupName });</span> |
| <span class="source-line-no">1502</span><span id="line-1502"> // Get user's home directory</span> |
| <span class="source-line-no">1503</span><span id="line-1503"> Path fooHomeDirectory = ugi.doAs(new PrivilegedAction<Path>() {</span> |
| <span class="source-line-no">1504</span><span id="line-1504"> @Override</span> |
| <span class="source-line-no">1505</span><span id="line-1505"> public Path run() {</span> |
| <span class="source-line-no">1506</span><span id="line-1506"> try (FileSystem fs = FileSystem.get(conf)) {</span> |
| <span class="source-line-no">1507</span><span id="line-1507"> return fs.makeQualified(fs.getHomeDirectory());</span> |
| <span class="source-line-no">1508</span><span id="line-1508"> } catch (IOException ioe) {</span> |
| <span class="source-line-no">1509</span><span id="line-1509"> LOG.error("Failed to get foo's home directory", ioe);</span> |
| <span class="source-line-no">1510</span><span id="line-1510"> }</span> |
| <span class="source-line-no">1511</span><span id="line-1511"> return null;</span> |
| <span class="source-line-no">1512</span><span id="line-1512"> }</span> |
| <span class="source-line-no">1513</span><span id="line-1513"> });</span> |
| <span class="source-line-no">1514</span><span id="line-1514"></span> |
| <span class="source-line-no">1515</span><span id="line-1515"> Job job = Mockito.mock(Job.class);</span> |
| <span class="source-line-no">1516</span><span id="line-1516"> Mockito.doReturn(conf).when(job).getConfiguration();</span> |
| <span class="source-line-no">1517</span><span id="line-1517"> ImmutableBytesWritable writable = new ImmutableBytesWritable();</span> |
| <span class="source-line-no">1518</span><span id="line-1518"> List<ImmutableBytesWritable> splitPoints = new LinkedList<ImmutableBytesWritable>();</span> |
| <span class="source-line-no">1519</span><span id="line-1519"> splitPoints.add(writable);</span> |
| <span class="source-line-no">1520</span><span id="line-1520"></span> |
| <span class="source-line-no">1521</span><span id="line-1521"> ugi.doAs(new PrivilegedAction<Void>() {</span> |
| <span class="source-line-no">1522</span><span id="line-1522"> @Override</span> |
| <span class="source-line-no">1523</span><span id="line-1523"> public Void run() {</span> |
| <span class="source-line-no">1524</span><span id="line-1524"> try {</span> |
| <span class="source-line-no">1525</span><span id="line-1525"> HFileOutputFormat2.configurePartitioner(job, splitPoints, false);</span> |
| <span class="source-line-no">1526</span><span id="line-1526"> } catch (IOException ioe) {</span> |
| <span class="source-line-no">1527</span><span id="line-1527"> LOG.error("Failed to configure partitioner", ioe);</span> |
| <span class="source-line-no">1528</span><span id="line-1528"> }</span> |
| <span class="source-line-no">1529</span><span id="line-1529"> return null;</span> |
| <span class="source-line-no">1530</span><span id="line-1530"> }</span> |
| <span class="source-line-no">1531</span><span id="line-1531"> });</span> |
| <span class="source-line-no">1532</span><span id="line-1532"> FileSystem fs = FileSystem.get(conf);</span> |
| <span class="source-line-no">1533</span><span id="line-1533"> // verify that the job uses TotalOrderPartitioner</span> |
| <span class="source-line-no">1534</span><span id="line-1534"> verify(job).setPartitionerClass(TotalOrderPartitioner.class);</span> |
| <span class="source-line-no">1535</span><span id="line-1535"> // verify that TotalOrderPartitioner.setPartitionFile() is called.</span> |
| <span class="source-line-no">1536</span><span id="line-1536"> String partitionPathString = conf.get("mapreduce.totalorderpartitioner.path");</span> |
| <span class="source-line-no">1537</span><span id="line-1537"> Assert.assertNotNull(partitionPathString);</span> |
| <span class="source-line-no">1538</span><span id="line-1538"> // Make sure the partion file is in foo1234's home directory, and that</span> |
| <span class="source-line-no">1539</span><span id="line-1539"> // the file exists.</span> |
| <span class="source-line-no">1540</span><span id="line-1540"> Assert.assertTrue(partitionPathString.startsWith(fooHomeDirectory.toString()));</span> |
| <span class="source-line-no">1541</span><span id="line-1541"> Assert.assertTrue(fs.exists(new Path(partitionPathString)));</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"> @Test</span> |
| <span class="source-line-no">1545</span><span id="line-1545"> public void TestConfigureCompression() throws Exception {</span> |
| <span class="source-line-no">1546</span><span id="line-1546"> Configuration conf = new Configuration(this.util.getConfiguration());</span> |
| <span class="source-line-no">1547</span><span id="line-1547"> RecordWriter<ImmutableBytesWritable, Cell> writer = null;</span> |
| <span class="source-line-no">1548</span><span id="line-1548"> TaskAttemptContext context = null;</span> |
| <span class="source-line-no">1549</span><span id="line-1549"> Path dir = util.getDataTestDir("TestConfigureCompression");</span> |
| <span class="source-line-no">1550</span><span id="line-1550"> String hfileoutputformatCompression = "gz";</span> |
| <span class="source-line-no">1551</span><span id="line-1551"></span> |
| <span class="source-line-no">1552</span><span id="line-1552"> try {</span> |
| <span class="source-line-no">1553</span><span id="line-1553"> conf.set(HFileOutputFormat2.OUTPUT_TABLE_NAME_CONF_KEY, TABLE_NAMES[0].getNameAsString());</span> |
| <span class="source-line-no">1554</span><span id="line-1554"> conf.setBoolean(HFileOutputFormat2.LOCALITY_SENSITIVE_CONF_KEY, false);</span> |
| <span class="source-line-no">1555</span><span id="line-1555"></span> |
| <span class="source-line-no">1556</span><span id="line-1556"> conf.set(HFileOutputFormat2.COMPRESSION_OVERRIDE_CONF_KEY, hfileoutputformatCompression);</span> |
| <span class="source-line-no">1557</span><span id="line-1557"></span> |
| <span class="source-line-no">1558</span><span id="line-1558"> Job job = Job.getInstance(conf);</span> |
| <span class="source-line-no">1559</span><span id="line-1559"> FileOutputFormat.setOutputPath(job, dir);</span> |
| <span class="source-line-no">1560</span><span id="line-1560"> context = createTestTaskAttemptContext(job);</span> |
| <span class="source-line-no">1561</span><span id="line-1561"> HFileOutputFormat2 hof = new HFileOutputFormat2();</span> |
| <span class="source-line-no">1562</span><span id="line-1562"> writer = hof.getRecordWriter(context);</span> |
| <span class="source-line-no">1563</span><span id="line-1563"> final byte[] b = Bytes.toBytes("b");</span> |
| <span class="source-line-no">1564</span><span id="line-1564"></span> |
| <span class="source-line-no">1565</span><span id="line-1565"> KeyValue kv = new KeyValue(b, b, b, HConstants.LATEST_TIMESTAMP, b);</span> |
| <span class="source-line-no">1566</span><span id="line-1566"> writer.write(new ImmutableBytesWritable(), kv);</span> |
| <span class="source-line-no">1567</span><span id="line-1567"> writer.close(context);</span> |
| <span class="source-line-no">1568</span><span id="line-1568"> writer = null;</span> |
| <span class="source-line-no">1569</span><span id="line-1569"> FileSystem fs = dir.getFileSystem(conf);</span> |
| <span class="source-line-no">1570</span><span id="line-1570"> RemoteIterator<LocatedFileStatus> iterator = fs.listFiles(dir, true);</span> |
| <span class="source-line-no">1571</span><span id="line-1571"> while (iterator.hasNext()) {</span> |
| <span class="source-line-no">1572</span><span id="line-1572"> LocatedFileStatus keyFileStatus = iterator.next();</span> |
| <span class="source-line-no">1573</span><span id="line-1573"> HFile.Reader reader =</span> |
| <span class="source-line-no">1574</span><span id="line-1574"> HFile.createReader(fs, keyFileStatus.getPath(), new CacheConfig(conf), true, conf);</span> |
| <span class="source-line-no">1575</span><span id="line-1575"> assertEquals(reader.getTrailer().getCompressionCodec().getName(),</span> |
| <span class="source-line-no">1576</span><span id="line-1576"> hfileoutputformatCompression);</span> |
| <span class="source-line-no">1577</span><span id="line-1577"> }</span> |
| <span class="source-line-no">1578</span><span id="line-1578"> } finally {</span> |
| <span class="source-line-no">1579</span><span id="line-1579"> if (writer != null && context != null) {</span> |
| <span class="source-line-no">1580</span><span id="line-1580"> writer.close(context);</span> |
| <span class="source-line-no">1581</span><span id="line-1581"> }</span> |
| <span class="source-line-no">1582</span><span id="line-1582"> dir.getFileSystem(conf).delete(dir, true);</span> |
| <span class="source-line-no">1583</span><span id="line-1583"> }</span> |
| <span class="source-line-no">1584</span><span id="line-1584"></span> |
| <span class="source-line-no">1585</span><span id="line-1585"> }</span> |
| <span class="source-line-no">1586</span><span id="line-1586"></span> |
| <span class="source-line-no">1587</span><span id="line-1587"> @Test</span> |
| <span class="source-line-no">1588</span><span id="line-1588"> public void testMRIncrementalLoadWithLocalityMultiCluster() throws Exception {</span> |
| <span class="source-line-no">1589</span><span id="line-1589"> // Start cluster A</span> |
| <span class="source-line-no">1590</span><span id="line-1590"> util = new HBaseTestingUtil();</span> |
| <span class="source-line-no">1591</span><span id="line-1591"> Configuration confA = util.getConfiguration();</span> |
| <span class="source-line-no">1592</span><span id="line-1592"> int hostCount = 3;</span> |
| <span class="source-line-no">1593</span><span id="line-1593"> int regionNum = 20;</span> |
| <span class="source-line-no">1594</span><span id="line-1594"> String[] hostnames = new String[hostCount];</span> |
| <span class="source-line-no">1595</span><span id="line-1595"> for (int i = 0; i < hostCount; ++i) {</span> |
| <span class="source-line-no">1596</span><span id="line-1596"> hostnames[i] = "datanode_" + i;</span> |
| <span class="source-line-no">1597</span><span id="line-1597"> }</span> |
| <span class="source-line-no">1598</span><span id="line-1598"> StartTestingClusterOption option = StartTestingClusterOption.builder()</span> |
| <span class="source-line-no">1599</span><span id="line-1599"> .numRegionServers(hostCount).dataNodeHosts(hostnames).build();</span> |
| <span class="source-line-no">1600</span><span id="line-1600"> util.startMiniCluster(option);</span> |
| <span class="source-line-no">1601</span><span id="line-1601"></span> |
| <span class="source-line-no">1602</span><span id="line-1602"> // Start cluster B</span> |
| <span class="source-line-no">1603</span><span id="line-1603"> HBaseTestingUtil utilB = new HBaseTestingUtil();</span> |
| <span class="source-line-no">1604</span><span id="line-1604"> Configuration confB = utilB.getConfiguration();</span> |
| <span class="source-line-no">1605</span><span id="line-1605"> utilB.startMiniCluster(option);</span> |
| <span class="source-line-no">1606</span><span id="line-1606"></span> |
| <span class="source-line-no">1607</span><span id="line-1607"> Path testDir = util.getDataTestDirOnTestFS("testLocalMRIncrementalLoad");</span> |
| <span class="source-line-no">1608</span><span id="line-1608"></span> |
| <span class="source-line-no">1609</span><span id="line-1609"> byte[][] splitKeys = generateRandomSplitKeys(regionNum - 1);</span> |
| <span class="source-line-no">1610</span><span id="line-1610"> TableName tableName = TableName.valueOf("table");</span> |
| <span class="source-line-no">1611</span><span id="line-1611"> // Create table in cluster B</span> |
| <span class="source-line-no">1612</span><span id="line-1612"> try (Table table = utilB.createTable(tableName, FAMILIES, splitKeys);</span> |
| <span class="source-line-no">1613</span><span id="line-1613"> RegionLocator r = utilB.getConnection().getRegionLocator(tableName)) {</span> |
| <span class="source-line-no">1614</span><span id="line-1614"> // Generate the bulk load files</span> |
| <span class="source-line-no">1615</span><span id="line-1615"> // Job has zookeeper configuration for cluster A</span> |
| <span class="source-line-no">1616</span><span id="line-1616"> // Assume reading from cluster A by TableInputFormat and creating hfiles to cluster B</span> |
| <span class="source-line-no">1617</span><span id="line-1617"> Job job = new Job(confA, "testLocalMRIncrementalLoad");</span> |
| <span class="source-line-no">1618</span><span id="line-1618"> Configuration jobConf = job.getConfiguration();</span> |
| <span class="source-line-no">1619</span><span id="line-1619"> final UUID key = ConfigurationCaptorConnection.configureConnectionImpl(jobConf);</span> |
| <span class="source-line-no">1620</span><span id="line-1620"> job.setWorkingDirectory(util.getDataTestDirOnTestFS("runIncrementalPELoad"));</span> |
| <span class="source-line-no">1621</span><span id="line-1621"> setupRandomGeneratorMapper(job, false);</span> |
| <span class="source-line-no">1622</span><span id="line-1622"> HFileOutputFormat2.configureIncrementalLoad(job, table, r);</span> |
| <span class="source-line-no">1623</span><span id="line-1623"></span> |
| <span class="source-line-no">1624</span><span id="line-1624"> assertEquals(confB.get(HConstants.ZOOKEEPER_QUORUM),</span> |
| <span class="source-line-no">1625</span><span id="line-1625"> jobConf.get(HFileOutputFormat2.REMOTE_CLUSTER_ZOOKEEPER_QUORUM_CONF_KEY));</span> |
| <span class="source-line-no">1626</span><span id="line-1626"> assertEquals(confB.get(HConstants.ZOOKEEPER_CLIENT_PORT),</span> |
| <span class="source-line-no">1627</span><span id="line-1627"> jobConf.get(HFileOutputFormat2.REMOTE_CLUSTER_ZOOKEEPER_CLIENT_PORT_CONF_KEY));</span> |
| <span class="source-line-no">1628</span><span id="line-1628"> assertEquals(confB.get(HConstants.ZOOKEEPER_ZNODE_PARENT),</span> |
| <span class="source-line-no">1629</span><span id="line-1629"> jobConf.get(HFileOutputFormat2.REMOTE_CLUSTER_ZOOKEEPER_ZNODE_PARENT_CONF_KEY));</span> |
| <span class="source-line-no">1630</span><span id="line-1630"></span> |
| <span class="source-line-no">1631</span><span id="line-1631"> String bSpecificConfigKey = "my.override.config.for.b";</span> |
| <span class="source-line-no">1632</span><span id="line-1632"> String bSpecificConfigValue = "b-specific-value";</span> |
| <span class="source-line-no">1633</span><span id="line-1633"> jobConf.set(HFileOutputFormat2.REMOTE_CLUSTER_CONF_PREFIX + bSpecificConfigKey,</span> |
| <span class="source-line-no">1634</span><span id="line-1634"> bSpecificConfigValue);</span> |
| <span class="source-line-no">1635</span><span id="line-1635"></span> |
| <span class="source-line-no">1636</span><span id="line-1636"> FileOutputFormat.setOutputPath(job, testDir);</span> |
| <span class="source-line-no">1637</span><span id="line-1637"></span> |
| <span class="source-line-no">1638</span><span id="line-1638"> assertFalse(util.getTestFileSystem().exists(testDir));</span> |
| <span class="source-line-no">1639</span><span id="line-1639"></span> |
| <span class="source-line-no">1640</span><span id="line-1640"> assertTrue(job.waitForCompletion(true));</span> |
| <span class="source-line-no">1641</span><span id="line-1641"></span> |
| <span class="source-line-no">1642</span><span id="line-1642"> final List<Configuration> configs =</span> |
| <span class="source-line-no">1643</span><span id="line-1643"> ConfigurationCaptorConnection.getCapturedConfigarutions(key);</span> |
| <span class="source-line-no">1644</span><span id="line-1644"></span> |
| <span class="source-line-no">1645</span><span id="line-1645"> assertFalse(configs.isEmpty());</span> |
| <span class="source-line-no">1646</span><span id="line-1646"> for (Configuration config : configs) {</span> |
| <span class="source-line-no">1647</span><span id="line-1647"> assertEquals(confB.get(HConstants.ZOOKEEPER_QUORUM),</span> |
| <span class="source-line-no">1648</span><span id="line-1648"> config.get(HConstants.ZOOKEEPER_QUORUM));</span> |
| <span class="source-line-no">1649</span><span id="line-1649"> assertEquals(confB.get(HConstants.ZOOKEEPER_CLIENT_PORT),</span> |
| <span class="source-line-no">1650</span><span id="line-1650"> config.get(HConstants.ZOOKEEPER_CLIENT_PORT));</span> |
| <span class="source-line-no">1651</span><span id="line-1651"> assertEquals(confB.get(HConstants.ZOOKEEPER_ZNODE_PARENT),</span> |
| <span class="source-line-no">1652</span><span id="line-1652"> config.get(HConstants.ZOOKEEPER_ZNODE_PARENT));</span> |
| <span class="source-line-no">1653</span><span id="line-1653"></span> |
| <span class="source-line-no">1654</span><span id="line-1654"> assertEquals(bSpecificConfigValue, config.get(bSpecificConfigKey));</span> |
| <span class="source-line-no">1655</span><span id="line-1655"> }</span> |
| <span class="source-line-no">1656</span><span id="line-1656"> } finally {</span> |
| <span class="source-line-no">1657</span><span id="line-1657"> utilB.deleteTable(tableName);</span> |
| <span class="source-line-no">1658</span><span id="line-1658"> testDir.getFileSystem(confA).delete(testDir, true);</span> |
| <span class="source-line-no">1659</span><span id="line-1659"> util.shutdownMiniCluster();</span> |
| <span class="source-line-no">1660</span><span id="line-1660"> utilB.shutdownMiniCluster();</span> |
| <span class="source-line-no">1661</span><span id="line-1661"> }</span> |
| <span class="source-line-no">1662</span><span id="line-1662"> }</span> |
| <span class="source-line-no">1663</span><span id="line-1663"></span> |
| <span class="source-line-no">1664</span><span id="line-1664"> private static class ConfigurationCaptorConnection implements Connection {</span> |
| <span class="source-line-no">1665</span><span id="line-1665"> private static final String UUID_KEY = "ConfigurationCaptorConnection.uuid";</span> |
| <span class="source-line-no">1666</span><span id="line-1666"></span> |
| <span class="source-line-no">1667</span><span id="line-1667"> private static final Map<UUID, List<Configuration>> confs = new ConcurrentHashMap<>();</span> |
| <span class="source-line-no">1668</span><span id="line-1668"></span> |
| <span class="source-line-no">1669</span><span id="line-1669"> private final Connection delegate;</span> |
| <span class="source-line-no">1670</span><span id="line-1670"></span> |
| <span class="source-line-no">1671</span><span id="line-1671"> public ConfigurationCaptorConnection(Configuration conf, ExecutorService es, User user,</span> |
| <span class="source-line-no">1672</span><span id="line-1672"> ConnectionRegistry registry, Map<String, byte[]> connectionAttributes) throws IOException {</span> |
| <span class="source-line-no">1673</span><span id="line-1673"> // here we do not use this registry, so close it...</span> |
| <span class="source-line-no">1674</span><span id="line-1674"> registry.close();</span> |
| <span class="source-line-no">1675</span><span id="line-1675"> // here we use createAsyncConnection, to avoid infinite recursive as we reset the Connection</span> |
| <span class="source-line-no">1676</span><span id="line-1676"> // implementation in below method</span> |
| <span class="source-line-no">1677</span><span id="line-1677"> delegate =</span> |
| <span class="source-line-no">1678</span><span id="line-1678"> FutureUtils.get(ConnectionFactory.createAsyncConnection(conf, user, connectionAttributes))</span> |
| <span class="source-line-no">1679</span><span id="line-1679"> .toConnection();</span> |
| <span class="source-line-no">1680</span><span id="line-1680"></span> |
| <span class="source-line-no">1681</span><span id="line-1681"> final String uuid = conf.get(UUID_KEY);</span> |
| <span class="source-line-no">1682</span><span id="line-1682"> if (uuid != null) {</span> |
| <span class="source-line-no">1683</span><span id="line-1683"> confs.computeIfAbsent(UUID.fromString(uuid), u -> new CopyOnWriteArrayList<>()).add(conf);</span> |
| <span class="source-line-no">1684</span><span id="line-1684"> }</span> |
| <span class="source-line-no">1685</span><span id="line-1685"> }</span> |
| <span class="source-line-no">1686</span><span id="line-1686"></span> |
| <span class="source-line-no">1687</span><span id="line-1687"> static UUID configureConnectionImpl(Configuration conf) {</span> |
| <span class="source-line-no">1688</span><span id="line-1688"> conf.setClass(ConnectionUtils.HBASE_CLIENT_CONNECTION_IMPL,</span> |
| <span class="source-line-no">1689</span><span id="line-1689"> ConfigurationCaptorConnection.class, Connection.class);</span> |
| <span class="source-line-no">1690</span><span id="line-1690"></span> |
| <span class="source-line-no">1691</span><span id="line-1691"> final UUID uuid = UUID.randomUUID();</span> |
| <span class="source-line-no">1692</span><span id="line-1692"> conf.set(UUID_KEY, uuid.toString());</span> |
| <span class="source-line-no">1693</span><span id="line-1693"> return uuid;</span> |
| <span class="source-line-no">1694</span><span id="line-1694"> }</span> |
| <span class="source-line-no">1695</span><span id="line-1695"></span> |
| <span class="source-line-no">1696</span><span id="line-1696"> static List<Configuration> getCapturedConfigarutions(UUID key) {</span> |
| <span class="source-line-no">1697</span><span id="line-1697"> return confs.get(key);</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"> @Override</span> |
| <span class="source-line-no">1701</span><span id="line-1701"> public Configuration getConfiguration() {</span> |
| <span class="source-line-no">1702</span><span id="line-1702"> return delegate.getConfiguration();</span> |
| <span class="source-line-no">1703</span><span id="line-1703"> }</span> |
| <span class="source-line-no">1704</span><span id="line-1704"></span> |
| <span class="source-line-no">1705</span><span id="line-1705"> @Override</span> |
| <span class="source-line-no">1706</span><span id="line-1706"> public Table getTable(TableName tableName) throws IOException {</span> |
| <span class="source-line-no">1707</span><span id="line-1707"> return delegate.getTable(tableName);</span> |
| <span class="source-line-no">1708</span><span id="line-1708"> }</span> |
| <span class="source-line-no">1709</span><span id="line-1709"></span> |
| <span class="source-line-no">1710</span><span id="line-1710"> @Override</span> |
| <span class="source-line-no">1711</span><span id="line-1711"> public Table getTable(TableName tableName, ExecutorService pool) throws IOException {</span> |
| <span class="source-line-no">1712</span><span id="line-1712"> return delegate.getTable(tableName, pool);</span> |
| <span class="source-line-no">1713</span><span id="line-1713"> }</span> |
| <span class="source-line-no">1714</span><span id="line-1714"></span> |
| <span class="source-line-no">1715</span><span id="line-1715"> @Override</span> |
| <span class="source-line-no">1716</span><span id="line-1716"> public BufferedMutator getBufferedMutator(TableName tableName) throws IOException {</span> |
| <span class="source-line-no">1717</span><span id="line-1717"> return delegate.getBufferedMutator(tableName);</span> |
| <span class="source-line-no">1718</span><span id="line-1718"> }</span> |
| <span class="source-line-no">1719</span><span id="line-1719"></span> |
| <span class="source-line-no">1720</span><span id="line-1720"> @Override</span> |
| <span class="source-line-no">1721</span><span id="line-1721"> public BufferedMutator getBufferedMutator(BufferedMutatorParams params) throws IOException {</span> |
| <span class="source-line-no">1722</span><span id="line-1722"> return delegate.getBufferedMutator(params);</span> |
| <span class="source-line-no">1723</span><span id="line-1723"> }</span> |
| <span class="source-line-no">1724</span><span id="line-1724"></span> |
| <span class="source-line-no">1725</span><span id="line-1725"> @Override</span> |
| <span class="source-line-no">1726</span><span id="line-1726"> public RegionLocator getRegionLocator(TableName tableName) throws IOException {</span> |
| <span class="source-line-no">1727</span><span id="line-1727"> return delegate.getRegionLocator(tableName);</span> |
| <span class="source-line-no">1728</span><span id="line-1728"> }</span> |
| <span class="source-line-no">1729</span><span id="line-1729"></span> |
| <span class="source-line-no">1730</span><span id="line-1730"> @Override</span> |
| <span class="source-line-no">1731</span><span id="line-1731"> public void clearRegionLocationCache() {</span> |
| <span class="source-line-no">1732</span><span id="line-1732"> delegate.clearRegionLocationCache();</span> |
| <span class="source-line-no">1733</span><span id="line-1733"> }</span> |
| <span class="source-line-no">1734</span><span id="line-1734"></span> |
| <span class="source-line-no">1735</span><span id="line-1735"> @Override</span> |
| <span class="source-line-no">1736</span><span id="line-1736"> public Admin getAdmin() throws IOException {</span> |
| <span class="source-line-no">1737</span><span id="line-1737"> return delegate.getAdmin();</span> |
| <span class="source-line-no">1738</span><span id="line-1738"> }</span> |
| <span class="source-line-no">1739</span><span id="line-1739"></span> |
| <span class="source-line-no">1740</span><span id="line-1740"> @Override</span> |
| <span class="source-line-no">1741</span><span id="line-1741"> public void close() throws IOException {</span> |
| <span class="source-line-no">1742</span><span id="line-1742"> delegate.close();</span> |
| <span class="source-line-no">1743</span><span id="line-1743"> }</span> |
| <span class="source-line-no">1744</span><span id="line-1744"></span> |
| <span class="source-line-no">1745</span><span id="line-1745"> @Override</span> |
| <span class="source-line-no">1746</span><span id="line-1746"> public boolean isClosed() {</span> |
| <span class="source-line-no">1747</span><span id="line-1747"> return delegate.isClosed();</span> |
| <span class="source-line-no">1748</span><span id="line-1748"> }</span> |
| <span class="source-line-no">1749</span><span id="line-1749"></span> |
| <span class="source-line-no">1750</span><span id="line-1750"> @Override</span> |
| <span class="source-line-no">1751</span><span id="line-1751"> public TableBuilder getTableBuilder(TableName tableName, ExecutorService pool) {</span> |
| <span class="source-line-no">1752</span><span id="line-1752"> return delegate.getTableBuilder(tableName, pool);</span> |
| <span class="source-line-no">1753</span><span id="line-1753"> }</span> |
| <span class="source-line-no">1754</span><span id="line-1754"></span> |
| <span class="source-line-no">1755</span><span id="line-1755"> @Override</span> |
| <span class="source-line-no">1756</span><span id="line-1756"> public AsyncConnection toAsyncConnection() {</span> |
| <span class="source-line-no">1757</span><span id="line-1757"> return delegate.toAsyncConnection();</span> |
| <span class="source-line-no">1758</span><span id="line-1758"> }</span> |
| <span class="source-line-no">1759</span><span id="line-1759"></span> |
| <span class="source-line-no">1760</span><span id="line-1760"> @Override</span> |
| <span class="source-line-no">1761</span><span id="line-1761"> public String getClusterId() {</span> |
| <span class="source-line-no">1762</span><span id="line-1762"> return delegate.getClusterId();</span> |
| <span class="source-line-no">1763</span><span id="line-1763"> }</span> |
| <span class="source-line-no">1764</span><span id="line-1764"></span> |
| <span class="source-line-no">1765</span><span id="line-1765"> @Override</span> |
| <span class="source-line-no">1766</span><span id="line-1766"> public Hbck getHbck() throws IOException {</span> |
| <span class="source-line-no">1767</span><span id="line-1767"> return delegate.getHbck();</span> |
| <span class="source-line-no">1768</span><span id="line-1768"> }</span> |
| <span class="source-line-no">1769</span><span id="line-1769"></span> |
| <span class="source-line-no">1770</span><span id="line-1770"> @Override</span> |
| <span class="source-line-no">1771</span><span id="line-1771"> public Hbck getHbck(ServerName masterServer) throws IOException {</span> |
| <span class="source-line-no">1772</span><span id="line-1772"> return delegate.getHbck(masterServer);</span> |
| <span class="source-line-no">1773</span><span id="line-1773"> }</span> |
| <span class="source-line-no">1774</span><span id="line-1774"></span> |
| <span class="source-line-no">1775</span><span id="line-1775"> @Override</span> |
| <span class="source-line-no">1776</span><span id="line-1776"> public void abort(String why, Throwable e) {</span> |
| <span class="source-line-no">1777</span><span id="line-1777"> delegate.abort(why, e);</span> |
| <span class="source-line-no">1778</span><span id="line-1778"> }</span> |
| <span class="source-line-no">1779</span><span id="line-1779"></span> |
| <span class="source-line-no">1780</span><span id="line-1780"> @Override</span> |
| <span class="source-line-no">1781</span><span id="line-1781"> public boolean isAborted() {</span> |
| <span class="source-line-no">1782</span><span id="line-1782"> return delegate.isAborted();</span> |
| <span class="source-line-no">1783</span><span id="line-1783"> }</span> |
| <span class="source-line-no">1784</span><span id="line-1784"> }</span> |
| <span class="source-line-no">1785</span><span id="line-1785"></span> |
| <span class="source-line-no">1786</span><span id="line-1786">}</span> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </pre> |
| </div> |
| </main> |
| </body> |
| </html> |