blob: 2e30a72bf505ab118e1a651f77373429d3914de4 [file] [log] [blame]
<!DOCTYPE HTML>
<html lang="en">
<head>
<!-- Generated by javadoc (17) -->
<title>Source code</title>
<meta name="viewport" content="width=device-width, initial-scale=1">
<meta name="description" content="source: package: org.apache.hadoop.hbase.tool, class: BulkLoadHFilesTool">
<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.tool;</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 java.lang.String.format;</span>
<span class="source-line-no">021</span><span id="line-21"></span>
<span class="source-line-no">022</span><span id="line-22">import java.io.FileNotFoundException;</span>
<span class="source-line-no">023</span><span id="line-23">import java.io.IOException;</span>
<span class="source-line-no">024</span><span id="line-24">import java.io.InterruptedIOException;</span>
<span class="source-line-no">025</span><span id="line-25">import java.net.InetSocketAddress;</span>
<span class="source-line-no">026</span><span id="line-26">import java.nio.ByteBuffer;</span>
<span class="source-line-no">027</span><span id="line-27">import java.util.ArrayDeque;</span>
<span class="source-line-no">028</span><span id="line-28">import java.util.ArrayList;</span>
<span class="source-line-no">029</span><span id="line-29">import java.util.Arrays;</span>
<span class="source-line-no">030</span><span id="line-30">import java.util.Collection;</span>
<span class="source-line-no">031</span><span id="line-31">import java.util.Collections;</span>
<span class="source-line-no">032</span><span id="line-32">import java.util.Deque;</span>
<span class="source-line-no">033</span><span id="line-33">import java.util.HashMap;</span>
<span class="source-line-no">034</span><span id="line-34">import java.util.HashSet;</span>
<span class="source-line-no">035</span><span id="line-35">import java.util.List;</span>
<span class="source-line-no">036</span><span id="line-36">import java.util.Map;</span>
<span class="source-line-no">037</span><span id="line-37">import java.util.Map.Entry;</span>
<span class="source-line-no">038</span><span id="line-38">import java.util.Optional;</span>
<span class="source-line-no">039</span><span id="line-39">import java.util.Set;</span>
<span class="source-line-no">040</span><span id="line-40">import java.util.SortedMap;</span>
<span class="source-line-no">041</span><span id="line-41">import java.util.TreeMap;</span>
<span class="source-line-no">042</span><span id="line-42">import java.util.UUID;</span>
<span class="source-line-no">043</span><span id="line-43">import java.util.concurrent.Callable;</span>
<span class="source-line-no">044</span><span id="line-44">import java.util.concurrent.CompletableFuture;</span>
<span class="source-line-no">045</span><span id="line-45">import java.util.concurrent.ExecutionException;</span>
<span class="source-line-no">046</span><span id="line-46">import java.util.concurrent.ExecutorService;</span>
<span class="source-line-no">047</span><span id="line-47">import java.util.concurrent.Future;</span>
<span class="source-line-no">048</span><span id="line-48">import java.util.concurrent.LinkedBlockingQueue;</span>
<span class="source-line-no">049</span><span id="line-49">import java.util.concurrent.ThreadPoolExecutor;</span>
<span class="source-line-no">050</span><span id="line-50">import java.util.concurrent.TimeUnit;</span>
<span class="source-line-no">051</span><span id="line-51">import java.util.concurrent.atomic.AtomicInteger;</span>
<span class="source-line-no">052</span><span id="line-52">import java.util.stream.Collectors;</span>
<span class="source-line-no">053</span><span id="line-53">import org.apache.commons.lang3.mutable.MutableInt;</span>
<span class="source-line-no">054</span><span id="line-54">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">055</span><span id="line-55">import org.apache.hadoop.conf.Configured;</span>
<span class="source-line-no">056</span><span id="line-56">import org.apache.hadoop.fs.FileStatus;</span>
<span class="source-line-no">057</span><span id="line-57">import org.apache.hadoop.fs.FileSystem;</span>
<span class="source-line-no">058</span><span id="line-58">import org.apache.hadoop.fs.Path;</span>
<span class="source-line-no">059</span><span id="line-59">import org.apache.hadoop.fs.permission.FsPermission;</span>
<span class="source-line-no">060</span><span id="line-60">import org.apache.hadoop.hbase.Cell;</span>
<span class="source-line-no">061</span><span id="line-61">import org.apache.hadoop.hbase.CellUtil;</span>
<span class="source-line-no">062</span><span id="line-62">import org.apache.hadoop.hbase.ExtendedCell;</span>
<span class="source-line-no">063</span><span id="line-63">import org.apache.hadoop.hbase.HBaseConfiguration;</span>
<span class="source-line-no">064</span><span id="line-64">import org.apache.hadoop.hbase.HBaseInterfaceAudience;</span>
<span class="source-line-no">065</span><span id="line-65">import org.apache.hadoop.hbase.HConstants;</span>
<span class="source-line-no">066</span><span id="line-66">import org.apache.hadoop.hbase.HRegionLocation;</span>
<span class="source-line-no">067</span><span id="line-67">import org.apache.hadoop.hbase.KeyValue;</span>
<span class="source-line-no">068</span><span id="line-68">import org.apache.hadoop.hbase.TableName;</span>
<span class="source-line-no">069</span><span id="line-69">import org.apache.hadoop.hbase.TableNotFoundException;</span>
<span class="source-line-no">070</span><span id="line-70">import org.apache.hadoop.hbase.client.AsyncAdmin;</span>
<span class="source-line-no">071</span><span id="line-71">import org.apache.hadoop.hbase.client.AsyncClusterConnection;</span>
<span class="source-line-no">072</span><span id="line-72">import org.apache.hadoop.hbase.client.AsyncTableRegionLocator;</span>
<span class="source-line-no">073</span><span id="line-73">import org.apache.hadoop.hbase.client.ClusterConnectionFactory;</span>
<span class="source-line-no">074</span><span id="line-74">import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;</span>
<span class="source-line-no">075</span><span id="line-75">import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;</span>
<span class="source-line-no">076</span><span id="line-76">import org.apache.hadoop.hbase.client.TableDescriptor;</span>
<span class="source-line-no">077</span><span id="line-77">import org.apache.hadoop.hbase.client.TableDescriptorBuilder;</span>
<span class="source-line-no">078</span><span id="line-78">import org.apache.hadoop.hbase.io.HFileLink;</span>
<span class="source-line-no">079</span><span id="line-79">import org.apache.hadoop.hbase.io.Reference;</span>
<span class="source-line-no">080</span><span id="line-80">import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;</span>
<span class="source-line-no">081</span><span id="line-81">import org.apache.hadoop.hbase.io.hfile.CacheConfig;</span>
<span class="source-line-no">082</span><span id="line-82">import org.apache.hadoop.hbase.io.hfile.HFile;</span>
<span class="source-line-no">083</span><span id="line-83">import org.apache.hadoop.hbase.io.hfile.HFileContext;</span>
<span class="source-line-no">084</span><span id="line-84">import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;</span>
<span class="source-line-no">085</span><span id="line-85">import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;</span>
<span class="source-line-no">086</span><span id="line-86">import org.apache.hadoop.hbase.io.hfile.HFileInfo;</span>
<span class="source-line-no">087</span><span id="line-87">import org.apache.hadoop.hbase.io.hfile.ReaderContext;</span>
<span class="source-line-no">088</span><span id="line-88">import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder;</span>
<span class="source-line-no">089</span><span id="line-89">import org.apache.hadoop.hbase.regionserver.BloomType;</span>
<span class="source-line-no">090</span><span id="line-90">import org.apache.hadoop.hbase.regionserver.StoreFileInfo;</span>
<span class="source-line-no">091</span><span id="line-91">import org.apache.hadoop.hbase.regionserver.StoreFileReader;</span>
<span class="source-line-no">092</span><span id="line-92">import org.apache.hadoop.hbase.regionserver.StoreFileScanner;</span>
<span class="source-line-no">093</span><span id="line-93">import org.apache.hadoop.hbase.regionserver.StoreFileWriter;</span>
<span class="source-line-no">094</span><span id="line-94">import org.apache.hadoop.hbase.regionserver.StoreUtils;</span>
<span class="source-line-no">095</span><span id="line-95">import org.apache.hadoop.hbase.security.UserProvider;</span>
<span class="source-line-no">096</span><span id="line-96">import org.apache.hadoop.hbase.security.token.FsDelegationToken;</span>
<span class="source-line-no">097</span><span id="line-97">import org.apache.hadoop.hbase.util.Bytes;</span>
<span class="source-line-no">098</span><span id="line-98">import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;</span>
<span class="source-line-no">099</span><span id="line-99">import org.apache.hadoop.hbase.util.FSVisitor;</span>
<span class="source-line-no">100</span><span id="line-100">import org.apache.hadoop.hbase.util.FutureUtils;</span>
<span class="source-line-no">101</span><span id="line-101">import org.apache.hadoop.hbase.util.Pair;</span>
<span class="source-line-no">102</span><span id="line-102">import org.apache.hadoop.util.Tool;</span>
<span class="source-line-no">103</span><span id="line-103">import org.apache.hadoop.util.ToolRunner;</span>
<span class="source-line-no">104</span><span id="line-104">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">105</span><span id="line-105">import org.slf4j.Logger;</span>
<span class="source-line-no">106</span><span id="line-106">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">107</span><span id="line-107"></span>
<span class="source-line-no">108</span><span id="line-108">import org.apache.hbase.thirdparty.com.google.common.collect.HashMultimap;</span>
<span class="source-line-no">109</span><span id="line-109">import org.apache.hbase.thirdparty.com.google.common.collect.Lists;</span>
<span class="source-line-no">110</span><span id="line-110">import org.apache.hbase.thirdparty.com.google.common.collect.Maps;</span>
<span class="source-line-no">111</span><span id="line-111">import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;</span>
<span class="source-line-no">112</span><span id="line-112">import org.apache.hbase.thirdparty.com.google.common.collect.Multimaps;</span>
<span class="source-line-no">113</span><span id="line-113">import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;</span>
<span class="source-line-no">114</span><span id="line-114"></span>
<span class="source-line-no">115</span><span id="line-115">/**</span>
<span class="source-line-no">116</span><span id="line-116"> * The implementation for {@link BulkLoadHFiles}, and also can be executed from command line as a</span>
<span class="source-line-no">117</span><span id="line-117"> * tool.</span>
<span class="source-line-no">118</span><span id="line-118"> */</span>
<span class="source-line-no">119</span><span id="line-119">@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)</span>
<span class="source-line-no">120</span><span id="line-120">public class BulkLoadHFilesTool extends Configured implements BulkLoadHFiles, Tool {</span>
<span class="source-line-no">121</span><span id="line-121"></span>
<span class="source-line-no">122</span><span id="line-122"> private static final Logger LOG = LoggerFactory.getLogger(BulkLoadHFilesTool.class);</span>
<span class="source-line-no">123</span><span id="line-123"></span>
<span class="source-line-no">124</span><span id="line-124"> /**</span>
<span class="source-line-no">125</span><span id="line-125"> * Keep locality while generating HFiles for bulkload. See HBASE-12596</span>
<span class="source-line-no">126</span><span id="line-126"> */</span>
<span class="source-line-no">127</span><span id="line-127"> public static final String LOCALITY_SENSITIVE_CONF_KEY =</span>
<span class="source-line-no">128</span><span id="line-128"> "hbase.bulkload.locality.sensitive.enabled";</span>
<span class="source-line-no">129</span><span id="line-129"> private static final boolean DEFAULT_LOCALITY_SENSITIVE = true;</span>
<span class="source-line-no">130</span><span id="line-130"></span>
<span class="source-line-no">131</span><span id="line-131"> public static final String NAME = "completebulkload";</span>
<span class="source-line-no">132</span><span id="line-132"> /**</span>
<span class="source-line-no">133</span><span id="line-133"> * Whether to run validation on hfiles before loading.</span>
<span class="source-line-no">134</span><span id="line-134"> */</span>
<span class="source-line-no">135</span><span id="line-135"> private static final String VALIDATE_HFILES = "hbase.loadincremental.validate.hfile";</span>
<span class="source-line-no">136</span><span id="line-136"> /**</span>
<span class="source-line-no">137</span><span id="line-137"> * HBASE-24221 Support bulkLoadHFile by family to avoid long time waiting of bulkLoadHFile because</span>
<span class="source-line-no">138</span><span id="line-138"> * of compacting at server side</span>
<span class="source-line-no">139</span><span id="line-139"> */</span>
<span class="source-line-no">140</span><span id="line-140"> public static final String BULK_LOAD_HFILES_BY_FAMILY = "hbase.mapreduce.bulkload.by.family";</span>
<span class="source-line-no">141</span><span id="line-141"></span>
<span class="source-line-no">142</span><span id="line-142"> public static final String FAIL_IF_NEED_SPLIT_HFILE =</span>
<span class="source-line-no">143</span><span id="line-143"> "hbase.loadincremental.fail.if.need.split.hfile";</span>
<span class="source-line-no">144</span><span id="line-144"></span>
<span class="source-line-no">145</span><span id="line-145"> // We use a '.' prefix which is ignored when walking directory trees</span>
<span class="source-line-no">146</span><span id="line-146"> // above. It is invalid family name.</span>
<span class="source-line-no">147</span><span id="line-147"> static final String TMP_DIR = ".tmp";</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 int maxFilesPerRegionPerFamily;</span>
<span class="source-line-no">150</span><span id="line-150"> private boolean assignSeqIds;</span>
<span class="source-line-no">151</span><span id="line-151"> private boolean bulkLoadByFamily;</span>
<span class="source-line-no">152</span><span id="line-152"></span>
<span class="source-line-no">153</span><span id="line-153"> // Source delegation token</span>
<span class="source-line-no">154</span><span id="line-154"> private FsDelegationToken fsDelegationToken;</span>
<span class="source-line-no">155</span><span id="line-155"> private UserProvider userProvider;</span>
<span class="source-line-no">156</span><span id="line-156"> private int nrThreads;</span>
<span class="source-line-no">157</span><span id="line-157"> private final AtomicInteger numRetries = new AtomicInteger(0);</span>
<span class="source-line-no">158</span><span id="line-158"> private String bulkToken;</span>
<span class="source-line-no">159</span><span id="line-159"></span>
<span class="source-line-no">160</span><span id="line-160"> private List&lt;String&gt; clusterIds = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">161</span><span id="line-161"> private boolean replicate = true;</span>
<span class="source-line-no">162</span><span id="line-162"> private boolean failIfNeedSplitHFile = false;</span>
<span class="source-line-no">163</span><span id="line-163"></span>
<span class="source-line-no">164</span><span id="line-164"> public BulkLoadHFilesTool(Configuration conf) {</span>
<span class="source-line-no">165</span><span id="line-165"> // make a copy, just to be sure we're not overriding someone else's config</span>
<span class="source-line-no">166</span><span id="line-166"> super(new Configuration(conf));</span>
<span class="source-line-no">167</span><span id="line-167"> initialize();</span>
<span class="source-line-no">168</span><span id="line-168"> }</span>
<span class="source-line-no">169</span><span id="line-169"></span>
<span class="source-line-no">170</span><span id="line-170"> public void initialize() {</span>
<span class="source-line-no">171</span><span id="line-171"> Configuration conf = getConf();</span>
<span class="source-line-no">172</span><span id="line-172"> // disable blockcache for tool invocation, see HBASE-10500</span>
<span class="source-line-no">173</span><span id="line-173"> conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0);</span>
<span class="source-line-no">174</span><span id="line-174"> userProvider = UserProvider.instantiate(conf);</span>
<span class="source-line-no">175</span><span id="line-175"> fsDelegationToken = new FsDelegationToken(userProvider, "renewer");</span>
<span class="source-line-no">176</span><span id="line-176"> assignSeqIds = conf.getBoolean(ASSIGN_SEQ_IDS, true);</span>
<span class="source-line-no">177</span><span id="line-177"> maxFilesPerRegionPerFamily = conf.getInt(MAX_FILES_PER_REGION_PER_FAMILY, 32);</span>
<span class="source-line-no">178</span><span id="line-178"> nrThreads =</span>
<span class="source-line-no">179</span><span id="line-179"> conf.getInt("hbase.loadincremental.threads.max", Runtime.getRuntime().availableProcessors());</span>
<span class="source-line-no">180</span><span id="line-180"> bulkLoadByFamily = conf.getBoolean(BULK_LOAD_HFILES_BY_FAMILY, false);</span>
<span class="source-line-no">181</span><span id="line-181"> failIfNeedSplitHFile = conf.getBoolean(FAIL_IF_NEED_SPLIT_HFILE, false);</span>
<span class="source-line-no">182</span><span id="line-182"> }</span>
<span class="source-line-no">183</span><span id="line-183"></span>
<span class="source-line-no">184</span><span id="line-184"> // Initialize a thread pool</span>
<span class="source-line-no">185</span><span id="line-185"> private ExecutorService createExecutorService() {</span>
<span class="source-line-no">186</span><span id="line-186"> ThreadPoolExecutor pool = new ThreadPoolExecutor(nrThreads, nrThreads, 60, TimeUnit.SECONDS,</span>
<span class="source-line-no">187</span><span id="line-187"> new LinkedBlockingQueue&lt;&gt;(),</span>
<span class="source-line-no">188</span><span id="line-188"> new ThreadFactoryBuilder().setNameFormat("BulkLoadHFilesTool-%1$d").setDaemon(true).build());</span>
<span class="source-line-no">189</span><span id="line-189"> pool.allowCoreThreadTimeOut(true);</span>
<span class="source-line-no">190</span><span id="line-190"> return pool;</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"> private boolean isCreateTable() {</span>
<span class="source-line-no">194</span><span id="line-194"> return "yes".equalsIgnoreCase(getConf().get(CREATE_TABLE_CONF_KEY, "yes"));</span>
<span class="source-line-no">195</span><span id="line-195"> }</span>
<span class="source-line-no">196</span><span id="line-196"></span>
<span class="source-line-no">197</span><span id="line-197"> private boolean isSilence() {</span>
<span class="source-line-no">198</span><span id="line-198"> return "yes".equalsIgnoreCase(getConf().get(IGNORE_UNMATCHED_CF_CONF_KEY, ""));</span>
<span class="source-line-no">199</span><span id="line-199"> }</span>
<span class="source-line-no">200</span><span id="line-200"></span>
<span class="source-line-no">201</span><span id="line-201"> private boolean isAlwaysCopyFiles() {</span>
<span class="source-line-no">202</span><span id="line-202"> return getConf().getBoolean(ALWAYS_COPY_FILES, false);</span>
<span class="source-line-no">203</span><span id="line-203"> }</span>
<span class="source-line-no">204</span><span id="line-204"></span>
<span class="source-line-no">205</span><span id="line-205"> private static boolean shouldCopyHFileMetaKey(byte[] key) {</span>
<span class="source-line-no">206</span><span id="line-206"> // skip encoding to keep hfile meta consistent with data block info, see HBASE-15085</span>
<span class="source-line-no">207</span><span id="line-207"> if (Bytes.equals(key, HFileDataBlockEncoder.DATA_BLOCK_ENCODING)) {</span>
<span class="source-line-no">208</span><span id="line-208"> return false;</span>
<span class="source-line-no">209</span><span id="line-209"> }</span>
<span class="source-line-no">210</span><span id="line-210"></span>
<span class="source-line-no">211</span><span id="line-211"> return !HFileInfo.isReservedFileInfoKey(key);</span>
<span class="source-line-no">212</span><span id="line-212"> }</span>
<span class="source-line-no">213</span><span id="line-213"></span>
<span class="source-line-no">214</span><span id="line-214"> /**</span>
<span class="source-line-no">215</span><span id="line-215"> * Checks whether there is any invalid family name in HFiles to be bulk loaded.</span>
<span class="source-line-no">216</span><span id="line-216"> */</span>
<span class="source-line-no">217</span><span id="line-217"> private static void validateFamiliesInHFiles(TableDescriptor tableDesc,</span>
<span class="source-line-no">218</span><span id="line-218"> Deque&lt;LoadQueueItem&gt; queue, boolean silence) throws IOException {</span>
<span class="source-line-no">219</span><span id="line-219"> Set&lt;String&gt; familyNames = Arrays.stream(tableDesc.getColumnFamilies())</span>
<span class="source-line-no">220</span><span id="line-220"> .map(ColumnFamilyDescriptor::getNameAsString).collect(Collectors.toSet());</span>
<span class="source-line-no">221</span><span id="line-221"> List&lt;String&gt; unmatchedFamilies = queue.stream().map(item -&gt; Bytes.toString(item.getFamily()))</span>
<span class="source-line-no">222</span><span id="line-222"> .filter(fn -&gt; !familyNames.contains(fn)).distinct().collect(Collectors.toList());</span>
<span class="source-line-no">223</span><span id="line-223"> if (unmatchedFamilies.size() &gt; 0) {</span>
<span class="source-line-no">224</span><span id="line-224"> String msg =</span>
<span class="source-line-no">225</span><span id="line-225"> "Unmatched family names found: unmatched family names in HFiles to be bulkloaded: "</span>
<span class="source-line-no">226</span><span id="line-226"> + unmatchedFamilies + "; valid family names of table " + tableDesc.getTableName()</span>
<span class="source-line-no">227</span><span id="line-227"> + " are: " + familyNames;</span>
<span class="source-line-no">228</span><span id="line-228"> LOG.error(msg);</span>
<span class="source-line-no">229</span><span id="line-229"> if (!silence) {</span>
<span class="source-line-no">230</span><span id="line-230"> throw new IOException(msg);</span>
<span class="source-line-no">231</span><span id="line-231"> }</span>
<span class="source-line-no">232</span><span id="line-232"> }</span>
<span class="source-line-no">233</span><span id="line-233"> }</span>
<span class="source-line-no">234</span><span id="line-234"></span>
<span class="source-line-no">235</span><span id="line-235"> /**</span>
<span class="source-line-no">236</span><span id="line-236"> * Populate the Queue with given HFiles</span>
<span class="source-line-no">237</span><span id="line-237"> */</span>
<span class="source-line-no">238</span><span id="line-238"> private static void populateLoadQueue(Deque&lt;LoadQueueItem&gt; ret, Map&lt;byte[], List&lt;Path&gt;&gt; map) {</span>
<span class="source-line-no">239</span><span id="line-239"> map.forEach((k, v) -&gt; v.stream().map(p -&gt; new LoadQueueItem(k, p)).forEachOrdered(ret::add));</span>
<span class="source-line-no">240</span><span id="line-240"> }</span>
<span class="source-line-no">241</span><span id="line-241"></span>
<span class="source-line-no">242</span><span id="line-242"> private interface BulkHFileVisitor&lt;TFamily&gt; {</span>
<span class="source-line-no">243</span><span id="line-243"></span>
<span class="source-line-no">244</span><span id="line-244"> TFamily bulkFamily(byte[] familyName) throws IOException;</span>
<span class="source-line-no">245</span><span id="line-245"></span>
<span class="source-line-no">246</span><span id="line-246"> void bulkHFile(TFamily family, FileStatus hfileStatus) throws IOException;</span>
<span class="source-line-no">247</span><span id="line-247"> }</span>
<span class="source-line-no">248</span><span id="line-248"></span>
<span class="source-line-no">249</span><span id="line-249"> /**</span>
<span class="source-line-no">250</span><span id="line-250"> * Iterate over the bulkDir hfiles. Skip reference, HFileLink, files starting with "_". Check and</span>
<span class="source-line-no">251</span><span id="line-251"> * skip non-valid hfiles by default, or skip this validation by setting {@link #VALIDATE_HFILES}</span>
<span class="source-line-no">252</span><span id="line-252"> * to false.</span>
<span class="source-line-no">253</span><span id="line-253"> */</span>
<span class="source-line-no">254</span><span id="line-254"> private static &lt;TFamily&gt; void visitBulkHFiles(FileSystem fs, Path bulkDir,</span>
<span class="source-line-no">255</span><span id="line-255"> BulkHFileVisitor&lt;TFamily&gt; visitor, boolean validateHFile) throws IOException {</span>
<span class="source-line-no">256</span><span id="line-256"> FileStatus[] familyDirStatuses = fs.listStatus(bulkDir);</span>
<span class="source-line-no">257</span><span id="line-257"> for (FileStatus familyStat : familyDirStatuses) {</span>
<span class="source-line-no">258</span><span id="line-258"> if (!familyStat.isDirectory()) {</span>
<span class="source-line-no">259</span><span id="line-259"> LOG.warn("Skipping non-directory " + familyStat.getPath());</span>
<span class="source-line-no">260</span><span id="line-260"> continue;</span>
<span class="source-line-no">261</span><span id="line-261"> }</span>
<span class="source-line-no">262</span><span id="line-262"> Path familyDir = familyStat.getPath();</span>
<span class="source-line-no">263</span><span id="line-263"> byte[] familyName = Bytes.toBytes(familyDir.getName());</span>
<span class="source-line-no">264</span><span id="line-264"> // Skip invalid family</span>
<span class="source-line-no">265</span><span id="line-265"> try {</span>
<span class="source-line-no">266</span><span id="line-266"> ColumnFamilyDescriptorBuilder.isLegalColumnFamilyName(familyName);</span>
<span class="source-line-no">267</span><span id="line-267"> } catch (IllegalArgumentException e) {</span>
<span class="source-line-no">268</span><span id="line-268"> LOG.warn("Skipping invalid " + familyStat.getPath());</span>
<span class="source-line-no">269</span><span id="line-269"> continue;</span>
<span class="source-line-no">270</span><span id="line-270"> }</span>
<span class="source-line-no">271</span><span id="line-271"> TFamily family = visitor.bulkFamily(familyName);</span>
<span class="source-line-no">272</span><span id="line-272"></span>
<span class="source-line-no">273</span><span id="line-273"> FileStatus[] hfileStatuses = fs.listStatus(familyDir);</span>
<span class="source-line-no">274</span><span id="line-274"> for (FileStatus hfileStatus : hfileStatuses) {</span>
<span class="source-line-no">275</span><span id="line-275"> if (!fs.isFile(hfileStatus.getPath())) {</span>
<span class="source-line-no">276</span><span id="line-276"> LOG.warn("Skipping non-file " + hfileStatus);</span>
<span class="source-line-no">277</span><span id="line-277"> continue;</span>
<span class="source-line-no">278</span><span id="line-278"> }</span>
<span class="source-line-no">279</span><span id="line-279"></span>
<span class="source-line-no">280</span><span id="line-280"> Path hfile = hfileStatus.getPath();</span>
<span class="source-line-no">281</span><span id="line-281"> // Skip "_", reference, HFileLink</span>
<span class="source-line-no">282</span><span id="line-282"> String fileName = hfile.getName();</span>
<span class="source-line-no">283</span><span id="line-283"> if (fileName.startsWith("_")) {</span>
<span class="source-line-no">284</span><span id="line-284"> continue;</span>
<span class="source-line-no">285</span><span id="line-285"> }</span>
<span class="source-line-no">286</span><span id="line-286"> if (StoreFileInfo.isReference(fileName)) {</span>
<span class="source-line-no">287</span><span id="line-287"> LOG.warn("Skipping reference " + fileName);</span>
<span class="source-line-no">288</span><span id="line-288"> continue;</span>
<span class="source-line-no">289</span><span id="line-289"> }</span>
<span class="source-line-no">290</span><span id="line-290"> if (HFileLink.isHFileLink(fileName)) {</span>
<span class="source-line-no">291</span><span id="line-291"> LOG.warn("Skipping HFileLink " + fileName);</span>
<span class="source-line-no">292</span><span id="line-292"> continue;</span>
<span class="source-line-no">293</span><span id="line-293"> }</span>
<span class="source-line-no">294</span><span id="line-294"></span>
<span class="source-line-no">295</span><span id="line-295"> // Validate HFile Format if needed</span>
<span class="source-line-no">296</span><span id="line-296"> if (validateHFile) {</span>
<span class="source-line-no">297</span><span id="line-297"> try {</span>
<span class="source-line-no">298</span><span id="line-298"> if (!HFile.isHFileFormat(fs, hfile)) {</span>
<span class="source-line-no">299</span><span id="line-299"> LOG.warn("the file " + hfile + " doesn't seems to be an hfile. skipping");</span>
<span class="source-line-no">300</span><span id="line-300"> continue;</span>
<span class="source-line-no">301</span><span id="line-301"> }</span>
<span class="source-line-no">302</span><span id="line-302"> } catch (FileNotFoundException e) {</span>
<span class="source-line-no">303</span><span id="line-303"> LOG.warn("the file " + hfile + " was removed");</span>
<span class="source-line-no">304</span><span id="line-304"> continue;</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"> visitor.bulkHFile(family, hfileStatus);</span>
<span class="source-line-no">309</span><span id="line-309"> }</span>
<span class="source-line-no">310</span><span id="line-310"> }</span>
<span class="source-line-no">311</span><span id="line-311"> }</span>
<span class="source-line-no">312</span><span id="line-312"></span>
<span class="source-line-no">313</span><span id="line-313"> /**</span>
<span class="source-line-no">314</span><span id="line-314"> * Walk the given directory for all HFiles, and return a Queue containing all such files.</span>
<span class="source-line-no">315</span><span id="line-315"> */</span>
<span class="source-line-no">316</span><span id="line-316"> private static void discoverLoadQueue(Configuration conf, Deque&lt;LoadQueueItem&gt; ret, Path hfofDir,</span>
<span class="source-line-no">317</span><span id="line-317"> boolean validateHFile) throws IOException {</span>
<span class="source-line-no">318</span><span id="line-318"> visitBulkHFiles(hfofDir.getFileSystem(conf), hfofDir, new BulkHFileVisitor&lt;byte[]&gt;() {</span>
<span class="source-line-no">319</span><span id="line-319"> @Override</span>
<span class="source-line-no">320</span><span id="line-320"> public byte[] bulkFamily(final byte[] familyName) {</span>
<span class="source-line-no">321</span><span id="line-321"> return familyName;</span>
<span class="source-line-no">322</span><span id="line-322"> }</span>
<span class="source-line-no">323</span><span id="line-323"></span>
<span class="source-line-no">324</span><span id="line-324"> @Override</span>
<span class="source-line-no">325</span><span id="line-325"> public void bulkHFile(final byte[] family, final FileStatus hfile) {</span>
<span class="source-line-no">326</span><span id="line-326"> long length = hfile.getLen();</span>
<span class="source-line-no">327</span><span id="line-327"> if (</span>
<span class="source-line-no">328</span><span id="line-328"> length &gt; conf.getLong(HConstants.HREGION_MAX_FILESIZE, HConstants.DEFAULT_MAX_FILE_SIZE)</span>
<span class="source-line-no">329</span><span id="line-329"> ) {</span>
<span class="source-line-no">330</span><span id="line-330"> LOG.warn("Trying to bulk load hfile " + hfile.getPath() + " with size: " + length</span>
<span class="source-line-no">331</span><span id="line-331"> + " bytes can be problematic as it may lead to oversplitting.");</span>
<span class="source-line-no">332</span><span id="line-332"> }</span>
<span class="source-line-no">333</span><span id="line-333"> ret.add(new LoadQueueItem(family, hfile.getPath()));</span>
<span class="source-line-no">334</span><span id="line-334"> }</span>
<span class="source-line-no">335</span><span id="line-335"> }, validateHFile);</span>
<span class="source-line-no">336</span><span id="line-336"> }</span>
<span class="source-line-no">337</span><span id="line-337"></span>
<span class="source-line-no">338</span><span id="line-338"> /**</span>
<span class="source-line-no">339</span><span id="line-339"> * Prepare a collection of {@code LoadQueueItem} from list of source hfiles contained in the</span>
<span class="source-line-no">340</span><span id="line-340"> * passed directory and validates whether the prepared queue has all the valid table column</span>
<span class="source-line-no">341</span><span id="line-341"> * families in it.</span>
<span class="source-line-no">342</span><span id="line-342"> * @param map map of family to List of hfiles</span>
<span class="source-line-no">343</span><span id="line-343"> * @param tableName table to which hfiles should be loaded</span>
<span class="source-line-no">344</span><span id="line-344"> * @param queue queue which needs to be loaded into the table</span>
<span class="source-line-no">345</span><span id="line-345"> * @param silence true to ignore unmatched column families</span>
<span class="source-line-no">346</span><span id="line-346"> * @throws IOException If any I/O or network error occurred</span>
<span class="source-line-no">347</span><span id="line-347"> */</span>
<span class="source-line-no">348</span><span id="line-348"> public static void prepareHFileQueue(AsyncClusterConnection conn, TableName tableName,</span>
<span class="source-line-no">349</span><span id="line-349"> Map&lt;byte[], List&lt;Path&gt;&gt; map, Deque&lt;LoadQueueItem&gt; queue, boolean silence) throws IOException {</span>
<span class="source-line-no">350</span><span id="line-350"> populateLoadQueue(queue, map);</span>
<span class="source-line-no">351</span><span id="line-351"> validateFamiliesInHFiles(FutureUtils.get(conn.getAdmin().getDescriptor(tableName)), queue,</span>
<span class="source-line-no">352</span><span id="line-352"> silence);</span>
<span class="source-line-no">353</span><span id="line-353"> }</span>
<span class="source-line-no">354</span><span id="line-354"></span>
<span class="source-line-no">355</span><span id="line-355"> /**</span>
<span class="source-line-no">356</span><span id="line-356"> * Prepare a collection of {@code LoadQueueItem} from list of source hfiles contained in the</span>
<span class="source-line-no">357</span><span id="line-357"> * passed directory and validates whether the prepared queue has all the valid table column</span>
<span class="source-line-no">358</span><span id="line-358"> * families in it.</span>
<span class="source-line-no">359</span><span id="line-359"> * @param hfilesDir directory containing list of hfiles to be loaded into the table</span>
<span class="source-line-no">360</span><span id="line-360"> * @param queue queue which needs to be loaded into the table</span>
<span class="source-line-no">361</span><span id="line-361"> * @param validateHFile if true hfiles will be validated for its format</span>
<span class="source-line-no">362</span><span id="line-362"> * @param silence true to ignore unmatched column families</span>
<span class="source-line-no">363</span><span id="line-363"> * @throws IOException If any I/O or network error occurred</span>
<span class="source-line-no">364</span><span id="line-364"> */</span>
<span class="source-line-no">365</span><span id="line-365"> public static void prepareHFileQueue(Configuration conf, AsyncClusterConnection conn,</span>
<span class="source-line-no">366</span><span id="line-366"> TableName tableName, Path hfilesDir, Deque&lt;LoadQueueItem&gt; queue, boolean validateHFile,</span>
<span class="source-line-no">367</span><span id="line-367"> boolean silence) throws IOException {</span>
<span class="source-line-no">368</span><span id="line-368"> discoverLoadQueue(conf, queue, hfilesDir, validateHFile);</span>
<span class="source-line-no">369</span><span id="line-369"> validateFamiliesInHFiles(FutureUtils.get(conn.getAdmin().getDescriptor(tableName)), queue,</span>
<span class="source-line-no">370</span><span id="line-370"> silence);</span>
<span class="source-line-no">371</span><span id="line-371"> }</span>
<span class="source-line-no">372</span><span id="line-372"></span>
<span class="source-line-no">373</span><span id="line-373"> /**</span>
<span class="source-line-no">374</span><span id="line-374"> * Used by the replication sink to load the hfiles from the source cluster. It does the following,</span>
<span class="source-line-no">375</span><span id="line-375"> * &lt;ol&gt;</span>
<span class="source-line-no">376</span><span id="line-376"> * &lt;li&gt;{@link #groupOrSplitPhase(AsyncClusterConnection, TableName, ExecutorService, Deque, List)}</span>
<span class="source-line-no">377</span><span id="line-377"> * &lt;/li&gt;</span>
<span class="source-line-no">378</span><span id="line-378"> * &lt;li&gt;{@link #bulkLoadPhase(AsyncClusterConnection, TableName, Deque, Multimap, boolean, Map)}</span>
<span class="source-line-no">379</span><span id="line-379"> * &lt;/li&gt;</span>
<span class="source-line-no">380</span><span id="line-380"> * &lt;/ol&gt;</span>
<span class="source-line-no">381</span><span id="line-381"> * @param conn Connection to use</span>
<span class="source-line-no">382</span><span id="line-382"> * @param tableName Table to which these hfiles should be loaded to</span>
<span class="source-line-no">383</span><span id="line-383"> * @param queue {@code LoadQueueItem} has hfiles yet to be loaded</span>
<span class="source-line-no">384</span><span id="line-384"> */</span>
<span class="source-line-no">385</span><span id="line-385"> public void loadHFileQueue(AsyncClusterConnection conn, TableName tableName,</span>
<span class="source-line-no">386</span><span id="line-386"> Deque&lt;LoadQueueItem&gt; queue, boolean copyFiles) throws IOException {</span>
<span class="source-line-no">387</span><span id="line-387"> ExecutorService pool = createExecutorService();</span>
<span class="source-line-no">388</span><span id="line-388"> try {</span>
<span class="source-line-no">389</span><span id="line-389"> Multimap&lt;ByteBuffer, LoadQueueItem&gt; regionGroups = groupOrSplitPhase(conn, tableName, pool,</span>
<span class="source-line-no">390</span><span id="line-390"> queue, FutureUtils.get(conn.getRegionLocator(tableName).getStartEndKeys())).getFirst();</span>
<span class="source-line-no">391</span><span id="line-391"> bulkLoadPhase(conn, tableName, queue, regionGroups, copyFiles, null);</span>
<span class="source-line-no">392</span><span id="line-392"> } finally {</span>
<span class="source-line-no">393</span><span id="line-393"> pool.shutdown();</span>
<span class="source-line-no">394</span><span id="line-394"> }</span>
<span class="source-line-no">395</span><span id="line-395"> }</span>
<span class="source-line-no">396</span><span id="line-396"></span>
<span class="source-line-no">397</span><span id="line-397"> /**</span>
<span class="source-line-no">398</span><span id="line-398"> * Attempts to do an atomic load of many hfiles into a region. If it fails, it returns a list of</span>
<span class="source-line-no">399</span><span id="line-399"> * hfiles that need to be retried. If it is successful it will return an empty list. NOTE: To</span>
<span class="source-line-no">400</span><span id="line-400"> * maintain row atomicity guarantees, region server side should succeed atomically and fails</span>
<span class="source-line-no">401</span><span id="line-401"> * atomically.</span>
<span class="source-line-no">402</span><span id="line-402"> * @param conn Connection to use</span>
<span class="source-line-no">403</span><span id="line-403"> * @param tableName Table to which these hfiles should be loaded to</span>
<span class="source-line-no">404</span><span id="line-404"> * @param copyFiles whether replicate to peer cluster while bulkloading</span>
<span class="source-line-no">405</span><span id="line-405"> * @param first the start key of region</span>
<span class="source-line-no">406</span><span id="line-406"> * @param lqis hfiles should be loaded</span>
<span class="source-line-no">407</span><span id="line-407"> * @return empty list if success, list of items to retry on recoverable failure</span>
<span class="source-line-no">408</span><span id="line-408"> */</span>
<span class="source-line-no">409</span><span id="line-409"> @InterfaceAudience.Private</span>
<span class="source-line-no">410</span><span id="line-410"> protected CompletableFuture&lt;Collection&lt;LoadQueueItem&gt;&gt; tryAtomicRegionLoad(</span>
<span class="source-line-no">411</span><span id="line-411"> final AsyncClusterConnection conn, final TableName tableName, boolean copyFiles,</span>
<span class="source-line-no">412</span><span id="line-412"> final byte[] first, Collection&lt;LoadQueueItem&gt; lqis) {</span>
<span class="source-line-no">413</span><span id="line-413"> List&lt;Pair&lt;byte[], String&gt;&gt; familyPaths =</span>
<span class="source-line-no">414</span><span id="line-414"> lqis.stream().map(lqi -&gt; Pair.newPair(lqi.getFamily(), lqi.getFilePath().toString()))</span>
<span class="source-line-no">415</span><span id="line-415"> .collect(Collectors.toList());</span>
<span class="source-line-no">416</span><span id="line-416"> CompletableFuture&lt;Collection&lt;LoadQueueItem&gt;&gt; future = new CompletableFuture&lt;&gt;();</span>
<span class="source-line-no">417</span><span id="line-417"> FutureUtils.addListener(conn.bulkLoad(tableName, familyPaths, first, assignSeqIds,</span>
<span class="source-line-no">418</span><span id="line-418"> fsDelegationToken.getUserToken(), bulkToken, copyFiles, clusterIds, replicate),</span>
<span class="source-line-no">419</span><span id="line-419"> (loaded, error) -&gt; {</span>
<span class="source-line-no">420</span><span id="line-420"> if (error != null) {</span>
<span class="source-line-no">421</span><span id="line-421"> LOG.error("Encountered unrecoverable error from region server", error);</span>
<span class="source-line-no">422</span><span id="line-422"> if (</span>
<span class="source-line-no">423</span><span id="line-423"> getConf().getBoolean(RETRY_ON_IO_EXCEPTION, false)</span>
<span class="source-line-no">424</span><span id="line-424"> &amp;&amp; numRetries.get() &lt; getConf().getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,</span>
<span class="source-line-no">425</span><span id="line-425"> HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER)</span>
<span class="source-line-no">426</span><span id="line-426"> ) {</span>
<span class="source-line-no">427</span><span id="line-427"> LOG.warn("Will attempt to retry loading failed HFiles. Retry #"</span>
<span class="source-line-no">428</span><span id="line-428"> + numRetries.incrementAndGet());</span>
<span class="source-line-no">429</span><span id="line-429"> // return lqi's to retry</span>
<span class="source-line-no">430</span><span id="line-430"> future.complete(lqis);</span>
<span class="source-line-no">431</span><span id="line-431"> } else {</span>
<span class="source-line-no">432</span><span id="line-432"> LOG.error(RETRY_ON_IO_EXCEPTION</span>
<span class="source-line-no">433</span><span id="line-433"> + " is disabled or we have reached retry limit. Unable to recover");</span>
<span class="source-line-no">434</span><span id="line-434"> future.completeExceptionally(error);</span>
<span class="source-line-no">435</span><span id="line-435"> }</span>
<span class="source-line-no">436</span><span id="line-436"> } else {</span>
<span class="source-line-no">437</span><span id="line-437"> if (loaded) {</span>
<span class="source-line-no">438</span><span id="line-438"> future.complete(Collections.emptyList());</span>
<span class="source-line-no">439</span><span id="line-439"> } else {</span>
<span class="source-line-no">440</span><span id="line-440"> LOG.warn("Attempt to bulk load region containing " + Bytes.toStringBinary(first)</span>
<span class="source-line-no">441</span><span id="line-441"> + " into table " + tableName + " with files " + lqis</span>
<span class="source-line-no">442</span><span id="line-442"> + " failed. This is recoverable and they will be retried.");</span>
<span class="source-line-no">443</span><span id="line-443"> // return lqi's to retry</span>
<span class="source-line-no">444</span><span id="line-444"> future.complete(lqis);</span>
<span class="source-line-no">445</span><span id="line-445"> }</span>
<span class="source-line-no">446</span><span id="line-446"> }</span>
<span class="source-line-no">447</span><span id="line-447"> });</span>
<span class="source-line-no">448</span><span id="line-448"> return future;</span>
<span class="source-line-no">449</span><span id="line-449"> }</span>
<span class="source-line-no">450</span><span id="line-450"></span>
<span class="source-line-no">451</span><span id="line-451"> /**</span>
<span class="source-line-no">452</span><span id="line-452"> * This takes the LQI's grouped by likely regions and attempts to bulk load them. Any failures are</span>
<span class="source-line-no">453</span><span id="line-453"> * re-queued for another pass with the groupOrSplitPhase.</span>
<span class="source-line-no">454</span><span id="line-454"> * &lt;p/&gt;</span>
<span class="source-line-no">455</span><span id="line-455"> * protected for testing.</span>
<span class="source-line-no">456</span><span id="line-456"> */</span>
<span class="source-line-no">457</span><span id="line-457"> @InterfaceAudience.Private</span>
<span class="source-line-no">458</span><span id="line-458"> protected void bulkLoadPhase(AsyncClusterConnection conn, TableName tableName,</span>
<span class="source-line-no">459</span><span id="line-459"> Deque&lt;LoadQueueItem&gt; queue, Multimap&lt;ByteBuffer, LoadQueueItem&gt; regionGroups, boolean copyFiles,</span>
<span class="source-line-no">460</span><span id="line-460"> Map&lt;LoadQueueItem, ByteBuffer&gt; item2RegionMap) throws IOException {</span>
<span class="source-line-no">461</span><span id="line-461"> // atomically bulk load the groups.</span>
<span class="source-line-no">462</span><span id="line-462"> List&lt;Future&lt;Collection&lt;LoadQueueItem&gt;&gt;&gt; loadingFutures = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">463</span><span id="line-463"> for (Entry&lt;ByteBuffer, ? extends Collection&lt;LoadQueueItem&gt;&gt; entry : regionGroups.asMap()</span>
<span class="source-line-no">464</span><span id="line-464"> .entrySet()) {</span>
<span class="source-line-no">465</span><span id="line-465"> byte[] first = entry.getKey().array();</span>
<span class="source-line-no">466</span><span id="line-466"> final Collection&lt;LoadQueueItem&gt; lqis = entry.getValue();</span>
<span class="source-line-no">467</span><span id="line-467"> if (bulkLoadByFamily) {</span>
<span class="source-line-no">468</span><span id="line-468"> groupByFamilies(lqis).values().forEach(familyQueue -&gt; loadingFutures</span>
<span class="source-line-no">469</span><span id="line-469"> .add(tryAtomicRegionLoad(conn, tableName, copyFiles, first, familyQueue)));</span>
<span class="source-line-no">470</span><span id="line-470"> } else {</span>
<span class="source-line-no">471</span><span id="line-471"> loadingFutures.add(tryAtomicRegionLoad(conn, tableName, copyFiles, first, lqis));</span>
<span class="source-line-no">472</span><span id="line-472"> }</span>
<span class="source-line-no">473</span><span id="line-473"> if (item2RegionMap != null) {</span>
<span class="source-line-no">474</span><span id="line-474"> for (LoadQueueItem lqi : lqis) {</span>
<span class="source-line-no">475</span><span id="line-475"> item2RegionMap.put(lqi, entry.getKey());</span>
<span class="source-line-no">476</span><span id="line-476"> }</span>
<span class="source-line-no">477</span><span id="line-477"> }</span>
<span class="source-line-no">478</span><span id="line-478"> }</span>
<span class="source-line-no">479</span><span id="line-479"></span>
<span class="source-line-no">480</span><span id="line-480"> // get all the results.</span>
<span class="source-line-no">481</span><span id="line-481"> for (Future&lt;Collection&lt;LoadQueueItem&gt;&gt; future : loadingFutures) {</span>
<span class="source-line-no">482</span><span id="line-482"> try {</span>
<span class="source-line-no">483</span><span id="line-483"> Collection&lt;LoadQueueItem&gt; toRetry = future.get();</span>
<span class="source-line-no">484</span><span id="line-484"></span>
<span class="source-line-no">485</span><span id="line-485"> if (item2RegionMap != null) {</span>
<span class="source-line-no">486</span><span id="line-486"> for (LoadQueueItem lqi : toRetry) {</span>
<span class="source-line-no">487</span><span id="line-487"> item2RegionMap.remove(lqi);</span>
<span class="source-line-no">488</span><span id="line-488"> }</span>
<span class="source-line-no">489</span><span id="line-489"> }</span>
<span class="source-line-no">490</span><span id="line-490"> // LQIs that are requeued to be regrouped.</span>
<span class="source-line-no">491</span><span id="line-491"> queue.addAll(toRetry);</span>
<span class="source-line-no">492</span><span id="line-492"> } catch (ExecutionException e1) {</span>
<span class="source-line-no">493</span><span id="line-493"> Throwable t = e1.getCause();</span>
<span class="source-line-no">494</span><span id="line-494"> if (t instanceof IOException) {</span>
<span class="source-line-no">495</span><span id="line-495"> // At this point something unrecoverable has happened.</span>
<span class="source-line-no">496</span><span id="line-496"> // TODO Implement bulk load recovery</span>
<span class="source-line-no">497</span><span id="line-497"> throw new IOException("BulkLoad encountered an unrecoverable problem", t);</span>
<span class="source-line-no">498</span><span id="line-498"> }</span>
<span class="source-line-no">499</span><span id="line-499"> LOG.error("Unexpected execution exception during bulk load", e1);</span>
<span class="source-line-no">500</span><span id="line-500"> throw new IllegalStateException(t);</span>
<span class="source-line-no">501</span><span id="line-501"> } catch (InterruptedException e1) {</span>
<span class="source-line-no">502</span><span id="line-502"> LOG.error("Unexpected interrupted exception during bulk load", e1);</span>
<span class="source-line-no">503</span><span id="line-503"> throw (InterruptedIOException) new InterruptedIOException().initCause(e1);</span>
<span class="source-line-no">504</span><span id="line-504"> }</span>
<span class="source-line-no">505</span><span id="line-505"> }</span>
<span class="source-line-no">506</span><span id="line-506"> }</span>
<span class="source-line-no">507</span><span id="line-507"></span>
<span class="source-line-no">508</span><span id="line-508"> private Map&lt;byte[], Collection&lt;LoadQueueItem&gt;&gt;</span>
<span class="source-line-no">509</span><span id="line-509"> groupByFamilies(Collection&lt;LoadQueueItem&gt; itemsInRegion) {</span>
<span class="source-line-no">510</span><span id="line-510"> Map&lt;byte[], Collection&lt;LoadQueueItem&gt;&gt; families2Queue = new TreeMap&lt;&gt;(Bytes.BYTES_COMPARATOR);</span>
<span class="source-line-no">511</span><span id="line-511"> itemsInRegion.forEach(item -&gt; families2Queue</span>
<span class="source-line-no">512</span><span id="line-512"> .computeIfAbsent(item.getFamily(), queue -&gt; new ArrayList&lt;&gt;()).add(item));</span>
<span class="source-line-no">513</span><span id="line-513"> return families2Queue;</span>
<span class="source-line-no">514</span><span id="line-514"> }</span>
<span class="source-line-no">515</span><span id="line-515"></span>
<span class="source-line-no">516</span><span id="line-516"> private boolean</span>
<span class="source-line-no">517</span><span id="line-517"> checkHFilesCountPerRegionPerFamily(final Multimap&lt;ByteBuffer, LoadQueueItem&gt; regionGroups) {</span>
<span class="source-line-no">518</span><span id="line-518"> for (Map.Entry&lt;ByteBuffer, Collection&lt;LoadQueueItem&gt;&gt; e : regionGroups.asMap().entrySet()) {</span>
<span class="source-line-no">519</span><span id="line-519"> Map&lt;byte[], MutableInt&gt; filesMap = new TreeMap&lt;&gt;(Bytes.BYTES_COMPARATOR);</span>
<span class="source-line-no">520</span><span id="line-520"> for (LoadQueueItem lqi : e.getValue()) {</span>
<span class="source-line-no">521</span><span id="line-521"> MutableInt count = filesMap.computeIfAbsent(lqi.getFamily(), k -&gt; new MutableInt());</span>
<span class="source-line-no">522</span><span id="line-522"> count.increment();</span>
<span class="source-line-no">523</span><span id="line-523"> if (count.intValue() &gt; maxFilesPerRegionPerFamily) {</span>
<span class="source-line-no">524</span><span id="line-524"> LOG.error("Trying to load more than " + maxFilesPerRegionPerFamily + " hfiles to family "</span>
<span class="source-line-no">525</span><span id="line-525"> + Bytes.toStringBinary(lqi.getFamily()) + " of region with start key "</span>
<span class="source-line-no">526</span><span id="line-526"> + Bytes.toStringBinary(e.getKey()));</span>
<span class="source-line-no">527</span><span id="line-527"> return false;</span>
<span class="source-line-no">528</span><span id="line-528"> }</span>
<span class="source-line-no">529</span><span id="line-529"> }</span>
<span class="source-line-no">530</span><span id="line-530"> }</span>
<span class="source-line-no">531</span><span id="line-531"> return 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"> * @param conn the HBase cluster connection</span>
<span class="source-line-no">536</span><span id="line-536"> * @param tableName the table name of the table to load into</span>
<span class="source-line-no">537</span><span id="line-537"> * @param pool the ExecutorService</span>
<span class="source-line-no">538</span><span id="line-538"> * @param queue the queue for LoadQueueItem</span>
<span class="source-line-no">539</span><span id="line-539"> * @param startEndKeys start and end keys</span>
<span class="source-line-no">540</span><span id="line-540"> * @return A map that groups LQI by likely bulk load region targets and Set of missing hfiles.</span>
<span class="source-line-no">541</span><span id="line-541"> */</span>
<span class="source-line-no">542</span><span id="line-542"> private Pair&lt;Multimap&lt;ByteBuffer, LoadQueueItem&gt;, Set&lt;String&gt;&gt; groupOrSplitPhase(</span>
<span class="source-line-no">543</span><span id="line-543"> AsyncClusterConnection conn, TableName tableName, ExecutorService pool,</span>
<span class="source-line-no">544</span><span id="line-544"> Deque&lt;LoadQueueItem&gt; queue, List&lt;Pair&lt;byte[], byte[]&gt;&gt; startEndKeys) throws IOException {</span>
<span class="source-line-no">545</span><span id="line-545"> // &lt;region start key, LQI&gt; need synchronized only within this scope of this</span>
<span class="source-line-no">546</span><span id="line-546"> // phase because of the puts that happen in futures.</span>
<span class="source-line-no">547</span><span id="line-547"> Multimap&lt;ByteBuffer, LoadQueueItem&gt; rgs = HashMultimap.create();</span>
<span class="source-line-no">548</span><span id="line-548"> final Multimap&lt;ByteBuffer, LoadQueueItem&gt; regionGroups = Multimaps.synchronizedMultimap(rgs);</span>
<span class="source-line-no">549</span><span id="line-549"> Set&lt;String&gt; missingHFiles = new HashSet&lt;&gt;();</span>
<span class="source-line-no">550</span><span id="line-550"> Pair&lt;Multimap&lt;ByteBuffer, LoadQueueItem&gt;, Set&lt;String&gt;&gt; pair =</span>
<span class="source-line-no">551</span><span id="line-551"> new Pair&lt;&gt;(regionGroups, missingHFiles);</span>
<span class="source-line-no">552</span><span id="line-552"></span>
<span class="source-line-no">553</span><span id="line-553"> // drain LQIs and figure out bulk load groups</span>
<span class="source-line-no">554</span><span id="line-554"> Set&lt;Future&lt;Pair&lt;List&lt;LoadQueueItem&gt;, String&gt;&gt;&gt; splittingFutures = new HashSet&lt;&gt;();</span>
<span class="source-line-no">555</span><span id="line-555"> while (!queue.isEmpty()) {</span>
<span class="source-line-no">556</span><span id="line-556"> final LoadQueueItem item = queue.remove();</span>
<span class="source-line-no">557</span><span id="line-557"> final Callable&lt;Pair&lt;List&lt;LoadQueueItem&gt;, String&gt;&gt; call =</span>
<span class="source-line-no">558</span><span id="line-558"> () -&gt; groupOrSplit(conn, tableName, regionGroups, item, startEndKeys);</span>
<span class="source-line-no">559</span><span id="line-559"> splittingFutures.add(pool.submit(call));</span>
<span class="source-line-no">560</span><span id="line-560"> }</span>
<span class="source-line-no">561</span><span id="line-561"> // get all the results. All grouping and splitting must finish before</span>
<span class="source-line-no">562</span><span id="line-562"> // we can attempt the atomic loads.</span>
<span class="source-line-no">563</span><span id="line-563"> for (Future&lt;Pair&lt;List&lt;LoadQueueItem&gt;, String&gt;&gt; lqis : splittingFutures) {</span>
<span class="source-line-no">564</span><span id="line-564"> try {</span>
<span class="source-line-no">565</span><span id="line-565"> Pair&lt;List&lt;LoadQueueItem&gt;, String&gt; splits = lqis.get();</span>
<span class="source-line-no">566</span><span id="line-566"> if (splits != null) {</span>
<span class="source-line-no">567</span><span id="line-567"> if (splits.getFirst() != null) {</span>
<span class="source-line-no">568</span><span id="line-568"> queue.addAll(splits.getFirst());</span>
<span class="source-line-no">569</span><span id="line-569"> } else {</span>
<span class="source-line-no">570</span><span id="line-570"> missingHFiles.add(splits.getSecond());</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"> } catch (ExecutionException e1) {</span>
<span class="source-line-no">574</span><span id="line-574"> Throwable t = e1.getCause();</span>
<span class="source-line-no">575</span><span id="line-575"> if (t instanceof IOException) {</span>
<span class="source-line-no">576</span><span id="line-576"> LOG.error("IOException during splitting", e1);</span>
<span class="source-line-no">577</span><span id="line-577"> throw (IOException) t; // would have been thrown if not parallelized,</span>
<span class="source-line-no">578</span><span id="line-578"> }</span>
<span class="source-line-no">579</span><span id="line-579"> LOG.error("Unexpected execution exception during splitting", e1);</span>
<span class="source-line-no">580</span><span id="line-580"> throw new IllegalStateException(t);</span>
<span class="source-line-no">581</span><span id="line-581"> } catch (InterruptedException e1) {</span>
<span class="source-line-no">582</span><span id="line-582"> LOG.error("Unexpected interrupted exception during splitting", e1);</span>
<span class="source-line-no">583</span><span id="line-583"> throw (InterruptedIOException) new InterruptedIOException().initCause(e1);</span>
<span class="source-line-no">584</span><span id="line-584"> }</span>
<span class="source-line-no">585</span><span id="line-585"> }</span>
<span class="source-line-no">586</span><span id="line-586"> return pair;</span>
<span class="source-line-no">587</span><span id="line-587"> }</span>
<span class="source-line-no">588</span><span id="line-588"></span>
<span class="source-line-no">589</span><span id="line-589"> // unique file name for the table</span>
<span class="source-line-no">590</span><span id="line-590"> private String getUniqueName() {</span>
<span class="source-line-no">591</span><span id="line-591"> return UUID.randomUUID().toString().replaceAll("-", "");</span>
<span class="source-line-no">592</span><span id="line-592"> }</span>
<span class="source-line-no">593</span><span id="line-593"></span>
<span class="source-line-no">594</span><span id="line-594"> private List&lt;LoadQueueItem&gt; splitStoreFile(AsyncTableRegionLocator loc, LoadQueueItem item,</span>
<span class="source-line-no">595</span><span id="line-595"> TableDescriptor tableDesc, byte[] splitKey) throws IOException {</span>
<span class="source-line-no">596</span><span id="line-596"> Path hfilePath = item.getFilePath();</span>
<span class="source-line-no">597</span><span id="line-597"> byte[] family = item.getFamily();</span>
<span class="source-line-no">598</span><span id="line-598"> Path tmpDir = hfilePath.getParent();</span>
<span class="source-line-no">599</span><span id="line-599"> if (!tmpDir.getName().equals(TMP_DIR)) {</span>
<span class="source-line-no">600</span><span id="line-600"> tmpDir = new Path(tmpDir, TMP_DIR);</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"> LOG.info("HFile at " + hfilePath + " no longer fits inside a single " + "region. Splitting...");</span>
<span class="source-line-no">604</span><span id="line-604"></span>
<span class="source-line-no">605</span><span id="line-605"> String uniqueName = getUniqueName();</span>
<span class="source-line-no">606</span><span id="line-606"> ColumnFamilyDescriptor familyDesc = tableDesc.getColumnFamily(family);</span>
<span class="source-line-no">607</span><span id="line-607"></span>
<span class="source-line-no">608</span><span id="line-608"> Path botOut = new Path(tmpDir, uniqueName + ".bottom");</span>
<span class="source-line-no">609</span><span id="line-609"> Path topOut = new Path(tmpDir, uniqueName + ".top");</span>
<span class="source-line-no">610</span><span id="line-610"></span>
<span class="source-line-no">611</span><span id="line-611"> splitStoreFile(loc, getConf(), hfilePath, familyDesc, splitKey, botOut, topOut);</span>
<span class="source-line-no">612</span><span id="line-612"></span>
<span class="source-line-no">613</span><span id="line-613"> FileSystem fs = tmpDir.getFileSystem(getConf());</span>
<span class="source-line-no">614</span><span id="line-614"> fs.setPermission(tmpDir, FsPermission.valueOf("-rwxrwxrwx"));</span>
<span class="source-line-no">615</span><span id="line-615"> fs.setPermission(botOut, FsPermission.valueOf("-rwxrwxrwx"));</span>
<span class="source-line-no">616</span><span id="line-616"> fs.setPermission(topOut, FsPermission.valueOf("-rwxrwxrwx"));</span>
<span class="source-line-no">617</span><span id="line-617"></span>
<span class="source-line-no">618</span><span id="line-618"> // Add these back at the *front* of the queue, so there's a lower</span>
<span class="source-line-no">619</span><span id="line-619"> // chance that the region will just split again before we get there.</span>
<span class="source-line-no">620</span><span id="line-620"> List&lt;LoadQueueItem&gt; lqis = new ArrayList&lt;&gt;(2);</span>
<span class="source-line-no">621</span><span id="line-621"> lqis.add(new LoadQueueItem(family, botOut));</span>
<span class="source-line-no">622</span><span id="line-622"> lqis.add(new LoadQueueItem(family, topOut));</span>
<span class="source-line-no">623</span><span id="line-623"></span>
<span class="source-line-no">624</span><span id="line-624"> // If the current item is already the result of previous splits,</span>
<span class="source-line-no">625</span><span id="line-625"> // we don't need it anymore. Clean up to save space.</span>
<span class="source-line-no">626</span><span id="line-626"> // It is not part of the original input files.</span>
<span class="source-line-no">627</span><span id="line-627"> try {</span>
<span class="source-line-no">628</span><span id="line-628"> if (tmpDir.getName().equals(TMP_DIR)) {</span>
<span class="source-line-no">629</span><span id="line-629"> fs.delete(hfilePath, false);</span>
<span class="source-line-no">630</span><span id="line-630"> }</span>
<span class="source-line-no">631</span><span id="line-631"> } catch (IOException e) {</span>
<span class="source-line-no">632</span><span id="line-632"> LOG.warn("Unable to delete temporary split file " + hfilePath);</span>
<span class="source-line-no">633</span><span id="line-633"> }</span>
<span class="source-line-no">634</span><span id="line-634"> LOG.info("Successfully split into new HFiles " + botOut + " and " + topOut);</span>
<span class="source-line-no">635</span><span id="line-635"> return lqis;</span>
<span class="source-line-no">636</span><span id="line-636"> }</span>
<span class="source-line-no">637</span><span id="line-637"></span>
<span class="source-line-no">638</span><span id="line-638"> /**</span>
<span class="source-line-no">639</span><span id="line-639"> * @param startEndKeys the start/end keys of regions belong to this table, the list in ascending</span>
<span class="source-line-no">640</span><span id="line-640"> * order by start key</span>
<span class="source-line-no">641</span><span id="line-641"> * @param key the key need to find which region belong to</span>
<span class="source-line-no">642</span><span id="line-642"> * @return region index</span>
<span class="source-line-no">643</span><span id="line-643"> */</span>
<span class="source-line-no">644</span><span id="line-644"> private int getRegionIndex(List&lt;Pair&lt;byte[], byte[]&gt;&gt; startEndKeys, byte[] key) {</span>
<span class="source-line-no">645</span><span id="line-645"> int idx = Collections.binarySearch(startEndKeys, Pair.newPair(key, HConstants.EMPTY_END_ROW),</span>
<span class="source-line-no">646</span><span id="line-646"> (p1, p2) -&gt; Bytes.compareTo(p1.getFirst(), p2.getFirst()));</span>
<span class="source-line-no">647</span><span id="line-647"> if (idx &lt; 0) {</span>
<span class="source-line-no">648</span><span id="line-648"> // not on boundary, returns -(insertion index). Calculate region it</span>
<span class="source-line-no">649</span><span id="line-649"> // would be in.</span>
<span class="source-line-no">650</span><span id="line-650"> idx = -(idx + 1) - 1;</span>
<span class="source-line-no">651</span><span id="line-651"> }</span>
<span class="source-line-no">652</span><span id="line-652"> return idx;</span>
<span class="source-line-no">653</span><span id="line-653"> }</span>
<span class="source-line-no">654</span><span id="line-654"></span>
<span class="source-line-no">655</span><span id="line-655"> /**</span>
<span class="source-line-no">656</span><span id="line-656"> * we can consider there is a region hole or overlap in following conditions. 1) if idx &lt; 0,then</span>
<span class="source-line-no">657</span><span id="line-657"> * first region info is lost. 2) if the endkey of a region is not equal to the startkey of the</span>
<span class="source-line-no">658</span><span id="line-658"> * next region. 3) if the endkey of the last region is not empty.</span>
<span class="source-line-no">659</span><span id="line-659"> */</span>
<span class="source-line-no">660</span><span id="line-660"> private void checkRegionIndexValid(int idx, List&lt;Pair&lt;byte[], byte[]&gt;&gt; startEndKeys,</span>
<span class="source-line-no">661</span><span id="line-661"> TableName tableName) throws IOException {</span>
<span class="source-line-no">662</span><span id="line-662"> if (idx &lt; 0) {</span>
<span class="source-line-no">663</span><span id="line-663"> throw new IOException("The first region info for table " + tableName</span>
<span class="source-line-no">664</span><span id="line-664"> + " can't be found in hbase:meta.Please use hbck tool to fix it first.");</span>
<span class="source-line-no">665</span><span id="line-665"> } else if (</span>
<span class="source-line-no">666</span><span id="line-666"> (idx == startEndKeys.size() - 1)</span>
<span class="source-line-no">667</span><span id="line-667"> &amp;&amp; !Bytes.equals(startEndKeys.get(idx).getSecond(), HConstants.EMPTY_BYTE_ARRAY)</span>
<span class="source-line-no">668</span><span id="line-668"> ) {</span>
<span class="source-line-no">669</span><span id="line-669"> throw new IOException("The last region info for table " + tableName</span>
<span class="source-line-no">670</span><span id="line-670"> + " can't be found in hbase:meta.Please use hbck tool to fix it first.");</span>
<span class="source-line-no">671</span><span id="line-671"> } else if (</span>
<span class="source-line-no">672</span><span id="line-672"> idx + 1 &lt; startEndKeys.size() &amp;&amp; !(Bytes.compareTo(startEndKeys.get(idx).getSecond(),</span>
<span class="source-line-no">673</span><span id="line-673"> startEndKeys.get(idx + 1).getFirst()) == 0)</span>
<span class="source-line-no">674</span><span id="line-674"> ) {</span>
<span class="source-line-no">675</span><span id="line-675"> throw new IOException("The endkey of one region for table " + tableName</span>
<span class="source-line-no">676</span><span id="line-676"> + " is not equal to the startkey of the next region in hbase:meta."</span>
<span class="source-line-no">677</span><span id="line-677"> + "Please use hbck tool to fix it first.");</span>
<span class="source-line-no">678</span><span id="line-678"> }</span>
<span class="source-line-no">679</span><span id="line-679"> }</span>
<span class="source-line-no">680</span><span id="line-680"></span>
<span class="source-line-no">681</span><span id="line-681"> /**</span>
<span class="source-line-no">682</span><span id="line-682"> * Attempt to assign the given load queue item into its target region group. If the hfile boundary</span>
<span class="source-line-no">683</span><span id="line-683"> * no longer fits into a region, physically splits the hfile such that the new bottom half will</span>
<span class="source-line-no">684</span><span id="line-684"> * fit and returns the list of LQI's corresponding to the resultant hfiles.</span>
<span class="source-line-no">685</span><span id="line-685"> * &lt;p/&gt;</span>
<span class="source-line-no">686</span><span id="line-686"> * protected for testing</span>
<span class="source-line-no">687</span><span id="line-687"> * @throws IOException if an IO failure is encountered</span>
<span class="source-line-no">688</span><span id="line-688"> */</span>
<span class="source-line-no">689</span><span id="line-689"> @InterfaceAudience.Private</span>
<span class="source-line-no">690</span><span id="line-690"> protected Pair&lt;List&lt;LoadQueueItem&gt;, String&gt; groupOrSplit(AsyncClusterConnection conn,</span>
<span class="source-line-no">691</span><span id="line-691"> TableName tableName, Multimap&lt;ByteBuffer, LoadQueueItem&gt; regionGroups, LoadQueueItem item,</span>
<span class="source-line-no">692</span><span id="line-692"> List&lt;Pair&lt;byte[], byte[]&gt;&gt; startEndKeys) throws IOException {</span>
<span class="source-line-no">693</span><span id="line-693"> Path hfilePath = item.getFilePath();</span>
<span class="source-line-no">694</span><span id="line-694"> Optional&lt;byte[]&gt; first, last;</span>
<span class="source-line-no">695</span><span id="line-695"> try (HFile.Reader hfr = HFile.createReader(hfilePath.getFileSystem(getConf()), hfilePath,</span>
<span class="source-line-no">696</span><span id="line-696"> CacheConfig.DISABLED, true, getConf())) {</span>
<span class="source-line-no">697</span><span id="line-697"> first = hfr.getFirstRowKey();</span>
<span class="source-line-no">698</span><span id="line-698"> last = hfr.getLastRowKey();</span>
<span class="source-line-no">699</span><span id="line-699"> } catch (FileNotFoundException fnfe) {</span>
<span class="source-line-no">700</span><span id="line-700"> LOG.debug("encountered", fnfe);</span>
<span class="source-line-no">701</span><span id="line-701"> return new Pair&lt;&gt;(null, hfilePath.getName());</span>
<span class="source-line-no">702</span><span id="line-702"> }</span>
<span class="source-line-no">703</span><span id="line-703"></span>
<span class="source-line-no">704</span><span id="line-704"> LOG.info("Trying to load hfile=" + hfilePath + " first=" + first.map(Bytes::toStringBinary)</span>
<span class="source-line-no">705</span><span id="line-705"> + " last=" + last.map(Bytes::toStringBinary));</span>
<span class="source-line-no">706</span><span id="line-706"> if (!first.isPresent() || !last.isPresent()) {</span>
<span class="source-line-no">707</span><span id="line-707"> assert !first.isPresent() &amp;&amp; !last.isPresent();</span>
<span class="source-line-no">708</span><span id="line-708"> // TODO what if this is due to a bad HFile?</span>
<span class="source-line-no">709</span><span id="line-709"> LOG.info("hfile " + hfilePath + " has no entries, skipping");</span>
<span class="source-line-no">710</span><span id="line-710"> return null;</span>
<span class="source-line-no">711</span><span id="line-711"> }</span>
<span class="source-line-no">712</span><span id="line-712"> if (Bytes.compareTo(first.get(), last.get()) &gt; 0) {</span>
<span class="source-line-no">713</span><span id="line-713"> throw new IllegalArgumentException("Invalid range: " + Bytes.toStringBinary(first.get())</span>
<span class="source-line-no">714</span><span id="line-714"> + " &gt; " + Bytes.toStringBinary(last.get()));</span>
<span class="source-line-no">715</span><span id="line-715"> }</span>
<span class="source-line-no">716</span><span id="line-716"> int firstKeyRegionIdx = getRegionIndex(startEndKeys, first.get());</span>
<span class="source-line-no">717</span><span id="line-717"> checkRegionIndexValid(firstKeyRegionIdx, startEndKeys, tableName);</span>
<span class="source-line-no">718</span><span id="line-718"> boolean lastKeyInRange =</span>
<span class="source-line-no">719</span><span id="line-719"> Bytes.compareTo(last.get(), startEndKeys.get(firstKeyRegionIdx).getSecond()) &lt; 0 || Bytes</span>
<span class="source-line-no">720</span><span id="line-720"> .equals(startEndKeys.get(firstKeyRegionIdx).getSecond(), HConstants.EMPTY_BYTE_ARRAY);</span>
<span class="source-line-no">721</span><span id="line-721"> if (!lastKeyInRange) {</span>
<span class="source-line-no">722</span><span id="line-722"> if (failIfNeedSplitHFile) {</span>
<span class="source-line-no">723</span><span id="line-723"> throw new IOException(</span>
<span class="source-line-no">724</span><span id="line-724"> "The key range of hfile=" + hfilePath + " fits into no region. " + "And because "</span>
<span class="source-line-no">725</span><span id="line-725"> + FAIL_IF_NEED_SPLIT_HFILE + " was set to true, we just skip the next steps.");</span>
<span class="source-line-no">726</span><span id="line-726"> }</span>
<span class="source-line-no">727</span><span id="line-727"> int lastKeyRegionIdx = getRegionIndex(startEndKeys, last.get());</span>
<span class="source-line-no">728</span><span id="line-728"> int splitIdx = (firstKeyRegionIdx + lastKeyRegionIdx) / 2;</span>
<span class="source-line-no">729</span><span id="line-729"> // make sure the splitPoint is valid in case region overlap occur, maybe the splitPoint bigger</span>
<span class="source-line-no">730</span><span id="line-730"> // than hfile.endkey w/o this check</span>
<span class="source-line-no">731</span><span id="line-731"> if (splitIdx != firstKeyRegionIdx) {</span>
<span class="source-line-no">732</span><span id="line-732"> checkRegionIndexValid(splitIdx, startEndKeys, tableName);</span>
<span class="source-line-no">733</span><span id="line-733"> }</span>
<span class="source-line-no">734</span><span id="line-734"> byte[] splitPoint = startEndKeys.get(splitIdx).getSecond();</span>
<span class="source-line-no">735</span><span id="line-735"> List&lt;LoadQueueItem&gt; lqis = splitStoreFile(conn.getRegionLocator(tableName), item,</span>
<span class="source-line-no">736</span><span id="line-736"> FutureUtils.get(conn.getAdmin().getDescriptor(tableName)), splitPoint);</span>
<span class="source-line-no">737</span><span id="line-737"></span>
<span class="source-line-no">738</span><span id="line-738"> return new Pair&lt;&gt;(lqis, null);</span>
<span class="source-line-no">739</span><span id="line-739"> }</span>
<span class="source-line-no">740</span><span id="line-740"></span>
<span class="source-line-no">741</span><span id="line-741"> // group regions.</span>
<span class="source-line-no">742</span><span id="line-742"> regionGroups.put(ByteBuffer.wrap(startEndKeys.get(firstKeyRegionIdx).getFirst()), item);</span>
<span class="source-line-no">743</span><span id="line-743"> return null;</span>
<span class="source-line-no">744</span><span id="line-744"> }</span>
<span class="source-line-no">745</span><span id="line-745"></span>
<span class="source-line-no">746</span><span id="line-746"> /**</span>
<span class="source-line-no">747</span><span id="line-747"> * Split a storefile into a top and bottom half with favored nodes, maintaining the metadata,</span>
<span class="source-line-no">748</span><span id="line-748"> * recreating bloom filters, etc.</span>
<span class="source-line-no">749</span><span id="line-749"> */</span>
<span class="source-line-no">750</span><span id="line-750"> @InterfaceAudience.Private</span>
<span class="source-line-no">751</span><span id="line-751"> static void splitStoreFile(AsyncTableRegionLocator loc, Configuration conf, Path inFile,</span>
<span class="source-line-no">752</span><span id="line-752"> ColumnFamilyDescriptor familyDesc, byte[] splitKey, Path bottomOut, Path topOut)</span>
<span class="source-line-no">753</span><span id="line-753"> throws IOException {</span>
<span class="source-line-no">754</span><span id="line-754"> // Open reader with no block cache, and not in-memory</span>
<span class="source-line-no">755</span><span id="line-755"> Reference topReference = Reference.createTopReference(splitKey);</span>
<span class="source-line-no">756</span><span id="line-756"> Reference bottomReference = Reference.createBottomReference(splitKey);</span>
<span class="source-line-no">757</span><span id="line-757"></span>
<span class="source-line-no">758</span><span id="line-758"> copyHFileHalf(conf, inFile, topOut, topReference, familyDesc, loc);</span>
<span class="source-line-no">759</span><span id="line-759"> copyHFileHalf(conf, inFile, bottomOut, bottomReference, familyDesc, loc);</span>
<span class="source-line-no">760</span><span id="line-760"> }</span>
<span class="source-line-no">761</span><span id="line-761"></span>
<span class="source-line-no">762</span><span id="line-762"> private static StoreFileWriter initStoreFileWriter(Configuration conf, Cell cell,</span>
<span class="source-line-no">763</span><span id="line-763"> HFileContext hFileContext, CacheConfig cacheConf, BloomType bloomFilterType, FileSystem fs,</span>
<span class="source-line-no">764</span><span id="line-764"> Path outFile, AsyncTableRegionLocator loc) throws IOException {</span>
<span class="source-line-no">765</span><span id="line-765"> if (conf.getBoolean(LOCALITY_SENSITIVE_CONF_KEY, DEFAULT_LOCALITY_SENSITIVE)) {</span>
<span class="source-line-no">766</span><span id="line-766"> byte[] rowKey = CellUtil.cloneRow(cell);</span>
<span class="source-line-no">767</span><span id="line-767"> HRegionLocation hRegionLocation = FutureUtils.get(loc.getRegionLocation(rowKey));</span>
<span class="source-line-no">768</span><span id="line-768"> InetSocketAddress[] favoredNodes = null;</span>
<span class="source-line-no">769</span><span id="line-769"> if (null == hRegionLocation) {</span>
<span class="source-line-no">770</span><span id="line-770"> LOG.warn("Failed get region location for rowkey {} , Using writer without favoured nodes.",</span>
<span class="source-line-no">771</span><span id="line-771"> Bytes.toString(rowKey));</span>
<span class="source-line-no">772</span><span id="line-772"> return new StoreFileWriter.Builder(conf, cacheConf, fs).withFilePath(outFile)</span>
<span class="source-line-no">773</span><span id="line-773"> .withBloomType(bloomFilterType).withFileContext(hFileContext).build();</span>
<span class="source-line-no">774</span><span id="line-774"> } else {</span>
<span class="source-line-no">775</span><span id="line-775"> LOG.debug("First rowkey: [{}]", Bytes.toString(rowKey));</span>
<span class="source-line-no">776</span><span id="line-776"> InetSocketAddress initialIsa =</span>
<span class="source-line-no">777</span><span id="line-777"> new InetSocketAddress(hRegionLocation.getHostname(), hRegionLocation.getPort());</span>
<span class="source-line-no">778</span><span id="line-778"> if (initialIsa.isUnresolved()) {</span>
<span class="source-line-no">779</span><span id="line-779"> LOG.warn("Failed get location for region {} , Using writer without favoured nodes.",</span>
<span class="source-line-no">780</span><span id="line-780"> hRegionLocation);</span>
<span class="source-line-no">781</span><span id="line-781"> return new StoreFileWriter.Builder(conf, cacheConf, fs).withFilePath(outFile)</span>
<span class="source-line-no">782</span><span id="line-782"> .withBloomType(bloomFilterType).withFileContext(hFileContext).build();</span>
<span class="source-line-no">783</span><span id="line-783"> } else {</span>
<span class="source-line-no">784</span><span id="line-784"> LOG.debug("Use favored nodes writer: {}", initialIsa.getHostString());</span>
<span class="source-line-no">785</span><span id="line-785"> favoredNodes = new InetSocketAddress[] { initialIsa };</span>
<span class="source-line-no">786</span><span id="line-786"> return new StoreFileWriter.Builder(conf, cacheConf, fs).withFilePath(outFile)</span>
<span class="source-line-no">787</span><span id="line-787"> .withBloomType(bloomFilterType).withFileContext(hFileContext)</span>
<span class="source-line-no">788</span><span id="line-788"> .withFavoredNodes(favoredNodes).build();</span>
<span class="source-line-no">789</span><span id="line-789"> }</span>
<span class="source-line-no">790</span><span id="line-790"> }</span>
<span class="source-line-no">791</span><span id="line-791"> } else {</span>
<span class="source-line-no">792</span><span id="line-792"> return new StoreFileWriter.Builder(conf, cacheConf, fs).withFilePath(outFile)</span>
<span class="source-line-no">793</span><span id="line-793"> .withBloomType(bloomFilterType).withFileContext(hFileContext).build();</span>
<span class="source-line-no">794</span><span id="line-794"> }</span>
<span class="source-line-no">795</span><span id="line-795"> }</span>
<span class="source-line-no">796</span><span id="line-796"></span>
<span class="source-line-no">797</span><span id="line-797"> /**</span>
<span class="source-line-no">798</span><span id="line-798"> * Copy half of an HFile into a new HFile with favored nodes.</span>
<span class="source-line-no">799</span><span id="line-799"> */</span>
<span class="source-line-no">800</span><span id="line-800"> private static void copyHFileHalf(Configuration conf, Path inFile, Path outFile,</span>
<span class="source-line-no">801</span><span id="line-801"> Reference reference, ColumnFamilyDescriptor familyDescriptor, AsyncTableRegionLocator loc)</span>
<span class="source-line-no">802</span><span id="line-802"> throws IOException {</span>
<span class="source-line-no">803</span><span id="line-803"> FileSystem fs = inFile.getFileSystem(conf);</span>
<span class="source-line-no">804</span><span id="line-804"> CacheConfig cacheConf = CacheConfig.DISABLED;</span>
<span class="source-line-no">805</span><span id="line-805"> StoreFileReader halfReader = null;</span>
<span class="source-line-no">806</span><span id="line-806"> StoreFileWriter halfWriter = null;</span>
<span class="source-line-no">807</span><span id="line-807"> try {</span>
<span class="source-line-no">808</span><span id="line-808"> ReaderContext context = new ReaderContextBuilder().withFileSystemAndPath(fs, inFile).build();</span>
<span class="source-line-no">809</span><span id="line-809"> StoreFileInfo storeFileInfo =</span>
<span class="source-line-no">810</span><span id="line-810"> new StoreFileInfo(conf, fs, fs.getFileStatus(inFile), reference);</span>
<span class="source-line-no">811</span><span id="line-811"> storeFileInfo.initHFileInfo(context);</span>
<span class="source-line-no">812</span><span id="line-812"> halfReader = storeFileInfo.createReader(context, cacheConf);</span>
<span class="source-line-no">813</span><span id="line-813"> storeFileInfo.getHFileInfo().initMetaAndIndex(halfReader.getHFileReader());</span>
<span class="source-line-no">814</span><span id="line-814"> Map&lt;byte[], byte[]&gt; fileInfo = halfReader.loadFileInfo();</span>
<span class="source-line-no">815</span><span id="line-815"></span>
<span class="source-line-no">816</span><span id="line-816"> int blocksize = familyDescriptor.getBlocksize();</span>
<span class="source-line-no">817</span><span id="line-817"> Algorithm compression = familyDescriptor.getCompressionType();</span>
<span class="source-line-no">818</span><span id="line-818"> BloomType bloomFilterType = familyDescriptor.getBloomFilterType();</span>
<span class="source-line-no">819</span><span id="line-819"> HFileContext hFileContext = new HFileContextBuilder().withCompression(compression)</span>
<span class="source-line-no">820</span><span id="line-820"> .withChecksumType(StoreUtils.getChecksumType(conf))</span>
<span class="source-line-no">821</span><span id="line-821"> .withBytesPerCheckSum(StoreUtils.getBytesPerChecksum(conf)).withBlockSize(blocksize)</span>
<span class="source-line-no">822</span><span id="line-822"> .withDataBlockEncoding(familyDescriptor.getDataBlockEncoding()).withIncludesTags(true)</span>
<span class="source-line-no">823</span><span id="line-823"> .withCreateTime(EnvironmentEdgeManager.currentTime()).build();</span>
<span class="source-line-no">824</span><span id="line-824"></span>
<span class="source-line-no">825</span><span id="line-825"> try (StoreFileScanner scanner =</span>
<span class="source-line-no">826</span><span id="line-826"> halfReader.getStoreFileScanner(false, false, false, Long.MAX_VALUE, 0, false)) {</span>
<span class="source-line-no">827</span><span id="line-827"> scanner.seek(KeyValue.LOWESTKEY);</span>
<span class="source-line-no">828</span><span id="line-828"> for (;;) {</span>
<span class="source-line-no">829</span><span id="line-829"> ExtendedCell cell = scanner.next();</span>
<span class="source-line-no">830</span><span id="line-830"> if (cell == null) {</span>
<span class="source-line-no">831</span><span id="line-831"> break;</span>
<span class="source-line-no">832</span><span id="line-832"> }</span>
<span class="source-line-no">833</span><span id="line-833"> if (halfWriter == null) {</span>
<span class="source-line-no">834</span><span id="line-834"> // init halfwriter</span>
<span class="source-line-no">835</span><span id="line-835"> halfWriter = initStoreFileWriter(conf, cell, hFileContext, cacheConf, bloomFilterType,</span>
<span class="source-line-no">836</span><span id="line-836"> fs, outFile, loc);</span>
<span class="source-line-no">837</span><span id="line-837"> }</span>
<span class="source-line-no">838</span><span id="line-838"> halfWriter.append(cell);</span>
<span class="source-line-no">839</span><span id="line-839"> }</span>
<span class="source-line-no">840</span><span id="line-840"> }</span>
<span class="source-line-no">841</span><span id="line-841"> for (Map.Entry&lt;byte[], byte[]&gt; entry : fileInfo.entrySet()) {</span>
<span class="source-line-no">842</span><span id="line-842"> if (shouldCopyHFileMetaKey(entry.getKey())) {</span>
<span class="source-line-no">843</span><span id="line-843"> halfWriter.appendFileInfo(entry.getKey(), entry.getValue());</span>
<span class="source-line-no">844</span><span id="line-844"> }</span>
<span class="source-line-no">845</span><span id="line-845"> }</span>
<span class="source-line-no">846</span><span id="line-846"> } finally {</span>
<span class="source-line-no">847</span><span id="line-847"> if (halfReader != null) {</span>
<span class="source-line-no">848</span><span id="line-848"> try {</span>
<span class="source-line-no">849</span><span id="line-849"> halfReader.close(cacheConf.shouldEvictOnClose());</span>
<span class="source-line-no">850</span><span id="line-850"> } catch (IOException e) {</span>
<span class="source-line-no">851</span><span id="line-851"> LOG.warn("failed to close hfile reader for " + inFile, e);</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"> if (halfWriter != null) {</span>
<span class="source-line-no">855</span><span id="line-855"> halfWriter.close();</span>
<span class="source-line-no">856</span><span id="line-856"> }</span>
<span class="source-line-no">857</span><span id="line-857"> }</span>
<span class="source-line-no">858</span><span id="line-858"> }</span>
<span class="source-line-no">859</span><span id="line-859"></span>
<span class="source-line-no">860</span><span id="line-860"> /**</span>
<span class="source-line-no">861</span><span id="line-861"> * Infers region boundaries for a new table.</span>
<span class="source-line-no">862</span><span id="line-862"> * &lt;p/&gt;</span>
<span class="source-line-no">863</span><span id="line-863"> * Parameter: &lt;br/&gt;</span>
<span class="source-line-no">864</span><span id="line-864"> * bdryMap is a map between keys to an integer belonging to {+1, -1}</span>
<span class="source-line-no">865</span><span id="line-865"> * &lt;ul&gt;</span>
<span class="source-line-no">866</span><span id="line-866"> * &lt;li&gt;If a key is a start key of a file, then it maps to +1&lt;/li&gt;</span>
<span class="source-line-no">867</span><span id="line-867"> * &lt;li&gt;If a key is an end key of a file, then it maps to -1&lt;/li&gt;</span>
<span class="source-line-no">868</span><span id="line-868"> * &lt;/ul&gt;</span>
<span class="source-line-no">869</span><span id="line-869"> * &lt;p&gt;</span>
<span class="source-line-no">870</span><span id="line-870"> * Algo:&lt;br/&gt;</span>
<span class="source-line-no">871</span><span id="line-871"> * &lt;ol&gt;</span>
<span class="source-line-no">872</span><span id="line-872"> * &lt;li&gt;Poll on the keys in order:</span>
<span class="source-line-no">873</span><span id="line-873"> * &lt;ol type="a"&gt;</span>
<span class="source-line-no">874</span><span id="line-874"> * &lt;li&gt;Keep adding the mapped values to these keys (runningSum)&lt;/li&gt;</span>
<span class="source-line-no">875</span><span id="line-875"> * &lt;li&gt;Each time runningSum reaches 0, add the start Key from when the runningSum had started to a</span>
<span class="source-line-no">876</span><span id="line-876"> * boundary list.&lt;/li&gt;</span>
<span class="source-line-no">877</span><span id="line-877"> * &lt;/ol&gt;</span>
<span class="source-line-no">878</span><span id="line-878"> * &lt;/li&gt;</span>
<span class="source-line-no">879</span><span id="line-879"> * &lt;li&gt;Return the boundary list.&lt;/li&gt;</span>
<span class="source-line-no">880</span><span id="line-880"> * &lt;/ol&gt;</span>
<span class="source-line-no">881</span><span id="line-881"> */</span>
<span class="source-line-no">882</span><span id="line-882"> public static byte[][] inferBoundaries(SortedMap&lt;byte[], Integer&gt; bdryMap) {</span>
<span class="source-line-no">883</span><span id="line-883"> List&lt;byte[]&gt; keysArray = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">884</span><span id="line-884"> int runningValue = 0;</span>
<span class="source-line-no">885</span><span id="line-885"> byte[] currStartKey = null;</span>
<span class="source-line-no">886</span><span id="line-886"> boolean firstBoundary = true;</span>
<span class="source-line-no">887</span><span id="line-887"></span>
<span class="source-line-no">888</span><span id="line-888"> for (Map.Entry&lt;byte[], Integer&gt; item : bdryMap.entrySet()) {</span>
<span class="source-line-no">889</span><span id="line-889"> if (runningValue == 0) {</span>
<span class="source-line-no">890</span><span id="line-890"> currStartKey = item.getKey();</span>
<span class="source-line-no">891</span><span id="line-891"> }</span>
<span class="source-line-no">892</span><span id="line-892"> runningValue += item.getValue();</span>
<span class="source-line-no">893</span><span id="line-893"> if (runningValue == 0) {</span>
<span class="source-line-no">894</span><span id="line-894"> if (!firstBoundary) {</span>
<span class="source-line-no">895</span><span id="line-895"> keysArray.add(currStartKey);</span>
<span class="source-line-no">896</span><span id="line-896"> }</span>
<span class="source-line-no">897</span><span id="line-897"> firstBoundary = false;</span>
<span class="source-line-no">898</span><span id="line-898"> }</span>
<span class="source-line-no">899</span><span id="line-899"> }</span>
<span class="source-line-no">900</span><span id="line-900"></span>
<span class="source-line-no">901</span><span id="line-901"> return keysArray.toArray(new byte[0][]);</span>
<span class="source-line-no">902</span><span id="line-902"> }</span>
<span class="source-line-no">903</span><span id="line-903"></span>
<span class="source-line-no">904</span><span id="line-904"> /**</span>
<span class="source-line-no">905</span><span id="line-905"> * If the table is created for the first time, then "completebulkload" reads the files twice. More</span>
<span class="source-line-no">906</span><span id="line-906"> * modifications necessary if we want to avoid doing it.</span>
<span class="source-line-no">907</span><span id="line-907"> */</span>
<span class="source-line-no">908</span><span id="line-908"> private void createTable(TableName tableName, Path hfofDir, AsyncAdmin admin) throws IOException {</span>
<span class="source-line-no">909</span><span id="line-909"> final FileSystem fs = hfofDir.getFileSystem(getConf());</span>
<span class="source-line-no">910</span><span id="line-910"></span>
<span class="source-line-no">911</span><span id="line-911"> // Add column families</span>
<span class="source-line-no">912</span><span id="line-912"> // Build a set of keys</span>
<span class="source-line-no">913</span><span id="line-913"> List&lt;ColumnFamilyDescriptorBuilder&gt; familyBuilders = new ArrayList&lt;&gt;();</span>
<span class="source-line-no">914</span><span id="line-914"> SortedMap&lt;byte[], Integer&gt; map = new TreeMap&lt;&gt;(Bytes.BYTES_COMPARATOR);</span>
<span class="source-line-no">915</span><span id="line-915"> visitBulkHFiles(fs, hfofDir, new BulkHFileVisitor&lt;ColumnFamilyDescriptorBuilder&gt;() {</span>
<span class="source-line-no">916</span><span id="line-916"> @Override</span>
<span class="source-line-no">917</span><span id="line-917"> public ColumnFamilyDescriptorBuilder bulkFamily(byte[] familyName) {</span>
<span class="source-line-no">918</span><span id="line-918"> ColumnFamilyDescriptorBuilder builder =</span>
<span class="source-line-no">919</span><span id="line-919"> ColumnFamilyDescriptorBuilder.newBuilder(familyName);</span>
<span class="source-line-no">920</span><span id="line-920"> familyBuilders.add(builder);</span>
<span class="source-line-no">921</span><span id="line-921"> return builder;</span>
<span class="source-line-no">922</span><span id="line-922"> }</span>
<span class="source-line-no">923</span><span id="line-923"></span>
<span class="source-line-no">924</span><span id="line-924"> @Override</span>
<span class="source-line-no">925</span><span id="line-925"> public void bulkHFile(ColumnFamilyDescriptorBuilder builder, FileStatus hfileStatus)</span>
<span class="source-line-no">926</span><span id="line-926"> throws IOException {</span>
<span class="source-line-no">927</span><span id="line-927"> Path hfile = hfileStatus.getPath();</span>
<span class="source-line-no">928</span><span id="line-928"> try (HFile.Reader reader =</span>
<span class="source-line-no">929</span><span id="line-929"> HFile.createReader(fs, hfile, CacheConfig.DISABLED, true, getConf())) {</span>
<span class="source-line-no">930</span><span id="line-930"> if (builder.getCompressionType() != reader.getFileContext().getCompression()) {</span>
<span class="source-line-no">931</span><span id="line-931"> builder.setCompressionType(reader.getFileContext().getCompression());</span>
<span class="source-line-no">932</span><span id="line-932"> LOG.info("Setting compression " + reader.getFileContext().getCompression().name()</span>
<span class="source-line-no">933</span><span id="line-933"> + " for family " + builder.getNameAsString());</span>
<span class="source-line-no">934</span><span id="line-934"> }</span>
<span class="source-line-no">935</span><span id="line-935"> byte[] first = reader.getFirstRowKey().get();</span>
<span class="source-line-no">936</span><span id="line-936"> byte[] last = reader.getLastRowKey().get();</span>
<span class="source-line-no">937</span><span id="line-937"></span>
<span class="source-line-no">938</span><span id="line-938"> LOG.info("Trying to figure out region boundaries hfile=" + hfile + " first="</span>
<span class="source-line-no">939</span><span id="line-939"> + Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last));</span>
<span class="source-line-no">940</span><span id="line-940"></span>
<span class="source-line-no">941</span><span id="line-941"> // To eventually infer start key-end key boundaries</span>
<span class="source-line-no">942</span><span id="line-942"> Integer value = map.getOrDefault(first, 0);</span>
<span class="source-line-no">943</span><span id="line-943"> map.put(first, value + 1);</span>
<span class="source-line-no">944</span><span id="line-944"></span>
<span class="source-line-no">945</span><span id="line-945"> value = map.containsKey(last) ? map.get(last) : 0;</span>
<span class="source-line-no">946</span><span id="line-946"> map.put(last, value - 1);</span>
<span class="source-line-no">947</span><span id="line-947"> }</span>
<span class="source-line-no">948</span><span id="line-948"> }</span>
<span class="source-line-no">949</span><span id="line-949"> }, true);</span>
<span class="source-line-no">950</span><span id="line-950"></span>
<span class="source-line-no">951</span><span id="line-951"> byte[][] keys = inferBoundaries(map);</span>
<span class="source-line-no">952</span><span id="line-952"> TableDescriptorBuilder tdBuilder = TableDescriptorBuilder.newBuilder(tableName);</span>
<span class="source-line-no">953</span><span id="line-953"> familyBuilders.stream().map(ColumnFamilyDescriptorBuilder::build)</span>
<span class="source-line-no">954</span><span id="line-954"> .forEachOrdered(tdBuilder::setColumnFamily);</span>
<span class="source-line-no">955</span><span id="line-955"> FutureUtils.get(admin.createTable(tdBuilder.build(), keys));</span>
<span class="source-line-no">956</span><span id="line-956"></span>
<span class="source-line-no">957</span><span id="line-957"> LOG.info("Table " + tableName + " is available!!");</span>
<span class="source-line-no">958</span><span id="line-958"> }</span>
<span class="source-line-no">959</span><span id="line-959"></span>
<span class="source-line-no">960</span><span id="line-960"> private Map&lt;LoadQueueItem, ByteBuffer&gt; performBulkLoad(AsyncClusterConnection conn,</span>
<span class="source-line-no">961</span><span id="line-961"> TableName tableName, Deque&lt;LoadQueueItem&gt; queue, ExecutorService pool, boolean copyFile)</span>
<span class="source-line-no">962</span><span id="line-962"> throws IOException {</span>
<span class="source-line-no">963</span><span id="line-963"> int count = 0;</span>
<span class="source-line-no">964</span><span id="line-964"></span>
<span class="source-line-no">965</span><span id="line-965"> fsDelegationToken.acquireDelegationToken(queue.peek().getFilePath().getFileSystem(getConf()));</span>
<span class="source-line-no">966</span><span id="line-966"> bulkToken = FutureUtils.get(conn.prepareBulkLoad(tableName));</span>
<span class="source-line-no">967</span><span id="line-967"> Pair&lt;Multimap&lt;ByteBuffer, LoadQueueItem&gt;, Set&lt;String&gt;&gt; pair = null;</span>
<span class="source-line-no">968</span><span id="line-968"></span>
<span class="source-line-no">969</span><span id="line-969"> Map&lt;LoadQueueItem, ByteBuffer&gt; item2RegionMap = new HashMap&lt;&gt;();</span>
<span class="source-line-no">970</span><span id="line-970"> // Assumes that region splits can happen while this occurs.</span>
<span class="source-line-no">971</span><span id="line-971"> while (!queue.isEmpty()) {</span>
<span class="source-line-no">972</span><span id="line-972"> // need to reload split keys each iteration.</span>
<span class="source-line-no">973</span><span id="line-973"> final List&lt;Pair&lt;byte[], byte[]&gt;&gt; startEndKeys =</span>
<span class="source-line-no">974</span><span id="line-974"> FutureUtils.get(conn.getRegionLocator(tableName).getStartEndKeys());</span>
<span class="source-line-no">975</span><span id="line-975"> if (count != 0) {</span>
<span class="source-line-no">976</span><span id="line-976"> LOG.info("Split occurred while grouping HFiles, retry attempt " + count + " with "</span>
<span class="source-line-no">977</span><span id="line-977"> + queue.size() + " files remaining to group or split");</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"> int maxRetries = getConf().getInt(HConstants.BULKLOAD_MAX_RETRIES_NUMBER, 10);</span>
<span class="source-line-no">981</span><span id="line-981"> maxRetries = Math.max(maxRetries, startEndKeys.size() + 1);</span>
<span class="source-line-no">982</span><span id="line-982"> if (maxRetries != 0 &amp;&amp; count &gt;= maxRetries) {</span>
<span class="source-line-no">983</span><span id="line-983"> throw new IOException(</span>
<span class="source-line-no">984</span><span id="line-984"> "Retry attempted " + count + " times without completing, bailing out");</span>
<span class="source-line-no">985</span><span id="line-985"> }</span>
<span class="source-line-no">986</span><span id="line-986"> count++;</span>
<span class="source-line-no">987</span><span id="line-987"></span>
<span class="source-line-no">988</span><span id="line-988"> // Using ByteBuffer for byte[] equality semantics</span>
<span class="source-line-no">989</span><span id="line-989"> pair = groupOrSplitPhase(conn, tableName, pool, queue, startEndKeys);</span>
<span class="source-line-no">990</span><span id="line-990"> Multimap&lt;ByteBuffer, LoadQueueItem&gt; regionGroups = pair.getFirst();</span>
<span class="source-line-no">991</span><span id="line-991"></span>
<span class="source-line-no">992</span><span id="line-992"> if (!checkHFilesCountPerRegionPerFamily(regionGroups)) {</span>
<span class="source-line-no">993</span><span id="line-993"> // Error is logged inside checkHFilesCountPerRegionPerFamily.</span>
<span class="source-line-no">994</span><span id="line-994"> throw new IOException("Trying to load more than " + maxFilesPerRegionPerFamily</span>
<span class="source-line-no">995</span><span id="line-995"> + " hfiles to one family of one region");</span>
<span class="source-line-no">996</span><span id="line-996"> }</span>
<span class="source-line-no">997</span><span id="line-997"></span>
<span class="source-line-no">998</span><span id="line-998"> bulkLoadPhase(conn, tableName, queue, regionGroups, copyFile, item2RegionMap);</span>
<span class="source-line-no">999</span><span id="line-999"></span>
<span class="source-line-no">1000</span><span id="line-1000"> // NOTE: The next iteration's split / group could happen in parallel to</span>
<span class="source-line-no">1001</span><span id="line-1001"> // atomic bulkloads assuming that there are splits and no merges, and</span>
<span class="source-line-no">1002</span><span id="line-1002"> // that we can atomically pull out the groups we want to retry.</span>
<span class="source-line-no">1003</span><span id="line-1003"> }</span>
<span class="source-line-no">1004</span><span id="line-1004"></span>
<span class="source-line-no">1005</span><span id="line-1005"> return item2RegionMap;</span>
<span class="source-line-no">1006</span><span id="line-1006"> }</span>
<span class="source-line-no">1007</span><span id="line-1007"></span>
<span class="source-line-no">1008</span><span id="line-1008"> private void cleanup(AsyncClusterConnection conn, TableName tableName, Deque&lt;LoadQueueItem&gt; queue,</span>
<span class="source-line-no">1009</span><span id="line-1009"> ExecutorService pool) throws IOException {</span>
<span class="source-line-no">1010</span><span id="line-1010"> fsDelegationToken.releaseDelegationToken();</span>
<span class="source-line-no">1011</span><span id="line-1011"> if (bulkToken != null) {</span>
<span class="source-line-no">1012</span><span id="line-1012"> conn.cleanupBulkLoad(tableName, bulkToken);</span>
<span class="source-line-no">1013</span><span id="line-1013"> }</span>
<span class="source-line-no">1014</span><span id="line-1014"> if (pool != null) {</span>
<span class="source-line-no">1015</span><span id="line-1015"> pool.shutdown();</span>
<span class="source-line-no">1016</span><span id="line-1016"> }</span>
<span class="source-line-no">1017</span><span id="line-1017"> if (!queue.isEmpty()) {</span>
<span class="source-line-no">1018</span><span id="line-1018"> StringBuilder err = new StringBuilder();</span>
<span class="source-line-no">1019</span><span id="line-1019"> err.append("-------------------------------------------------\n");</span>
<span class="source-line-no">1020</span><span id="line-1020"> err.append("Bulk load aborted with some files not yet loaded:\n");</span>
<span class="source-line-no">1021</span><span id="line-1021"> err.append("-------------------------------------------------\n");</span>
<span class="source-line-no">1022</span><span id="line-1022"> for (LoadQueueItem q : queue) {</span>
<span class="source-line-no">1023</span><span id="line-1023"> err.append(" ").append(q.getFilePath()).append('\n');</span>
<span class="source-line-no">1024</span><span id="line-1024"> }</span>
<span class="source-line-no">1025</span><span id="line-1025"> LOG.error(err.toString());</span>
<span class="source-line-no">1026</span><span id="line-1026"> }</span>
<span class="source-line-no">1027</span><span id="line-1027"> }</span>
<span class="source-line-no">1028</span><span id="line-1028"></span>
<span class="source-line-no">1029</span><span id="line-1029"> /**</span>
<span class="source-line-no">1030</span><span id="line-1030"> * Perform a bulk load of the given map of families to hfiles into the given pre-existing table.</span>
<span class="source-line-no">1031</span><span id="line-1031"> * This method is not threadsafe.</span>
<span class="source-line-no">1032</span><span id="line-1032"> * @param map map of family to List of hfiles</span>
<span class="source-line-no">1033</span><span id="line-1033"> * @param tableName table to load the hfiles</span>
<span class="source-line-no">1034</span><span id="line-1034"> * @param silence true to ignore unmatched column families</span>
<span class="source-line-no">1035</span><span id="line-1035"> * @param copyFile always copy hfiles if true</span>
<span class="source-line-no">1036</span><span id="line-1036"> */</span>
<span class="source-line-no">1037</span><span id="line-1037"> private Map&lt;LoadQueueItem, ByteBuffer&gt; doBulkLoad(AsyncClusterConnection conn,</span>
<span class="source-line-no">1038</span><span id="line-1038"> TableName tableName, Map&lt;byte[], List&lt;Path&gt;&gt; map, boolean silence, boolean copyFile)</span>
<span class="source-line-no">1039</span><span id="line-1039"> throws IOException {</span>
<span class="source-line-no">1040</span><span id="line-1040"> tableExists(conn, tableName);</span>
<span class="source-line-no">1041</span><span id="line-1041"> // LQI queue does not need to be threadsafe -- all operations on this queue</span>
<span class="source-line-no">1042</span><span id="line-1042"> // happen in this thread</span>
<span class="source-line-no">1043</span><span id="line-1043"> Deque&lt;LoadQueueItem&gt; queue = new ArrayDeque&lt;&gt;();</span>
<span class="source-line-no">1044</span><span id="line-1044"> ExecutorService pool = null;</span>
<span class="source-line-no">1045</span><span id="line-1045"> try {</span>
<span class="source-line-no">1046</span><span id="line-1046"> prepareHFileQueue(conn, tableName, map, queue, silence);</span>
<span class="source-line-no">1047</span><span id="line-1047"> if (queue.isEmpty()) {</span>
<span class="source-line-no">1048</span><span id="line-1048"> LOG.warn("Bulk load operation did not get any files to load");</span>
<span class="source-line-no">1049</span><span id="line-1049"> return Collections.emptyMap();</span>
<span class="source-line-no">1050</span><span id="line-1050"> }</span>
<span class="source-line-no">1051</span><span id="line-1051"> pool = createExecutorService();</span>
<span class="source-line-no">1052</span><span id="line-1052"> return performBulkLoad(conn, tableName, queue, pool, copyFile);</span>
<span class="source-line-no">1053</span><span id="line-1053"> } finally {</span>
<span class="source-line-no">1054</span><span id="line-1054"> cleanup(conn, tableName, queue, pool);</span>
<span class="source-line-no">1055</span><span id="line-1055"> }</span>
<span class="source-line-no">1056</span><span id="line-1056"> }</span>
<span class="source-line-no">1057</span><span id="line-1057"></span>
<span class="source-line-no">1058</span><span id="line-1058"> /**</span>
<span class="source-line-no">1059</span><span id="line-1059"> * Perform a bulk load of the given directory into the given pre-existing table. This method is</span>
<span class="source-line-no">1060</span><span id="line-1060"> * not threadsafe.</span>
<span class="source-line-no">1061</span><span id="line-1061"> * @param tableName table to load the hfiles</span>
<span class="source-line-no">1062</span><span id="line-1062"> * @param hfofDir the directory that was provided as the output path of a job using</span>
<span class="source-line-no">1063</span><span id="line-1063"> * HFileOutputFormat</span>
<span class="source-line-no">1064</span><span id="line-1064"> * @param silence true to ignore unmatched column families</span>
<span class="source-line-no">1065</span><span id="line-1065"> * @param copyFile always copy hfiles if true</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&lt;LoadQueueItem, ByteBuffer&gt; doBulkLoad(AsyncClusterConnection conn,</span>
<span class="source-line-no">1068</span><span id="line-1068"> TableName tableName, Path hfofDir, boolean silence, boolean copyFile) throws IOException {</span>
<span class="source-line-no">1069</span><span id="line-1069"> tableExists(conn, tableName);</span>
<span class="source-line-no">1070</span><span id="line-1070"></span>
<span class="source-line-no">1071</span><span id="line-1071"> /*</span>
<span class="source-line-no">1072</span><span id="line-1072"> * Checking hfile format is a time-consuming operation, we should have an option to skip this</span>
<span class="source-line-no">1073</span><span id="line-1073"> * step when bulkloading millions of HFiles. See HBASE-13985.</span>
<span class="source-line-no">1074</span><span id="line-1074"> */</span>
<span class="source-line-no">1075</span><span id="line-1075"> boolean validateHFile = getConf().getBoolean(VALIDATE_HFILES, true);</span>
<span class="source-line-no">1076</span><span id="line-1076"> if (!validateHFile) {</span>
<span class="source-line-no">1077</span><span id="line-1077"> LOG.warn("You are skipping HFiles validation, it might cause some data loss if files "</span>
<span class="source-line-no">1078</span><span id="line-1078"> + "are not correct. If you fail to read data from your table after using this "</span>
<span class="source-line-no">1079</span><span id="line-1079"> + "option, consider removing the files and bulkload again without this option. "</span>
<span class="source-line-no">1080</span><span id="line-1080"> + "See HBASE-13985");</span>
<span class="source-line-no">1081</span><span id="line-1081"> }</span>
<span class="source-line-no">1082</span><span id="line-1082"> // LQI queue does not need to be threadsafe -- all operations on this queue</span>
<span class="source-line-no">1083</span><span id="line-1083"> // happen in this thread</span>
<span class="source-line-no">1084</span><span id="line-1084"> Deque&lt;LoadQueueItem&gt; queue = new ArrayDeque&lt;&gt;();</span>
<span class="source-line-no">1085</span><span id="line-1085"> ExecutorService pool = null;</span>
<span class="source-line-no">1086</span><span id="line-1086"> try {</span>
<span class="source-line-no">1087</span><span id="line-1087"> prepareHFileQueue(getConf(), conn, tableName, hfofDir, queue, validateHFile, silence);</span>
<span class="source-line-no">1088</span><span id="line-1088"></span>
<span class="source-line-no">1089</span><span id="line-1089"> if (queue.isEmpty()) {</span>
<span class="source-line-no">1090</span><span id="line-1090"> LOG.warn(</span>
<span class="source-line-no">1091</span><span id="line-1091"> "Bulk load operation did not find any files to load in directory {}. "</span>
<span class="source-line-no">1092</span><span id="line-1092"> + "Does it contain files in subdirectories that correspond to column family names?",</span>
<span class="source-line-no">1093</span><span id="line-1093"> (hfofDir != null ? hfofDir.toUri().toString() : ""));</span>
<span class="source-line-no">1094</span><span id="line-1094"> return Collections.emptyMap();</span>
<span class="source-line-no">1095</span><span id="line-1095"> }</span>
<span class="source-line-no">1096</span><span id="line-1096"> pool = createExecutorService();</span>
<span class="source-line-no">1097</span><span id="line-1097"> return performBulkLoad(conn, tableName, queue, pool, copyFile);</span>
<span class="source-line-no">1098</span><span id="line-1098"> } finally {</span>
<span class="source-line-no">1099</span><span id="line-1099"> cleanup(conn, tableName, queue, pool);</span>
<span class="source-line-no">1100</span><span id="line-1100"> }</span>
<span class="source-line-no">1101</span><span id="line-1101"> }</span>
<span class="source-line-no">1102</span><span id="line-1102"></span>
<span class="source-line-no">1103</span><span id="line-1103"> @Override</span>
<span class="source-line-no">1104</span><span id="line-1104"> public Map&lt;LoadQueueItem, ByteBuffer&gt; bulkLoad(TableName tableName,</span>
<span class="source-line-no">1105</span><span id="line-1105"> Map&lt;byte[], List&lt;Path&gt;&gt; family2Files) throws IOException {</span>
<span class="source-line-no">1106</span><span id="line-1106"> try (AsyncClusterConnection conn = ClusterConnectionFactory</span>
<span class="source-line-no">1107</span><span id="line-1107"> .createAsyncClusterConnection(getConf(), null, userProvider.getCurrent())) {</span>
<span class="source-line-no">1108</span><span id="line-1108"> return doBulkLoad(conn, tableName, family2Files, isSilence(), isAlwaysCopyFiles());</span>
<span class="source-line-no">1109</span><span id="line-1109"> }</span>
<span class="source-line-no">1110</span><span id="line-1110"> }</span>
<span class="source-line-no">1111</span><span id="line-1111"></span>
<span class="source-line-no">1112</span><span id="line-1112"> @Override</span>
<span class="source-line-no">1113</span><span id="line-1113"> public Map&lt;LoadQueueItem, ByteBuffer&gt; bulkLoad(TableName tableName, Path dir) throws IOException {</span>
<span class="source-line-no">1114</span><span id="line-1114"> try (AsyncClusterConnection conn = ClusterConnectionFactory</span>
<span class="source-line-no">1115</span><span id="line-1115"> .createAsyncClusterConnection(getConf(), null, userProvider.getCurrent())) {</span>
<span class="source-line-no">1116</span><span id="line-1116"> AsyncAdmin admin = conn.getAdmin();</span>
<span class="source-line-no">1117</span><span id="line-1117"> if (!FutureUtils.get(admin.tableExists(tableName))) {</span>
<span class="source-line-no">1118</span><span id="line-1118"> if (isCreateTable()) {</span>
<span class="source-line-no">1119</span><span id="line-1119"> createTable(tableName, dir, admin);</span>
<span class="source-line-no">1120</span><span id="line-1120"> } else {</span>
<span class="source-line-no">1121</span><span id="line-1121"> throwAndLogTableNotFoundException(tableName);</span>
<span class="source-line-no">1122</span><span id="line-1122"> }</span>
<span class="source-line-no">1123</span><span id="line-1123"> }</span>
<span class="source-line-no">1124</span><span id="line-1124"> return doBulkLoad(conn, tableName, dir, isSilence(), isAlwaysCopyFiles());</span>
<span class="source-line-no">1125</span><span id="line-1125"> }</span>
<span class="source-line-no">1126</span><span id="line-1126"> }</span>
<span class="source-line-no">1127</span><span id="line-1127"></span>
<span class="source-line-no">1128</span><span id="line-1128"> /**</span>
<span class="source-line-no">1129</span><span id="line-1129"> * @throws TableNotFoundException if table does not exist.</span>
<span class="source-line-no">1130</span><span id="line-1130"> */</span>
<span class="source-line-no">1131</span><span id="line-1131"> private void tableExists(AsyncClusterConnection conn, TableName tableName) throws IOException {</span>
<span class="source-line-no">1132</span><span id="line-1132"> if (!FutureUtils.get(conn.getAdmin().tableExists(tableName))) {</span>
<span class="source-line-no">1133</span><span id="line-1133"> throwAndLogTableNotFoundException(tableName);</span>
<span class="source-line-no">1134</span><span id="line-1134"> }</span>
<span class="source-line-no">1135</span><span id="line-1135"> }</span>
<span class="source-line-no">1136</span><span id="line-1136"></span>
<span class="source-line-no">1137</span><span id="line-1137"> private void throwAndLogTableNotFoundException(TableName tn) throws TableNotFoundException {</span>
<span class="source-line-no">1138</span><span id="line-1138"> String errorMsg = format("Table '%s' does not exist.", tn);</span>
<span class="source-line-no">1139</span><span id="line-1139"> LOG.error(errorMsg);</span>
<span class="source-line-no">1140</span><span id="line-1140"> throw new TableNotFoundException(errorMsg);</span>
<span class="source-line-no">1141</span><span id="line-1141"> }</span>
<span class="source-line-no">1142</span><span id="line-1142"></span>
<span class="source-line-no">1143</span><span id="line-1143"> public void setBulkToken(String bulkToken) {</span>
<span class="source-line-no">1144</span><span id="line-1144"> this.bulkToken = bulkToken;</span>
<span class="source-line-no">1145</span><span id="line-1145"> }</span>
<span class="source-line-no">1146</span><span id="line-1146"></span>
<span class="source-line-no">1147</span><span id="line-1147"> public void setClusterIds(List&lt;String&gt; clusterIds) {</span>
<span class="source-line-no">1148</span><span id="line-1148"> this.clusterIds = clusterIds;</span>
<span class="source-line-no">1149</span><span id="line-1149"> }</span>
<span class="source-line-no">1150</span><span id="line-1150"></span>
<span class="source-line-no">1151</span><span id="line-1151"> private void usage() {</span>
<span class="source-line-no">1152</span><span id="line-1152"> System.err.println("Usage: " + "bin/hbase completebulkload [OPTIONS] "</span>
<span class="source-line-no">1153</span><span id="line-1153"> + "&lt;/PATH/TO/HFILEOUTPUTFORMAT-OUTPUT&gt; &lt;TABLENAME&gt;\n"</span>
<span class="source-line-no">1154</span><span id="line-1154"> + "Loads directory of hfiles -- a region dir or product of HFileOutputFormat -- "</span>
<span class="source-line-no">1155</span><span id="line-1155"> + "into an hbase table.\n" + "OPTIONS (for other -D options, see source code):\n" + " -D"</span>
<span class="source-line-no">1156</span><span id="line-1156"> + CREATE_TABLE_CONF_KEY + "=no whether to create table; when 'no', target "</span>
<span class="source-line-no">1157</span><span id="line-1157"> + "table must exist.\n" + " -D" + IGNORE_UNMATCHED_CF_CONF_KEY</span>
<span class="source-line-no">1158</span><span id="line-1158"> + "=yes to ignore unmatched column families.\n"</span>
<span class="source-line-no">1159</span><span id="line-1159"> + " -loadTable for when directory of files to load has a depth of 3; target table must "</span>
<span class="source-line-no">1160</span><span id="line-1160"> + "exist;\n" + " must be last of the options on command line.\n"</span>
<span class="source-line-no">1161</span><span id="line-1161"> + "See http://hbase.apache.org/book.html#arch.bulk.load.complete.strays for "</span>
<span class="source-line-no">1162</span><span id="line-1162"> + "documentation.\n");</span>
<span class="source-line-no">1163</span><span id="line-1163"> }</span>
<span class="source-line-no">1164</span><span id="line-1164"></span>
<span class="source-line-no">1165</span><span id="line-1165"> @Override</span>
<span class="source-line-no">1166</span><span id="line-1166"> public int run(String[] args) throws Exception {</span>
<span class="source-line-no">1167</span><span id="line-1167"> if (args.length != 2 &amp;&amp; args.length != 3) {</span>
<span class="source-line-no">1168</span><span id="line-1168"> usage();</span>
<span class="source-line-no">1169</span><span id="line-1169"> return -1;</span>
<span class="source-line-no">1170</span><span id="line-1170"> }</span>
<span class="source-line-no">1171</span><span id="line-1171"> // Re-initialize to apply -D options from the command line parameters</span>
<span class="source-line-no">1172</span><span id="line-1172"> initialize();</span>
<span class="source-line-no">1173</span><span id="line-1173"> Path dirPath = new Path(args[0]);</span>
<span class="source-line-no">1174</span><span id="line-1174"> TableName tableName = TableName.valueOf(args[1]);</span>
<span class="source-line-no">1175</span><span id="line-1175"> if (args.length == 2) {</span>
<span class="source-line-no">1176</span><span id="line-1176"> return !bulkLoad(tableName, dirPath).isEmpty() ? 0 : -1;</span>
<span class="source-line-no">1177</span><span id="line-1177"> } else {</span>
<span class="source-line-no">1178</span><span id="line-1178"> Map&lt;byte[], List&lt;Path&gt;&gt; family2Files = Maps.newHashMap();</span>
<span class="source-line-no">1179</span><span id="line-1179"> FileSystem fs = FileSystem.get(getConf());</span>
<span class="source-line-no">1180</span><span id="line-1180"> for (FileStatus regionDir : fs.listStatus(dirPath)) {</span>
<span class="source-line-no">1181</span><span id="line-1181"> FSVisitor.visitRegionStoreFiles(fs, regionDir.getPath(), (region, family, hfileName) -&gt; {</span>
<span class="source-line-no">1182</span><span id="line-1182"> Path path = new Path(regionDir.getPath(), new Path(family, hfileName));</span>
<span class="source-line-no">1183</span><span id="line-1183"> byte[] familyName = Bytes.toBytes(family);</span>
<span class="source-line-no">1184</span><span id="line-1184"> if (family2Files.containsKey(familyName)) {</span>
<span class="source-line-no">1185</span><span id="line-1185"> family2Files.get(familyName).add(path);</span>
<span class="source-line-no">1186</span><span id="line-1186"> } else {</span>
<span class="source-line-no">1187</span><span id="line-1187"> family2Files.put(familyName, Lists.newArrayList(path));</span>
<span class="source-line-no">1188</span><span id="line-1188"> }</span>
<span class="source-line-no">1189</span><span id="line-1189"> });</span>
<span class="source-line-no">1190</span><span id="line-1190"> }</span>
<span class="source-line-no">1191</span><span id="line-1191"> return !bulkLoad(tableName, family2Files).isEmpty() ? 0 : -1;</span>
<span class="source-line-no">1192</span><span id="line-1192"> }</span>
<span class="source-line-no">1193</span><span id="line-1193"> }</span>
<span class="source-line-no">1194</span><span id="line-1194"></span>
<span class="source-line-no">1195</span><span id="line-1195"> public static void main(String[] args) throws Exception {</span>
<span class="source-line-no">1196</span><span id="line-1196"> Configuration conf = HBaseConfiguration.create();</span>
<span class="source-line-no">1197</span><span id="line-1197"> int ret = ToolRunner.run(conf, new BulkLoadHFilesTool(conf), args);</span>
<span class="source-line-no">1198</span><span id="line-1198"> System.exit(ret);</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"> @Override</span>
<span class="source-line-no">1202</span><span id="line-1202"> public void disableReplication() {</span>
<span class="source-line-no">1203</span><span id="line-1203"> this.replicate = false;</span>
<span class="source-line-no">1204</span><span id="line-1204"> }</span>
<span class="source-line-no">1205</span><span id="line-1205"></span>
<span class="source-line-no">1206</span><span id="line-1206"> @Override</span>
<span class="source-line-no">1207</span><span id="line-1207"> public boolean isReplicationDisabled() {</span>
<span class="source-line-no">1208</span><span id="line-1208"> return !this.replicate;</span>
<span class="source-line-no">1209</span><span id="line-1209"> }</span>
<span class="source-line-no">1210</span><span id="line-1210">}</span>
</pre>
</div>
</main>
</body>
</html>