| <!DOCTYPE HTML> |
| <html lang="en"> |
| <head> |
| <!-- Generated by javadoc (17) --> |
| <title>Source code</title> |
| <meta name="viewport" content="width=device-width, initial-scale=1"> |
| <meta name="description" content="source: package: org.apache.hadoop.hbase.mapreduce, class: HFileOutputFormat2, class: WriterLength"> |
| <meta name="generator" content="javadoc/SourceToHTMLConverter"> |
| <link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style"> |
| </head> |
| <body class="source-page"> |
| <main role="main"> |
| <div class="source-container"> |
| <pre><span class="source-line-no">001</span><span id="line-1">/*</span> |
| <span class="source-line-no">002</span><span id="line-2"> * Licensed to the Apache Software Foundation (ASF) under one</span> |
| <span class="source-line-no">003</span><span id="line-3"> * or more contributor license agreements. See the NOTICE file</span> |
| <span class="source-line-no">004</span><span id="line-4"> * distributed with this work for additional information</span> |
| <span class="source-line-no">005</span><span id="line-5"> * regarding copyright ownership. The ASF licenses this file</span> |
| <span class="source-line-no">006</span><span id="line-6"> * to you under the Apache License, Version 2.0 (the</span> |
| <span class="source-line-no">007</span><span id="line-7"> * "License"); you may not use this file except in compliance</span> |
| <span class="source-line-no">008</span><span id="line-8"> * with the License. You may obtain a copy of the License at</span> |
| <span class="source-line-no">009</span><span id="line-9"> *</span> |
| <span class="source-line-no">010</span><span id="line-10"> * http://www.apache.org/licenses/LICENSE-2.0</span> |
| <span class="source-line-no">011</span><span id="line-11"> *</span> |
| <span class="source-line-no">012</span><span id="line-12"> * Unless required by applicable law or agreed to in writing, software</span> |
| <span class="source-line-no">013</span><span id="line-13"> * distributed under the License is distributed on an "AS IS" BASIS,</span> |
| <span class="source-line-no">014</span><span id="line-14"> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span> |
| <span class="source-line-no">015</span><span id="line-15"> * See the License for the specific language governing permissions and</span> |
| <span class="source-line-no">016</span><span id="line-16"> * limitations under the License.</span> |
| <span class="source-line-no">017</span><span id="line-17"> */</span> |
| <span class="source-line-no">018</span><span id="line-18">package org.apache.hadoop.hbase.mapreduce;</span> |
| <span class="source-line-no">019</span><span id="line-19"></span> |
| <span class="source-line-no">020</span><span id="line-20">import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TASK_KEY;</span> |
| <span class="source-line-no">021</span><span id="line-21">import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY;</span> |
| <span class="source-line-no">022</span><span id="line-22">import static org.apache.hadoop.hbase.regionserver.HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY;</span> |
| <span class="source-line-no">023</span><span id="line-23">import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;</span> |
| <span class="source-line-no">024</span><span id="line-24"></span> |
| <span class="source-line-no">025</span><span id="line-25">import java.io.IOException;</span> |
| <span class="source-line-no">026</span><span id="line-26">import java.io.UnsupportedEncodingException;</span> |
| <span class="source-line-no">027</span><span id="line-27">import java.net.InetSocketAddress;</span> |
| <span class="source-line-no">028</span><span id="line-28">import java.net.URLDecoder;</span> |
| <span class="source-line-no">029</span><span id="line-29">import java.net.URLEncoder;</span> |
| <span class="source-line-no">030</span><span id="line-30">import java.nio.charset.Charset;</span> |
| <span class="source-line-no">031</span><span id="line-31">import java.util.ArrayList;</span> |
| <span class="source-line-no">032</span><span id="line-32">import java.util.Arrays;</span> |
| <span class="source-line-no">033</span><span id="line-33">import java.util.Collections;</span> |
| <span class="source-line-no">034</span><span id="line-34">import java.util.List;</span> |
| <span class="source-line-no">035</span><span id="line-35">import java.util.Map;</span> |
| <span class="source-line-no">036</span><span id="line-36">import java.util.Map.Entry;</span> |
| <span class="source-line-no">037</span><span id="line-37">import java.util.Set;</span> |
| <span class="source-line-no">038</span><span id="line-38">import java.util.TreeMap;</span> |
| <span class="source-line-no">039</span><span id="line-39">import java.util.TreeSet;</span> |
| <span class="source-line-no">040</span><span id="line-40">import java.util.UUID;</span> |
| <span class="source-line-no">041</span><span id="line-41">import java.util.function.Function;</span> |
| <span class="source-line-no">042</span><span id="line-42">import java.util.stream.Collectors;</span> |
| <span class="source-line-no">043</span><span id="line-43">import org.apache.commons.lang3.StringUtils;</span> |
| <span class="source-line-no">044</span><span id="line-44">import org.apache.hadoop.conf.Configuration;</span> |
| <span class="source-line-no">045</span><span id="line-45">import org.apache.hadoop.fs.FileSystem;</span> |
| <span class="source-line-no">046</span><span id="line-46">import org.apache.hadoop.fs.Path;</span> |
| <span class="source-line-no">047</span><span id="line-47">import org.apache.hadoop.hbase.Cell;</span> |
| <span class="source-line-no">048</span><span id="line-48">import org.apache.hadoop.hbase.CellUtil;</span> |
| <span class="source-line-no">049</span><span id="line-49">import org.apache.hadoop.hbase.ExtendedCell;</span> |
| <span class="source-line-no">050</span><span id="line-50">import org.apache.hadoop.hbase.HConstants;</span> |
| <span class="source-line-no">051</span><span id="line-51">import org.apache.hadoop.hbase.HRegionLocation;</span> |
| <span class="source-line-no">052</span><span id="line-52">import org.apache.hadoop.hbase.KeyValue;</span> |
| <span class="source-line-no">053</span><span id="line-53">import org.apache.hadoop.hbase.KeyValueUtil;</span> |
| <span class="source-line-no">054</span><span id="line-54">import org.apache.hadoop.hbase.PrivateCellUtil;</span> |
| <span class="source-line-no">055</span><span id="line-55">import org.apache.hadoop.hbase.TableName;</span> |
| <span class="source-line-no">056</span><span id="line-56">import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;</span> |
| <span class="source-line-no">057</span><span id="line-57">import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;</span> |
| <span class="source-line-no">058</span><span id="line-58">import org.apache.hadoop.hbase.client.Connection;</span> |
| <span class="source-line-no">059</span><span id="line-59">import org.apache.hadoop.hbase.client.ConnectionFactory;</span> |
| <span class="source-line-no">060</span><span id="line-60">import org.apache.hadoop.hbase.client.Put;</span> |
| <span class="source-line-no">061</span><span id="line-61">import org.apache.hadoop.hbase.client.RegionLocator;</span> |
| <span class="source-line-no">062</span><span id="line-62">import org.apache.hadoop.hbase.client.Table;</span> |
| <span class="source-line-no">063</span><span id="line-63">import org.apache.hadoop.hbase.client.TableDescriptor;</span> |
| <span class="source-line-no">064</span><span id="line-64">import org.apache.hadoop.hbase.fs.HFileSystem;</span> |
| <span class="source-line-no">065</span><span id="line-65">import org.apache.hadoop.hbase.io.ImmutableBytesWritable;</span> |
| <span class="source-line-no">066</span><span id="line-66">import org.apache.hadoop.hbase.io.compress.Compression;</span> |
| <span class="source-line-no">067</span><span id="line-67">import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;</span> |
| <span class="source-line-no">068</span><span id="line-68">import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;</span> |
| <span class="source-line-no">069</span><span id="line-69">import org.apache.hadoop.hbase.io.hfile.CacheConfig;</span> |
| <span class="source-line-no">070</span><span id="line-70">import org.apache.hadoop.hbase.io.hfile.HFile;</span> |
| <span class="source-line-no">071</span><span id="line-71">import org.apache.hadoop.hbase.io.hfile.HFileContext;</span> |
| <span class="source-line-no">072</span><span id="line-72">import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;</span> |
| <span class="source-line-no">073</span><span id="line-73">import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl;</span> |
| <span class="source-line-no">074</span><span id="line-74">import org.apache.hadoop.hbase.regionserver.BloomType;</span> |
| <span class="source-line-no">075</span><span id="line-75">import org.apache.hadoop.hbase.regionserver.HStore;</span> |
| <span class="source-line-no">076</span><span id="line-76">import org.apache.hadoop.hbase.regionserver.StoreFileWriter;</span> |
| <span class="source-line-no">077</span><span id="line-77">import org.apache.hadoop.hbase.regionserver.StoreUtils;</span> |
| <span class="source-line-no">078</span><span id="line-78">import org.apache.hadoop.hbase.util.BloomFilterUtil;</span> |
| <span class="source-line-no">079</span><span id="line-79">import org.apache.hadoop.hbase.util.Bytes;</span> |
| <span class="source-line-no">080</span><span id="line-80">import org.apache.hadoop.hbase.util.CommonFSUtils;</span> |
| <span class="source-line-no">081</span><span id="line-81">import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;</span> |
| <span class="source-line-no">082</span><span id="line-82">import org.apache.hadoop.hbase.util.MapReduceExtendedCell;</span> |
| <span class="source-line-no">083</span><span id="line-83">import org.apache.hadoop.io.NullWritable;</span> |
| <span class="source-line-no">084</span><span id="line-84">import org.apache.hadoop.io.SequenceFile;</span> |
| <span class="source-line-no">085</span><span id="line-85">import org.apache.hadoop.io.Text;</span> |
| <span class="source-line-no">086</span><span id="line-86">import org.apache.hadoop.io.Writable;</span> |
| <span class="source-line-no">087</span><span id="line-87">import org.apache.hadoop.mapreduce.Job;</span> |
| <span class="source-line-no">088</span><span id="line-88">import org.apache.hadoop.mapreduce.OutputCommitter;</span> |
| <span class="source-line-no">089</span><span id="line-89">import org.apache.hadoop.mapreduce.OutputFormat;</span> |
| <span class="source-line-no">090</span><span id="line-90">import org.apache.hadoop.mapreduce.RecordWriter;</span> |
| <span class="source-line-no">091</span><span id="line-91">import org.apache.hadoop.mapreduce.TaskAttemptContext;</span> |
| <span class="source-line-no">092</span><span id="line-92">import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;</span> |
| <span class="source-line-no">093</span><span id="line-93">import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter;</span> |
| <span class="source-line-no">094</span><span id="line-94">import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;</span> |
| <span class="source-line-no">095</span><span id="line-95">import org.apache.yetus.audience.InterfaceAudience;</span> |
| <span class="source-line-no">096</span><span id="line-96">import org.slf4j.Logger;</span> |
| <span class="source-line-no">097</span><span id="line-97">import org.slf4j.LoggerFactory;</span> |
| <span class="source-line-no">098</span><span id="line-98"></span> |
| <span class="source-line-no">099</span><span id="line-99">/**</span> |
| <span class="source-line-no">100</span><span id="line-100"> * Writes HFiles. Passed Cells must arrive in order. Writes current time as the sequence id for the</span> |
| <span class="source-line-no">101</span><span id="line-101"> * file. Sets the major compacted attribute on created {@link HFile}s. Calling write(null,null) will</span> |
| <span class="source-line-no">102</span><span id="line-102"> * forcibly roll all HFiles being written.</span> |
| <span class="source-line-no">103</span><span id="line-103"> * <p></span> |
| <span class="source-line-no">104</span><span id="line-104"> * Using this class as part of a MapReduce job is best done using</span> |
| <span class="source-line-no">105</span><span id="line-105"> * {@link #configureIncrementalLoad(Job, TableDescriptor, RegionLocator)}.</span> |
| <span class="source-line-no">106</span><span id="line-106"> */</span> |
| <span class="source-line-no">107</span><span id="line-107">@InterfaceAudience.Public</span> |
| <span class="source-line-no">108</span><span id="line-108">public class HFileOutputFormat2 extends FileOutputFormat<ImmutableBytesWritable, Cell> {</span> |
| <span class="source-line-no">109</span><span id="line-109"> private static final Logger LOG = LoggerFactory.getLogger(HFileOutputFormat2.class);</span> |
| <span class="source-line-no">110</span><span id="line-110"></span> |
| <span class="source-line-no">111</span><span id="line-111"> static class TableInfo {</span> |
| <span class="source-line-no">112</span><span id="line-112"> private TableDescriptor tableDesctiptor;</span> |
| <span class="source-line-no">113</span><span id="line-113"> private RegionLocator regionLocator;</span> |
| <span class="source-line-no">114</span><span id="line-114"></span> |
| <span class="source-line-no">115</span><span id="line-115"> public TableInfo(TableDescriptor tableDesctiptor, RegionLocator regionLocator) {</span> |
| <span class="source-line-no">116</span><span id="line-116"> this.tableDesctiptor = tableDesctiptor;</span> |
| <span class="source-line-no">117</span><span id="line-117"> this.regionLocator = regionLocator;</span> |
| <span class="source-line-no">118</span><span id="line-118"> }</span> |
| <span class="source-line-no">119</span><span id="line-119"></span> |
| <span class="source-line-no">120</span><span id="line-120"> public TableDescriptor getTableDescriptor() {</span> |
| <span class="source-line-no">121</span><span id="line-121"> return tableDesctiptor;</span> |
| <span class="source-line-no">122</span><span id="line-122"> }</span> |
| <span class="source-line-no">123</span><span id="line-123"></span> |
| <span class="source-line-no">124</span><span id="line-124"> public RegionLocator getRegionLocator() {</span> |
| <span class="source-line-no">125</span><span id="line-125"> return regionLocator;</span> |
| <span class="source-line-no">126</span><span id="line-126"> }</span> |
| <span class="source-line-no">127</span><span id="line-127"> }</span> |
| <span class="source-line-no">128</span><span id="line-128"></span> |
| <span class="source-line-no">129</span><span id="line-129"> protected static final byte[] tableSeparator = Bytes.toBytes(";");</span> |
| <span class="source-line-no">130</span><span id="line-130"></span> |
| <span class="source-line-no">131</span><span id="line-131"> protected static byte[] combineTableNameSuffix(byte[] tableName, byte[] suffix) {</span> |
| <span class="source-line-no">132</span><span id="line-132"> return Bytes.add(tableName, tableSeparator, suffix);</span> |
| <span class="source-line-no">133</span><span id="line-133"> }</span> |
| <span class="source-line-no">134</span><span id="line-134"></span> |
| <span class="source-line-no">135</span><span id="line-135"> // The following constants are private since these are used by</span> |
| <span class="source-line-no">136</span><span id="line-136"> // HFileOutputFormat2 to internally transfer data between job setup and</span> |
| <span class="source-line-no">137</span><span id="line-137"> // reducer run using conf.</span> |
| <span class="source-line-no">138</span><span id="line-138"> // These should not be changed by the client.</span> |
| <span class="source-line-no">139</span><span id="line-139"> static final String COMPRESSION_FAMILIES_CONF_KEY =</span> |
| <span class="source-line-no">140</span><span id="line-140"> "hbase.hfileoutputformat.families.compression";</span> |
| <span class="source-line-no">141</span><span id="line-141"> static final String BLOOM_TYPE_FAMILIES_CONF_KEY = "hbase.hfileoutputformat.families.bloomtype";</span> |
| <span class="source-line-no">142</span><span id="line-142"> static final String BLOOM_PARAM_FAMILIES_CONF_KEY = "hbase.hfileoutputformat.families.bloomparam";</span> |
| <span class="source-line-no">143</span><span id="line-143"> static final String BLOCK_SIZE_FAMILIES_CONF_KEY = "hbase.mapreduce.hfileoutputformat.blocksize";</span> |
| <span class="source-line-no">144</span><span id="line-144"> static final String DATABLOCK_ENCODING_FAMILIES_CONF_KEY =</span> |
| <span class="source-line-no">145</span><span id="line-145"> "hbase.mapreduce.hfileoutputformat.families.datablock.encoding";</span> |
| <span class="source-line-no">146</span><span id="line-146"></span> |
| <span class="source-line-no">147</span><span id="line-147"> // This constant is public since the client can modify this when setting</span> |
| <span class="source-line-no">148</span><span id="line-148"> // up their conf object and thus refer to this symbol.</span> |
| <span class="source-line-no">149</span><span id="line-149"> // It is present for backwards compatibility reasons. Use it only to</span> |
| <span class="source-line-no">150</span><span id="line-150"> // override the auto-detection of datablock encoding and compression.</span> |
| <span class="source-line-no">151</span><span id="line-151"> public static final String DATABLOCK_ENCODING_OVERRIDE_CONF_KEY =</span> |
| <span class="source-line-no">152</span><span id="line-152"> "hbase.mapreduce.hfileoutputformat.datablock.encoding";</span> |
| <span class="source-line-no">153</span><span id="line-153"> public static final String COMPRESSION_OVERRIDE_CONF_KEY =</span> |
| <span class="source-line-no">154</span><span id="line-154"> "hbase.mapreduce.hfileoutputformat.compression";</span> |
| <span class="source-line-no">155</span><span id="line-155"></span> |
| <span class="source-line-no">156</span><span id="line-156"> /**</span> |
| <span class="source-line-no">157</span><span id="line-157"> * Keep locality while generating HFiles for bulkload. See HBASE-12596</span> |
| <span class="source-line-no">158</span><span id="line-158"> */</span> |
| <span class="source-line-no">159</span><span id="line-159"> public static final String LOCALITY_SENSITIVE_CONF_KEY =</span> |
| <span class="source-line-no">160</span><span id="line-160"> "hbase.bulkload.locality.sensitive.enabled";</span> |
| <span class="source-line-no">161</span><span id="line-161"> private static final boolean DEFAULT_LOCALITY_SENSITIVE = true;</span> |
| <span class="source-line-no">162</span><span id="line-162"> static final String OUTPUT_TABLE_NAME_CONF_KEY = "hbase.mapreduce.hfileoutputformat.table.name";</span> |
| <span class="source-line-no">163</span><span id="line-163"> static final String MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY =</span> |
| <span class="source-line-no">164</span><span id="line-164"> "hbase.mapreduce.use.multi.table.hfileoutputformat";</span> |
| <span class="source-line-no">165</span><span id="line-165"></span> |
| <span class="source-line-no">166</span><span id="line-166"> /**</span> |
| <span class="source-line-no">167</span><span id="line-167"> * ExtendedCell and ExtendedCellSerialization are InterfaceAudience.Private. We expose this config</span> |
| <span class="source-line-no">168</span><span id="line-168"> * for internal usage in jobs like WALPlayer which need to use features of ExtendedCell.</span> |
| <span class="source-line-no">169</span><span id="line-169"> */</span> |
| <span class="source-line-no">170</span><span id="line-170"> @InterfaceAudience.Private</span> |
| <span class="source-line-no">171</span><span id="line-171"> public static final String EXTENDED_CELL_SERIALIZATION_ENABLED_KEY =</span> |
| <span class="source-line-no">172</span><span id="line-172"> "hbase.mapreduce.hfileoutputformat.extendedcell.enabled";</span> |
| <span class="source-line-no">173</span><span id="line-173"> static final boolean EXTENDED_CELL_SERIALIZATION_ENABLED_DEFULT = false;</span> |
| <span class="source-line-no">174</span><span id="line-174"></span> |
| <span class="source-line-no">175</span><span id="line-175"> @InterfaceAudience.Private</span> |
| <span class="source-line-no">176</span><span id="line-176"> public static final String DISK_BASED_SORTING_ENABLED_KEY =</span> |
| <span class="source-line-no">177</span><span id="line-177"> "hbase.mapreduce.hfileoutputformat.disk.based.sorting.enabled";</span> |
| <span class="source-line-no">178</span><span id="line-178"> private static final boolean DISK_BASED_SORTING_ENABLED_DEFAULT = false;</span> |
| <span class="source-line-no">179</span><span id="line-179"></span> |
| <span class="source-line-no">180</span><span id="line-180"> public static final String REMOTE_CLUSTER_CONF_PREFIX = "hbase.hfileoutputformat.remote.cluster.";</span> |
| <span class="source-line-no">181</span><span id="line-181"> public static final String REMOTE_CLUSTER_ZOOKEEPER_QUORUM_CONF_KEY =</span> |
| <span class="source-line-no">182</span><span id="line-182"> REMOTE_CLUSTER_CONF_PREFIX + "zookeeper.quorum";</span> |
| <span class="source-line-no">183</span><span id="line-183"> public static final String REMOTE_CLUSTER_ZOOKEEPER_CLIENT_PORT_CONF_KEY =</span> |
| <span class="source-line-no">184</span><span id="line-184"> REMOTE_CLUSTER_CONF_PREFIX + "zookeeper." + HConstants.CLIENT_PORT_STR;</span> |
| <span class="source-line-no">185</span><span id="line-185"> public static final String REMOTE_CLUSTER_ZOOKEEPER_ZNODE_PARENT_CONF_KEY =</span> |
| <span class="source-line-no">186</span><span id="line-186"> REMOTE_CLUSTER_CONF_PREFIX + HConstants.ZOOKEEPER_ZNODE_PARENT;</span> |
| <span class="source-line-no">187</span><span id="line-187"></span> |
| <span class="source-line-no">188</span><span id="line-188"> public static final String STORAGE_POLICY_PROPERTY = HStore.BLOCK_STORAGE_POLICY_KEY;</span> |
| <span class="source-line-no">189</span><span id="line-189"> public static final String STORAGE_POLICY_PROPERTY_CF_PREFIX = STORAGE_POLICY_PROPERTY + ".";</span> |
| <span class="source-line-no">190</span><span id="line-190"></span> |
| <span class="source-line-no">191</span><span id="line-191"> @Override</span> |
| <span class="source-line-no">192</span><span id="line-192"> public RecordWriter<ImmutableBytesWritable, Cell></span> |
| <span class="source-line-no">193</span><span id="line-193"> getRecordWriter(final TaskAttemptContext context) throws IOException, InterruptedException {</span> |
| <span class="source-line-no">194</span><span id="line-194"> return createRecordWriter(context, this.getOutputCommitter(context));</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"> protected static byte[] getTableNameSuffixedWithFamily(byte[] tableName, byte[] family) {</span> |
| <span class="source-line-no">198</span><span id="line-198"> return combineTableNameSuffix(tableName, family);</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"> static <V extends Cell> RecordWriter<ImmutableBytesWritable, V> createRecordWriter(</span> |
| <span class="source-line-no">202</span><span id="line-202"> final TaskAttemptContext context, final OutputCommitter committer) throws IOException {</span> |
| <span class="source-line-no">203</span><span id="line-203"></span> |
| <span class="source-line-no">204</span><span id="line-204"> // Get the path of the temporary output file</span> |
| <span class="source-line-no">205</span><span id="line-205"> final Path outputDir = ((PathOutputCommitter) committer).getWorkPath();</span> |
| <span class="source-line-no">206</span><span id="line-206"> final Configuration conf = context.getConfiguration();</span> |
| <span class="source-line-no">207</span><span id="line-207"> final boolean writeMultipleTables =</span> |
| <span class="source-line-no">208</span><span id="line-208"> conf.getBoolean(MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY, false);</span> |
| <span class="source-line-no">209</span><span id="line-209"> final String writeTableNames = conf.get(OUTPUT_TABLE_NAME_CONF_KEY);</span> |
| <span class="source-line-no">210</span><span id="line-210"> if (writeTableNames == null || writeTableNames.isEmpty()) {</span> |
| <span class="source-line-no">211</span><span id="line-211"> throw new IllegalArgumentException("" + OUTPUT_TABLE_NAME_CONF_KEY + " cannot be empty");</span> |
| <span class="source-line-no">212</span><span id="line-212"> }</span> |
| <span class="source-line-no">213</span><span id="line-213"> final FileSystem fs = outputDir.getFileSystem(conf);</span> |
| <span class="source-line-no">214</span><span id="line-214"> // These configs. are from hbase-*.xml</span> |
| <span class="source-line-no">215</span><span id="line-215"> final long maxsize =</span> |
| <span class="source-line-no">216</span><span id="line-216"> conf.getLong(HConstants.HREGION_MAX_FILESIZE, HConstants.DEFAULT_MAX_FILE_SIZE);</span> |
| <span class="source-line-no">217</span><span id="line-217"> // Invented config. Add to hbase-*.xml if other than default compression.</span> |
| <span class="source-line-no">218</span><span id="line-218"> final String defaultCompressionStr =</span> |
| <span class="source-line-no">219</span><span id="line-219"> conf.get("hfile.compression", Compression.Algorithm.NONE.getName());</span> |
| <span class="source-line-no">220</span><span id="line-220"> final Algorithm defaultCompression = HFileWriterImpl.compressionByName(defaultCompressionStr);</span> |
| <span class="source-line-no">221</span><span id="line-221"> String compressionStr = conf.get(COMPRESSION_OVERRIDE_CONF_KEY);</span> |
| <span class="source-line-no">222</span><span id="line-222"> final Algorithm overriddenCompression =</span> |
| <span class="source-line-no">223</span><span id="line-223"> compressionStr != null ? Compression.getCompressionAlgorithmByName(compressionStr) : null;</span> |
| <span class="source-line-no">224</span><span id="line-224"> final boolean compactionExclude =</span> |
| <span class="source-line-no">225</span><span id="line-225"> conf.getBoolean("hbase.mapreduce.hfileoutputformat.compaction.exclude", false);</span> |
| <span class="source-line-no">226</span><span id="line-226"> final Set<String> allTableNames = Arrays</span> |
| <span class="source-line-no">227</span><span id="line-227"> .stream(writeTableNames.split(Bytes.toString(tableSeparator))).collect(Collectors.toSet());</span> |
| <span class="source-line-no">228</span><span id="line-228"></span> |
| <span class="source-line-no">229</span><span id="line-229"> // create a map from column family to the compression algorithm</span> |
| <span class="source-line-no">230</span><span id="line-230"> final Map<byte[], Algorithm> compressionMap = createFamilyCompressionMap(conf);</span> |
| <span class="source-line-no">231</span><span id="line-231"> final Map<byte[], BloomType> bloomTypeMap = createFamilyBloomTypeMap(conf);</span> |
| <span class="source-line-no">232</span><span id="line-232"> final Map<byte[], String> bloomParamMap = createFamilyBloomParamMap(conf);</span> |
| <span class="source-line-no">233</span><span id="line-233"> final Map<byte[], Integer> blockSizeMap = createFamilyBlockSizeMap(conf);</span> |
| <span class="source-line-no">234</span><span id="line-234"></span> |
| <span class="source-line-no">235</span><span id="line-235"> String dataBlockEncodingStr = conf.get(DATABLOCK_ENCODING_OVERRIDE_CONF_KEY);</span> |
| <span class="source-line-no">236</span><span id="line-236"> final Map<byte[], DataBlockEncoding> datablockEncodingMap =</span> |
| <span class="source-line-no">237</span><span id="line-237"> createFamilyDataBlockEncodingMap(conf);</span> |
| <span class="source-line-no">238</span><span id="line-238"> final DataBlockEncoding overriddenEncoding =</span> |
| <span class="source-line-no">239</span><span id="line-239"> dataBlockEncodingStr != null ? DataBlockEncoding.valueOf(dataBlockEncodingStr) : null;</span> |
| <span class="source-line-no">240</span><span id="line-240"></span> |
| <span class="source-line-no">241</span><span id="line-241"> return new RecordWriter<ImmutableBytesWritable, V>() {</span> |
| <span class="source-line-no">242</span><span id="line-242"> // Map of families to writers and how much has been output on the writer.</span> |
| <span class="source-line-no">243</span><span id="line-243"> private final Map<byte[], WriterLength> writers = new TreeMap<>(Bytes.BYTES_COMPARATOR);</span> |
| <span class="source-line-no">244</span><span id="line-244"> private final Map<byte[], byte[]> previousRows = new TreeMap<>(Bytes.BYTES_COMPARATOR);</span> |
| <span class="source-line-no">245</span><span id="line-245"> private final long now = EnvironmentEdgeManager.currentTime();</span> |
| <span class="source-line-no">246</span><span id="line-246"> private byte[] tableNameBytes = writeMultipleTables ? null : Bytes.toBytes(writeTableNames);</span> |
| <span class="source-line-no">247</span><span id="line-247"></span> |
| <span class="source-line-no">248</span><span id="line-248"> @Override</span> |
| <span class="source-line-no">249</span><span id="line-249"> public void write(ImmutableBytesWritable row, V cell) throws IOException {</span> |
| <span class="source-line-no">250</span><span id="line-250"> // null input == user explicitly wants to flush</span> |
| <span class="source-line-no">251</span><span id="line-251"> if (row == null && cell == null) {</span> |
| <span class="source-line-no">252</span><span id="line-252"> rollWriters(null);</span> |
| <span class="source-line-no">253</span><span id="line-253"> return;</span> |
| <span class="source-line-no">254</span><span id="line-254"> }</span> |
| <span class="source-line-no">255</span><span id="line-255"></span> |
| <span class="source-line-no">256</span><span id="line-256"> ExtendedCell kv = PrivateCellUtil.ensureExtendedCell(cell);</span> |
| <span class="source-line-no">257</span><span id="line-257"> byte[] rowKey = CellUtil.cloneRow(kv);</span> |
| <span class="source-line-no">258</span><span id="line-258"> int length = (PrivateCellUtil.estimatedSerializedSizeOf(kv)) - Bytes.SIZEOF_INT;</span> |
| <span class="source-line-no">259</span><span id="line-259"> byte[] family = CellUtil.cloneFamily(kv);</span> |
| <span class="source-line-no">260</span><span id="line-260"> if (writeMultipleTables) {</span> |
| <span class="source-line-no">261</span><span id="line-261"> tableNameBytes = MultiTableHFileOutputFormat.getTableName(row.get());</span> |
| <span class="source-line-no">262</span><span id="line-262"> tableNameBytes = TableName.valueOf(tableNameBytes).getNameWithNamespaceInclAsString()</span> |
| <span class="source-line-no">263</span><span id="line-263"> .getBytes(Charset.defaultCharset());</span> |
| <span class="source-line-no">264</span><span id="line-264"> if (!allTableNames.contains(Bytes.toString(tableNameBytes))) {</span> |
| <span class="source-line-no">265</span><span id="line-265"> throw new IllegalArgumentException(</span> |
| <span class="source-line-no">266</span><span id="line-266"> "TableName " + Bytes.toString(tableNameBytes) + " not expected");</span> |
| <span class="source-line-no">267</span><span id="line-267"> }</span> |
| <span class="source-line-no">268</span><span id="line-268"> }</span> |
| <span class="source-line-no">269</span><span id="line-269"> byte[] tableAndFamily = getTableNameSuffixedWithFamily(tableNameBytes, family);</span> |
| <span class="source-line-no">270</span><span id="line-270"></span> |
| <span class="source-line-no">271</span><span id="line-271"> WriterLength wl = this.writers.get(tableAndFamily);</span> |
| <span class="source-line-no">272</span><span id="line-272"></span> |
| <span class="source-line-no">273</span><span id="line-273"> // If this is a new column family, verify that the directory exists</span> |
| <span class="source-line-no">274</span><span id="line-274"> if (wl == null) {</span> |
| <span class="source-line-no">275</span><span id="line-275"> Path writerPath = null;</span> |
| <span class="source-line-no">276</span><span id="line-276"> if (writeMultipleTables) {</span> |
| <span class="source-line-no">277</span><span id="line-277"> Path tableRelPath = getTableRelativePath(tableNameBytes);</span> |
| <span class="source-line-no">278</span><span id="line-278"> writerPath = new Path(outputDir, new Path(tableRelPath, Bytes.toString(family)));</span> |
| <span class="source-line-no">279</span><span id="line-279"> } else {</span> |
| <span class="source-line-no">280</span><span id="line-280"> writerPath = new Path(outputDir, Bytes.toString(family));</span> |
| <span class="source-line-no">281</span><span id="line-281"> }</span> |
| <span class="source-line-no">282</span><span id="line-282"> fs.mkdirs(writerPath);</span> |
| <span class="source-line-no">283</span><span id="line-283"> configureStoragePolicy(conf, fs, tableAndFamily, writerPath);</span> |
| <span class="source-line-no">284</span><span id="line-284"> }</span> |
| <span class="source-line-no">285</span><span id="line-285"></span> |
| <span class="source-line-no">286</span><span id="line-286"> // This can only happen once a row is finished though</span> |
| <span class="source-line-no">287</span><span id="line-287"> if (</span> |
| <span class="source-line-no">288</span><span id="line-288"> wl != null && wl.written + length >= maxsize</span> |
| <span class="source-line-no">289</span><span id="line-289"> && Bytes.compareTo(this.previousRows.get(family), rowKey) != 0</span> |
| <span class="source-line-no">290</span><span id="line-290"> ) {</span> |
| <span class="source-line-no">291</span><span id="line-291"> rollWriters(wl);</span> |
| <span class="source-line-no">292</span><span id="line-292"> }</span> |
| <span class="source-line-no">293</span><span id="line-293"></span> |
| <span class="source-line-no">294</span><span id="line-294"> // create a new WAL writer, if necessary</span> |
| <span class="source-line-no">295</span><span id="line-295"> if (wl == null || wl.writer == null) {</span> |
| <span class="source-line-no">296</span><span id="line-296"> InetSocketAddress[] favoredNodes = null;</span> |
| <span class="source-line-no">297</span><span id="line-297"> if (conf.getBoolean(LOCALITY_SENSITIVE_CONF_KEY, DEFAULT_LOCALITY_SENSITIVE)) {</span> |
| <span class="source-line-no">298</span><span id="line-298"> HRegionLocation loc = null;</span> |
| <span class="source-line-no">299</span><span id="line-299"> String tableName = Bytes.toString(tableNameBytes);</span> |
| <span class="source-line-no">300</span><span id="line-300"> if (tableName != null) {</span> |
| <span class="source-line-no">301</span><span id="line-301"> try (</span> |
| <span class="source-line-no">302</span><span id="line-302"> Connection connection =</span> |
| <span class="source-line-no">303</span><span id="line-303"> ConnectionFactory.createConnection(createRemoteClusterConf(conf));</span> |
| <span class="source-line-no">304</span><span id="line-304"> RegionLocator locator = connection.getRegionLocator(TableName.valueOf(tableName))) {</span> |
| <span class="source-line-no">305</span><span id="line-305"> loc = locator.getRegionLocation(rowKey);</span> |
| <span class="source-line-no">306</span><span id="line-306"> } catch (Throwable e) {</span> |
| <span class="source-line-no">307</span><span id="line-307"> LOG.warn("Something wrong locating rowkey {} in {}", Bytes.toString(rowKey),</span> |
| <span class="source-line-no">308</span><span id="line-308"> tableName, e);</span> |
| <span class="source-line-no">309</span><span id="line-309"> loc = null;</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"> if (null == loc) {</span> |
| <span class="source-line-no">313</span><span id="line-313"> LOG.trace("Failed get of location, use default writer {}", Bytes.toString(rowKey));</span> |
| <span class="source-line-no">314</span><span id="line-314"> } else {</span> |
| <span class="source-line-no">315</span><span id="line-315"> LOG.debug("First rowkey: [{}]", Bytes.toString(rowKey));</span> |
| <span class="source-line-no">316</span><span id="line-316"> InetSocketAddress initialIsa =</span> |
| <span class="source-line-no">317</span><span id="line-317"> new InetSocketAddress(loc.getHostname(), loc.getPort());</span> |
| <span class="source-line-no">318</span><span id="line-318"> if (initialIsa.isUnresolved()) {</span> |
| <span class="source-line-no">319</span><span id="line-319"> LOG.trace("Failed resolve address {}, use default writer", loc.getHostnamePort());</span> |
| <span class="source-line-no">320</span><span id="line-320"> } else {</span> |
| <span class="source-line-no">321</span><span id="line-321"> LOG.debug("Use favored nodes writer: {}", initialIsa.getHostString());</span> |
| <span class="source-line-no">322</span><span id="line-322"> favoredNodes = new InetSocketAddress[] { initialIsa };</span> |
| <span class="source-line-no">323</span><span id="line-323"> }</span> |
| <span class="source-line-no">324</span><span id="line-324"> }</span> |
| <span class="source-line-no">325</span><span id="line-325"> }</span> |
| <span class="source-line-no">326</span><span id="line-326"> wl = getNewWriter(tableNameBytes, family, conf, favoredNodes);</span> |
| <span class="source-line-no">327</span><span id="line-327"></span> |
| <span class="source-line-no">328</span><span id="line-328"> }</span> |
| <span class="source-line-no">329</span><span id="line-329"></span> |
| <span class="source-line-no">330</span><span id="line-330"> // we now have the proper WAL writer. full steam ahead</span> |
| <span class="source-line-no">331</span><span id="line-331"> PrivateCellUtil.updateLatestStamp(kv, this.now);</span> |
| <span class="source-line-no">332</span><span id="line-332"> wl.writer.append((ExtendedCell) kv);</span> |
| <span class="source-line-no">333</span><span id="line-333"> wl.written += length;</span> |
| <span class="source-line-no">334</span><span id="line-334"></span> |
| <span class="source-line-no">335</span><span id="line-335"> // Copy the row so we know when a row transition.</span> |
| <span class="source-line-no">336</span><span id="line-336"> this.previousRows.put(family, rowKey);</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"> private Path getTableRelativePath(byte[] tableNameBytes) {</span> |
| <span class="source-line-no">340</span><span id="line-340"> String tableName = Bytes.toString(tableNameBytes);</span> |
| <span class="source-line-no">341</span><span id="line-341"> String[] tableNameParts = tableName.split(":");</span> |
| <span class="source-line-no">342</span><span id="line-342"> Path tableRelPath = new Path(tableNameParts[0]);</span> |
| <span class="source-line-no">343</span><span id="line-343"> if (tableNameParts.length > 1) {</span> |
| <span class="source-line-no">344</span><span id="line-344"> tableRelPath = new Path(tableRelPath, tableNameParts[1]);</span> |
| <span class="source-line-no">345</span><span id="line-345"> }</span> |
| <span class="source-line-no">346</span><span id="line-346"> return tableRelPath;</span> |
| <span class="source-line-no">347</span><span id="line-347"> }</span> |
| <span class="source-line-no">348</span><span id="line-348"></span> |
| <span class="source-line-no">349</span><span id="line-349"> private void rollWriters(WriterLength writerLength) throws IOException {</span> |
| <span class="source-line-no">350</span><span id="line-350"> if (writerLength != null) {</span> |
| <span class="source-line-no">351</span><span id="line-351"> closeWriter(writerLength);</span> |
| <span class="source-line-no">352</span><span id="line-352"> } else {</span> |
| <span class="source-line-no">353</span><span id="line-353"> for (WriterLength wl : this.writers.values()) {</span> |
| <span class="source-line-no">354</span><span id="line-354"> closeWriter(wl);</span> |
| <span class="source-line-no">355</span><span id="line-355"> }</span> |
| <span class="source-line-no">356</span><span id="line-356"> }</span> |
| <span class="source-line-no">357</span><span id="line-357"> }</span> |
| <span class="source-line-no">358</span><span id="line-358"></span> |
| <span class="source-line-no">359</span><span id="line-359"> private void closeWriter(WriterLength wl) throws IOException {</span> |
| <span class="source-line-no">360</span><span id="line-360"> if (wl.writer != null) {</span> |
| <span class="source-line-no">361</span><span id="line-361"> LOG.info(</span> |
| <span class="source-line-no">362</span><span id="line-362"> "Writer=" + wl.writer.getPath() + ((wl.written == 0) ? "" : ", wrote=" + wl.written));</span> |
| <span class="source-line-no">363</span><span id="line-363"> close(wl.writer);</span> |
| <span class="source-line-no">364</span><span id="line-364"> wl.writer = null;</span> |
| <span class="source-line-no">365</span><span id="line-365"> }</span> |
| <span class="source-line-no">366</span><span id="line-366"> wl.written = 0;</span> |
| <span class="source-line-no">367</span><span id="line-367"> }</span> |
| <span class="source-line-no">368</span><span id="line-368"></span> |
| <span class="source-line-no">369</span><span id="line-369"> private Configuration createRemoteClusterConf(Configuration conf) {</span> |
| <span class="source-line-no">370</span><span id="line-370"> final Configuration newConf = new Configuration(conf);</span> |
| <span class="source-line-no">371</span><span id="line-371"></span> |
| <span class="source-line-no">372</span><span id="line-372"> final String quorum = conf.get(REMOTE_CLUSTER_ZOOKEEPER_QUORUM_CONF_KEY);</span> |
| <span class="source-line-no">373</span><span id="line-373"> final String clientPort = conf.get(REMOTE_CLUSTER_ZOOKEEPER_CLIENT_PORT_CONF_KEY);</span> |
| <span class="source-line-no">374</span><span id="line-374"> final String parent = conf.get(REMOTE_CLUSTER_ZOOKEEPER_ZNODE_PARENT_CONF_KEY);</span> |
| <span class="source-line-no">375</span><span id="line-375"></span> |
| <span class="source-line-no">376</span><span id="line-376"> if (quorum != null && clientPort != null && parent != null) {</span> |
| <span class="source-line-no">377</span><span id="line-377"> newConf.set(HConstants.ZOOKEEPER_QUORUM, quorum);</span> |
| <span class="source-line-no">378</span><span id="line-378"> newConf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, Integer.parseInt(clientPort));</span> |
| <span class="source-line-no">379</span><span id="line-379"> newConf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, parent);</span> |
| <span class="source-line-no">380</span><span id="line-380"> }</span> |
| <span class="source-line-no">381</span><span id="line-381"></span> |
| <span class="source-line-no">382</span><span id="line-382"> for (Entry<String, String> entry : conf) {</span> |
| <span class="source-line-no">383</span><span id="line-383"> String key = entry.getKey();</span> |
| <span class="source-line-no">384</span><span id="line-384"> if (</span> |
| <span class="source-line-no">385</span><span id="line-385"> REMOTE_CLUSTER_ZOOKEEPER_QUORUM_CONF_KEY.equals(key)</span> |
| <span class="source-line-no">386</span><span id="line-386"> || REMOTE_CLUSTER_ZOOKEEPER_CLIENT_PORT_CONF_KEY.equals(key)</span> |
| <span class="source-line-no">387</span><span id="line-387"> || REMOTE_CLUSTER_ZOOKEEPER_ZNODE_PARENT_CONF_KEY.equals(key)</span> |
| <span class="source-line-no">388</span><span id="line-388"> ) {</span> |
| <span class="source-line-no">389</span><span id="line-389"> // Handled them above</span> |
| <span class="source-line-no">390</span><span id="line-390"> continue;</span> |
| <span class="source-line-no">391</span><span id="line-391"> }</span> |
| <span class="source-line-no">392</span><span id="line-392"></span> |
| <span class="source-line-no">393</span><span id="line-393"> if (entry.getKey().startsWith(REMOTE_CLUSTER_CONF_PREFIX)) {</span> |
| <span class="source-line-no">394</span><span id="line-394"> String originalKey = entry.getKey().substring(REMOTE_CLUSTER_CONF_PREFIX.length());</span> |
| <span class="source-line-no">395</span><span id="line-395"> if (!originalKey.isEmpty()) {</span> |
| <span class="source-line-no">396</span><span id="line-396"> newConf.set(originalKey, entry.getValue());</span> |
| <span class="source-line-no">397</span><span id="line-397"> }</span> |
| <span class="source-line-no">398</span><span id="line-398"> }</span> |
| <span class="source-line-no">399</span><span id="line-399"> }</span> |
| <span class="source-line-no">400</span><span id="line-400"></span> |
| <span class="source-line-no">401</span><span id="line-401"> return newConf;</span> |
| <span class="source-line-no">402</span><span id="line-402"> }</span> |
| <span class="source-line-no">403</span><span id="line-403"></span> |
| <span class="source-line-no">404</span><span id="line-404"> /*</span> |
| <span class="source-line-no">405</span><span id="line-405"> * Create a new StoreFile.Writer.</span> |
| <span class="source-line-no">406</span><span id="line-406"> * @return A WriterLength, containing a new StoreFile.Writer.</span> |
| <span class="source-line-no">407</span><span id="line-407"> */</span> |
| <span class="source-line-no">408</span><span id="line-408"> @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "BX_UNBOXING_IMMEDIATELY_REBOXED",</span> |
| <span class="source-line-no">409</span><span id="line-409"> justification = "Not important")</span> |
| <span class="source-line-no">410</span><span id="line-410"> private WriterLength getNewWriter(byte[] tableName, byte[] family, Configuration conf,</span> |
| <span class="source-line-no">411</span><span id="line-411"> InetSocketAddress[] favoredNodes) throws IOException {</span> |
| <span class="source-line-no">412</span><span id="line-412"> byte[] tableAndFamily = getTableNameSuffixedWithFamily(tableName, family);</span> |
| <span class="source-line-no">413</span><span id="line-413"> Path familydir = new Path(outputDir, Bytes.toString(family));</span> |
| <span class="source-line-no">414</span><span id="line-414"> if (writeMultipleTables) {</span> |
| <span class="source-line-no">415</span><span id="line-415"> familydir =</span> |
| <span class="source-line-no">416</span><span id="line-416"> new Path(outputDir, new Path(getTableRelativePath(tableName), Bytes.toString(family)));</span> |
| <span class="source-line-no">417</span><span id="line-417"> }</span> |
| <span class="source-line-no">418</span><span id="line-418"> WriterLength wl = new WriterLength();</span> |
| <span class="source-line-no">419</span><span id="line-419"> Algorithm compression = overriddenCompression;</span> |
| <span class="source-line-no">420</span><span id="line-420"> compression = compression == null ? compressionMap.get(tableAndFamily) : compression;</span> |
| <span class="source-line-no">421</span><span id="line-421"> compression = compression == null ? defaultCompression : compression;</span> |
| <span class="source-line-no">422</span><span id="line-422"> BloomType bloomType = bloomTypeMap.get(tableAndFamily);</span> |
| <span class="source-line-no">423</span><span id="line-423"> bloomType = bloomType == null ? BloomType.NONE : bloomType;</span> |
| <span class="source-line-no">424</span><span id="line-424"> String bloomParam = bloomParamMap.get(tableAndFamily);</span> |
| <span class="source-line-no">425</span><span id="line-425"> if (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH) {</span> |
| <span class="source-line-no">426</span><span id="line-426"> conf.set(BloomFilterUtil.PREFIX_LENGTH_KEY, bloomParam);</span> |
| <span class="source-line-no">427</span><span id="line-427"> }</span> |
| <span class="source-line-no">428</span><span id="line-428"> Integer blockSize = blockSizeMap.get(tableAndFamily);</span> |
| <span class="source-line-no">429</span><span id="line-429"> blockSize = blockSize == null ? HConstants.DEFAULT_BLOCKSIZE : blockSize;</span> |
| <span class="source-line-no">430</span><span id="line-430"> DataBlockEncoding encoding = overriddenEncoding;</span> |
| <span class="source-line-no">431</span><span id="line-431"> encoding = encoding == null ? datablockEncodingMap.get(tableAndFamily) : encoding;</span> |
| <span class="source-line-no">432</span><span id="line-432"> encoding = encoding == null ? DataBlockEncoding.NONE : encoding;</span> |
| <span class="source-line-no">433</span><span id="line-433"> HFileContextBuilder contextBuilder = new HFileContextBuilder().withCompression(compression)</span> |
| <span class="source-line-no">434</span><span id="line-434"> .withDataBlockEncoding(encoding).withChecksumType(StoreUtils.getChecksumType(conf))</span> |
| <span class="source-line-no">435</span><span id="line-435"> .withBytesPerCheckSum(StoreUtils.getBytesPerChecksum(conf)).withBlockSize(blockSize)</span> |
| <span class="source-line-no">436</span><span id="line-436"> .withColumnFamily(family).withTableName(tableName)</span> |
| <span class="source-line-no">437</span><span id="line-437"> .withCreateTime(EnvironmentEdgeManager.currentTime());</span> |
| <span class="source-line-no">438</span><span id="line-438"></span> |
| <span class="source-line-no">439</span><span id="line-439"> if (HFile.getFormatVersion(conf) >= HFile.MIN_FORMAT_VERSION_WITH_TAGS) {</span> |
| <span class="source-line-no">440</span><span id="line-440"> contextBuilder.withIncludesTags(true);</span> |
| <span class="source-line-no">441</span><span id="line-441"> }</span> |
| <span class="source-line-no">442</span><span id="line-442"></span> |
| <span class="source-line-no">443</span><span id="line-443"> HFileContext hFileContext = contextBuilder.build();</span> |
| <span class="source-line-no">444</span><span id="line-444"> if (null == favoredNodes) {</span> |
| <span class="source-line-no">445</span><span id="line-445"> wl.writer =</span> |
| <span class="source-line-no">446</span><span id="line-446"> new StoreFileWriter.Builder(conf, CacheConfig.DISABLED, fs).withOutputDir(familydir)</span> |
| <span class="source-line-no">447</span><span id="line-447"> .withBloomType(bloomType).withFileContext(hFileContext).build();</span> |
| <span class="source-line-no">448</span><span id="line-448"> } else {</span> |
| <span class="source-line-no">449</span><span id="line-449"> wl.writer = new StoreFileWriter.Builder(conf, CacheConfig.DISABLED, new HFileSystem(fs))</span> |
| <span class="source-line-no">450</span><span id="line-450"> .withOutputDir(familydir).withBloomType(bloomType).withFileContext(hFileContext)</span> |
| <span class="source-line-no">451</span><span id="line-451"> .withFavoredNodes(favoredNodes).build();</span> |
| <span class="source-line-no">452</span><span id="line-452"> }</span> |
| <span class="source-line-no">453</span><span id="line-453"></span> |
| <span class="source-line-no">454</span><span id="line-454"> this.writers.put(tableAndFamily, wl);</span> |
| <span class="source-line-no">455</span><span id="line-455"> return wl;</span> |
| <span class="source-line-no">456</span><span id="line-456"> }</span> |
| <span class="source-line-no">457</span><span id="line-457"></span> |
| <span class="source-line-no">458</span><span id="line-458"> private void close(final StoreFileWriter w) throws IOException {</span> |
| <span class="source-line-no">459</span><span id="line-459"> if (w != null) {</span> |
| <span class="source-line-no">460</span><span id="line-460"> w.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(EnvironmentEdgeManager.currentTime()));</span> |
| <span class="source-line-no">461</span><span id="line-461"> w.appendFileInfo(BULKLOAD_TASK_KEY, Bytes.toBytes(context.getTaskAttemptID().toString()));</span> |
| <span class="source-line-no">462</span><span id="line-462"> w.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(true));</span> |
| <span class="source-line-no">463</span><span id="line-463"> w.appendFileInfo(EXCLUDE_FROM_MINOR_COMPACTION_KEY, Bytes.toBytes(compactionExclude));</span> |
| <span class="source-line-no">464</span><span id="line-464"> w.appendTrackedTimestampsToMetadata();</span> |
| <span class="source-line-no">465</span><span id="line-465"> w.close();</span> |
| <span class="source-line-no">466</span><span id="line-466"> }</span> |
| <span class="source-line-no">467</span><span id="line-467"> }</span> |
| <span class="source-line-no">468</span><span id="line-468"></span> |
| <span class="source-line-no">469</span><span id="line-469"> @Override</span> |
| <span class="source-line-no">470</span><span id="line-470"> public void close(TaskAttemptContext c) throws IOException, InterruptedException {</span> |
| <span class="source-line-no">471</span><span id="line-471"> for (WriterLength wl : this.writers.values()) {</span> |
| <span class="source-line-no">472</span><span id="line-472"> close(wl.writer);</span> |
| <span class="source-line-no">473</span><span id="line-473"> }</span> |
| <span class="source-line-no">474</span><span id="line-474"> }</span> |
| <span class="source-line-no">475</span><span id="line-475"> };</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"> * Configure block storage policy for CF after the directory is created.</span> |
| <span class="source-line-no">480</span><span id="line-480"> */</span> |
| <span class="source-line-no">481</span><span id="line-481"> static void configureStoragePolicy(final Configuration conf, final FileSystem fs,</span> |
| <span class="source-line-no">482</span><span id="line-482"> byte[] tableAndFamily, Path cfPath) {</span> |
| <span class="source-line-no">483</span><span id="line-483"> if (null == conf || null == fs || null == tableAndFamily || null == cfPath) {</span> |
| <span class="source-line-no">484</span><span id="line-484"> return;</span> |
| <span class="source-line-no">485</span><span id="line-485"> }</span> |
| <span class="source-line-no">486</span><span id="line-486"></span> |
| <span class="source-line-no">487</span><span id="line-487"> String policy = conf.get(STORAGE_POLICY_PROPERTY_CF_PREFIX + Bytes.toString(tableAndFamily),</span> |
| <span class="source-line-no">488</span><span id="line-488"> conf.get(STORAGE_POLICY_PROPERTY));</span> |
| <span class="source-line-no">489</span><span id="line-489"> CommonFSUtils.setStoragePolicy(fs, cfPath, policy);</span> |
| <span class="source-line-no">490</span><span id="line-490"> }</span> |
| <span class="source-line-no">491</span><span id="line-491"></span> |
| <span class="source-line-no">492</span><span id="line-492"> /*</span> |
| <span class="source-line-no">493</span><span id="line-493"> * Data structure to hold a Writer and amount of data written on it.</span> |
| <span class="source-line-no">494</span><span id="line-494"> */</span> |
| <span class="source-line-no">495</span><span id="line-495"> static class WriterLength {</span> |
| <span class="source-line-no">496</span><span id="line-496"> long written = 0;</span> |
| <span class="source-line-no">497</span><span id="line-497"> StoreFileWriter writer = null;</span> |
| <span class="source-line-no">498</span><span id="line-498"> }</span> |
| <span class="source-line-no">499</span><span id="line-499"></span> |
| <span class="source-line-no">500</span><span id="line-500"> /**</span> |
| <span class="source-line-no">501</span><span id="line-501"> * Return the start keys of all of the regions in this table, as a list of ImmutableBytesWritable.</span> |
| <span class="source-line-no">502</span><span id="line-502"> */</span> |
| <span class="source-line-no">503</span><span id="line-503"> private static List<ImmutableBytesWritable> getRegionStartKeys(List<RegionLocator> regionLocators,</span> |
| <span class="source-line-no">504</span><span id="line-504"> boolean writeMultipleTables) throws IOException {</span> |
| <span class="source-line-no">505</span><span id="line-505"></span> |
| <span class="source-line-no">506</span><span id="line-506"> ArrayList<ImmutableBytesWritable> ret = new ArrayList<>();</span> |
| <span class="source-line-no">507</span><span id="line-507"> for (RegionLocator regionLocator : regionLocators) {</span> |
| <span class="source-line-no">508</span><span id="line-508"> TableName tableName = regionLocator.getName();</span> |
| <span class="source-line-no">509</span><span id="line-509"> LOG.info("Looking up current regions for table " + tableName);</span> |
| <span class="source-line-no">510</span><span id="line-510"> byte[][] byteKeys = regionLocator.getStartKeys();</span> |
| <span class="source-line-no">511</span><span id="line-511"> for (byte[] byteKey : byteKeys) {</span> |
| <span class="source-line-no">512</span><span id="line-512"> byte[] fullKey = byteKey; // HFileOutputFormat2 use case</span> |
| <span class="source-line-no">513</span><span id="line-513"> if (writeMultipleTables) {</span> |
| <span class="source-line-no">514</span><span id="line-514"> // MultiTableHFileOutputFormat use case</span> |
| <span class="source-line-no">515</span><span id="line-515"> fullKey = combineTableNameSuffix(tableName.getName(), byteKey);</span> |
| <span class="source-line-no">516</span><span id="line-516"> }</span> |
| <span class="source-line-no">517</span><span id="line-517"> if (LOG.isDebugEnabled()) {</span> |
| <span class="source-line-no">518</span><span id="line-518"> LOG.debug("SplitPoint startkey for " + tableName + ": " + Bytes.toStringBinary(fullKey));</span> |
| <span class="source-line-no">519</span><span id="line-519"> }</span> |
| <span class="source-line-no">520</span><span id="line-520"> ret.add(new ImmutableBytesWritable(fullKey));</span> |
| <span class="source-line-no">521</span><span id="line-521"> }</span> |
| <span class="source-line-no">522</span><span id="line-522"> }</span> |
| <span class="source-line-no">523</span><span id="line-523"> return ret;</span> |
| <span class="source-line-no">524</span><span id="line-524"> }</span> |
| <span class="source-line-no">525</span><span id="line-525"></span> |
| <span class="source-line-no">526</span><span id="line-526"> /**</span> |
| <span class="source-line-no">527</span><span id="line-527"> * Write out a {@link SequenceFile} that can be read by {@link TotalOrderPartitioner} that</span> |
| <span class="source-line-no">528</span><span id="line-528"> * contains the split points in startKeys.</span> |
| <span class="source-line-no">529</span><span id="line-529"> */</span> |
| <span class="source-line-no">530</span><span id="line-530"> @SuppressWarnings("deprecation")</span> |
| <span class="source-line-no">531</span><span id="line-531"> private static void writePartitions(Configuration conf, Path partitionsPath,</span> |
| <span class="source-line-no">532</span><span id="line-532"> List<ImmutableBytesWritable> startKeys, boolean writeMultipleTables) throws IOException {</span> |
| <span class="source-line-no">533</span><span id="line-533"> LOG.info("Writing partition information to " + partitionsPath);</span> |
| <span class="source-line-no">534</span><span id="line-534"> if (startKeys.isEmpty()) {</span> |
| <span class="source-line-no">535</span><span id="line-535"> throw new IllegalArgumentException("No regions passed");</span> |
| <span class="source-line-no">536</span><span id="line-536"> }</span> |
| <span class="source-line-no">537</span><span id="line-537"></span> |
| <span class="source-line-no">538</span><span id="line-538"> // We're generating a list of split points, and we don't ever</span> |
| <span class="source-line-no">539</span><span id="line-539"> // have keys < the first region (which has an empty start key)</span> |
| <span class="source-line-no">540</span><span id="line-540"> // so we need to remove it. Otherwise we would end up with an</span> |
| <span class="source-line-no">541</span><span id="line-541"> // empty reducer with index 0</span> |
| <span class="source-line-no">542</span><span id="line-542"> TreeSet<ImmutableBytesWritable> sorted = new TreeSet<>(startKeys);</span> |
| <span class="source-line-no">543</span><span id="line-543"> ImmutableBytesWritable first = sorted.first();</span> |
| <span class="source-line-no">544</span><span id="line-544"> if (writeMultipleTables) {</span> |
| <span class="source-line-no">545</span><span id="line-545"> first =</span> |
| <span class="source-line-no">546</span><span id="line-546"> new ImmutableBytesWritable(MultiTableHFileOutputFormat.getSuffix(sorted.first().get()));</span> |
| <span class="source-line-no">547</span><span id="line-547"> }</span> |
| <span class="source-line-no">548</span><span id="line-548"> if (!first.equals(HConstants.EMPTY_BYTE_ARRAY)) {</span> |
| <span class="source-line-no">549</span><span id="line-549"> throw new IllegalArgumentException(</span> |
| <span class="source-line-no">550</span><span id="line-550"> "First region of table should have empty start key. Instead has: "</span> |
| <span class="source-line-no">551</span><span id="line-551"> + Bytes.toStringBinary(first.get()));</span> |
| <span class="source-line-no">552</span><span id="line-552"> }</span> |
| <span class="source-line-no">553</span><span id="line-553"> sorted.remove(sorted.first());</span> |
| <span class="source-line-no">554</span><span id="line-554"></span> |
| <span class="source-line-no">555</span><span id="line-555"> // Write the actual file</span> |
| <span class="source-line-no">556</span><span id="line-556"> FileSystem fs = partitionsPath.getFileSystem(conf);</span> |
| <span class="source-line-no">557</span><span id="line-557"> boolean diskBasedSortingEnabled = diskBasedSortingEnabled(conf);</span> |
| <span class="source-line-no">558</span><span id="line-558"> Class<? extends Writable> keyClass =</span> |
| <span class="source-line-no">559</span><span id="line-559"> diskBasedSortingEnabled ? KeyOnlyCellComparable.class : ImmutableBytesWritable.class;</span> |
| <span class="source-line-no">560</span><span id="line-560"> SequenceFile.Writer writer =</span> |
| <span class="source-line-no">561</span><span id="line-561"> SequenceFile.createWriter(fs, conf, partitionsPath, keyClass, NullWritable.class);</span> |
| <span class="source-line-no">562</span><span id="line-562"></span> |
| <span class="source-line-no">563</span><span id="line-563"> try {</span> |
| <span class="source-line-no">564</span><span id="line-564"> for (ImmutableBytesWritable startKey : sorted) {</span> |
| <span class="source-line-no">565</span><span id="line-565"> Writable writable = diskBasedSortingEnabled</span> |
| <span class="source-line-no">566</span><span id="line-566"> ? new KeyOnlyCellComparable(KeyValueUtil.createFirstOnRow(startKey.get()))</span> |
| <span class="source-line-no">567</span><span id="line-567"> : startKey;</span> |
| <span class="source-line-no">568</span><span id="line-568"></span> |
| <span class="source-line-no">569</span><span id="line-569"> writer.append(writable, NullWritable.get());</span> |
| <span class="source-line-no">570</span><span id="line-570"> }</span> |
| <span class="source-line-no">571</span><span id="line-571"> } finally {</span> |
| <span class="source-line-no">572</span><span id="line-572"> writer.close();</span> |
| <span class="source-line-no">573</span><span id="line-573"> }</span> |
| <span class="source-line-no">574</span><span id="line-574"> }</span> |
| <span class="source-line-no">575</span><span id="line-575"></span> |
| <span class="source-line-no">576</span><span id="line-576"> /**</span> |
| <span class="source-line-no">577</span><span id="line-577"> * Configure a MapReduce Job to perform an incremental load into the given table. This</span> |
| <span class="source-line-no">578</span><span id="line-578"> * <ul></span> |
| <span class="source-line-no">579</span><span id="line-579"> * <li>Inspects the table to configure a total order partitioner</li></span> |
| <span class="source-line-no">580</span><span id="line-580"> * <li>Uploads the partitions file to the cluster and adds it to the DistributedCache</li></span> |
| <span class="source-line-no">581</span><span id="line-581"> * <li>Sets the number of reduce tasks to match the current number of regions</li></span> |
| <span class="source-line-no">582</span><span id="line-582"> * <li>Sets the output key/value class to match HFileOutputFormat2's requirements</li></span> |
| <span class="source-line-no">583</span><span id="line-583"> * <li>Sets the reducer up to perform the appropriate sorting (either KeyValueSortReducer or</span> |
| <span class="source-line-no">584</span><span id="line-584"> * PutSortReducer)</li></span> |
| <span class="source-line-no">585</span><span id="line-585"> * <li>Sets the HBase cluster key to load region locations for locality-sensitive</li></span> |
| <span class="source-line-no">586</span><span id="line-586"> * </ul></span> |
| <span class="source-line-no">587</span><span id="line-587"> * The user should be sure to set the map output value class to either KeyValue or Put before</span> |
| <span class="source-line-no">588</span><span id="line-588"> * running this function.</span> |
| <span class="source-line-no">589</span><span id="line-589"> */</span> |
| <span class="source-line-no">590</span><span id="line-590"> public static void configureIncrementalLoad(Job job, Table table, RegionLocator regionLocator)</span> |
| <span class="source-line-no">591</span><span id="line-591"> throws IOException {</span> |
| <span class="source-line-no">592</span><span id="line-592"> configureIncrementalLoad(job, table.getDescriptor(), regionLocator);</span> |
| <span class="source-line-no">593</span><span id="line-593"> configureRemoteCluster(job, table.getConfiguration());</span> |
| <span class="source-line-no">594</span><span id="line-594"> }</span> |
| <span class="source-line-no">595</span><span id="line-595"></span> |
| <span class="source-line-no">596</span><span id="line-596"> /**</span> |
| <span class="source-line-no">597</span><span id="line-597"> * Configure a MapReduce Job to perform an incremental load into the given table. This</span> |
| <span class="source-line-no">598</span><span id="line-598"> * <ul></span> |
| <span class="source-line-no">599</span><span id="line-599"> * <li>Inspects the table to configure a total order partitioner</li></span> |
| <span class="source-line-no">600</span><span id="line-600"> * <li>Uploads the partitions file to the cluster and adds it to the DistributedCache</li></span> |
| <span class="source-line-no">601</span><span id="line-601"> * <li>Sets the number of reduce tasks to match the current number of regions</li></span> |
| <span class="source-line-no">602</span><span id="line-602"> * <li>Sets the output key/value class to match HFileOutputFormat2's requirements</li></span> |
| <span class="source-line-no">603</span><span id="line-603"> * <li>Sets the reducer up to perform the appropriate sorting (either KeyValueSortReducer or</span> |
| <span class="source-line-no">604</span><span id="line-604"> * PutSortReducer)</li></span> |
| <span class="source-line-no">605</span><span id="line-605"> * </ul></span> |
| <span class="source-line-no">606</span><span id="line-606"> * The user should be sure to set the map output value class to either KeyValue or Put before</span> |
| <span class="source-line-no">607</span><span id="line-607"> * running this function.</span> |
| <span class="source-line-no">608</span><span id="line-608"> */</span> |
| <span class="source-line-no">609</span><span id="line-609"> public static void configureIncrementalLoad(Job job, TableDescriptor tableDescriptor,</span> |
| <span class="source-line-no">610</span><span id="line-610"> RegionLocator regionLocator) throws IOException {</span> |
| <span class="source-line-no">611</span><span id="line-611"> ArrayList<TableInfo> singleTableInfo = new ArrayList<>();</span> |
| <span class="source-line-no">612</span><span id="line-612"> singleTableInfo.add(new TableInfo(tableDescriptor, regionLocator));</span> |
| <span class="source-line-no">613</span><span id="line-613"> configureIncrementalLoad(job, singleTableInfo, HFileOutputFormat2.class);</span> |
| <span class="source-line-no">614</span><span id="line-614"> }</span> |
| <span class="source-line-no">615</span><span id="line-615"></span> |
| <span class="source-line-no">616</span><span id="line-616"> public static boolean diskBasedSortingEnabled(Configuration conf) {</span> |
| <span class="source-line-no">617</span><span id="line-617"> return conf.getBoolean(DISK_BASED_SORTING_ENABLED_KEY, DISK_BASED_SORTING_ENABLED_DEFAULT);</span> |
| <span class="source-line-no">618</span><span id="line-618"> }</span> |
| <span class="source-line-no">619</span><span id="line-619"></span> |
| <span class="source-line-no">620</span><span id="line-620"> static void configureIncrementalLoad(Job job, List<TableInfo> multiTableInfo,</span> |
| <span class="source-line-no">621</span><span id="line-621"> Class<? extends OutputFormat<?, ?>> cls) throws IOException {</span> |
| <span class="source-line-no">622</span><span id="line-622"> Configuration conf = job.getConfiguration();</span> |
| <span class="source-line-no">623</span><span id="line-623"> job.setOutputKeyClass(ImmutableBytesWritable.class);</span> |
| <span class="source-line-no">624</span><span id="line-624"> job.setOutputValueClass(MapReduceExtendedCell.class);</span> |
| <span class="source-line-no">625</span><span id="line-625"> job.setOutputFormatClass(cls);</span> |
| <span class="source-line-no">626</span><span id="line-626"></span> |
| <span class="source-line-no">627</span><span id="line-627"> if (multiTableInfo.stream().distinct().count() != multiTableInfo.size()) {</span> |
| <span class="source-line-no">628</span><span id="line-628"> throw new IllegalArgumentException("Duplicate entries found in TableInfo argument");</span> |
| <span class="source-line-no">629</span><span id="line-629"> }</span> |
| <span class="source-line-no">630</span><span id="line-630"> boolean writeMultipleTables = false;</span> |
| <span class="source-line-no">631</span><span id="line-631"> if (MultiTableHFileOutputFormat.class.equals(cls)) {</span> |
| <span class="source-line-no">632</span><span id="line-632"> writeMultipleTables = true;</span> |
| <span class="source-line-no">633</span><span id="line-633"> conf.setBoolean(MULTI_TABLE_HFILEOUTPUTFORMAT_CONF_KEY, true);</span> |
| <span class="source-line-no">634</span><span id="line-634"> }</span> |
| <span class="source-line-no">635</span><span id="line-635"> // Based on the configured map output class, set the correct reducer to properly</span> |
| <span class="source-line-no">636</span><span id="line-636"> // sort the incoming values.</span> |
| <span class="source-line-no">637</span><span id="line-637"> // TODO it would be nice to pick one or the other of these formats.</span> |
| <span class="source-line-no">638</span><span id="line-638"> boolean diskBasedSorting = diskBasedSortingEnabled(conf);</span> |
| <span class="source-line-no">639</span><span id="line-639"></span> |
| <span class="source-line-no">640</span><span id="line-640"> if (diskBasedSorting) {</span> |
| <span class="source-line-no">641</span><span id="line-641"> job.setMapOutputKeyClass(KeyOnlyCellComparable.class);</span> |
| <span class="source-line-no">642</span><span id="line-642"> job.setSortComparatorClass(KeyOnlyCellComparable.KeyOnlyCellComparator.class);</span> |
| <span class="source-line-no">643</span><span id="line-643"> job.setReducerClass(PreSortedCellsReducer.class);</span> |
| <span class="source-line-no">644</span><span id="line-644"> } else if (</span> |
| <span class="source-line-no">645</span><span id="line-645"> KeyValue.class.equals(job.getMapOutputValueClass())</span> |
| <span class="source-line-no">646</span><span id="line-646"> || MapReduceExtendedCell.class.equals(job.getMapOutputValueClass())</span> |
| <span class="source-line-no">647</span><span id="line-647"> ) {</span> |
| <span class="source-line-no">648</span><span id="line-648"> job.setReducerClass(CellSortReducer.class);</span> |
| <span class="source-line-no">649</span><span id="line-649"> } else if (Put.class.equals(job.getMapOutputValueClass())) {</span> |
| <span class="source-line-no">650</span><span id="line-650"> job.setReducerClass(PutSortReducer.class);</span> |
| <span class="source-line-no">651</span><span id="line-651"> } else if (Text.class.equals(job.getMapOutputValueClass())) {</span> |
| <span class="source-line-no">652</span><span id="line-652"> job.setReducerClass(TextSortReducer.class);</span> |
| <span class="source-line-no">653</span><span id="line-653"> } else {</span> |
| <span class="source-line-no">654</span><span id="line-654"> LOG.warn("Unknown map output value type:" + job.getMapOutputValueClass());</span> |
| <span class="source-line-no">655</span><span id="line-655"> }</span> |
| <span class="source-line-no">656</span><span id="line-656"></span> |
| <span class="source-line-no">657</span><span id="line-657"> mergeSerializations(conf);</span> |
| <span class="source-line-no">658</span><span id="line-658"></span> |
| <span class="source-line-no">659</span><span id="line-659"> if (conf.getBoolean(LOCALITY_SENSITIVE_CONF_KEY, DEFAULT_LOCALITY_SENSITIVE)) {</span> |
| <span class="source-line-no">660</span><span id="line-660"> LOG.info("bulkload locality sensitive enabled");</span> |
| <span class="source-line-no">661</span><span id="line-661"> }</span> |
| <span class="source-line-no">662</span><span id="line-662"></span> |
| <span class="source-line-no">663</span><span id="line-663"> /* Now get the region start keys for every table required */</span> |
| <span class="source-line-no">664</span><span id="line-664"> List<String> allTableNames = new ArrayList<>(multiTableInfo.size());</span> |
| <span class="source-line-no">665</span><span id="line-665"> List<RegionLocator> regionLocators = new ArrayList<>(multiTableInfo.size());</span> |
| <span class="source-line-no">666</span><span id="line-666"> List<TableDescriptor> tableDescriptors = new ArrayList<>(multiTableInfo.size());</span> |
| <span class="source-line-no">667</span><span id="line-667"></span> |
| <span class="source-line-no">668</span><span id="line-668"> for (TableInfo tableInfo : multiTableInfo) {</span> |
| <span class="source-line-no">669</span><span id="line-669"> regionLocators.add(tableInfo.getRegionLocator());</span> |
| <span class="source-line-no">670</span><span id="line-670"> String tn = writeMultipleTables</span> |
| <span class="source-line-no">671</span><span id="line-671"> ? tableInfo.getRegionLocator().getName().getNameWithNamespaceInclAsString()</span> |
| <span class="source-line-no">672</span><span id="line-672"> : tableInfo.getRegionLocator().getName().getNameAsString();</span> |
| <span class="source-line-no">673</span><span id="line-673"> allTableNames.add(tn);</span> |
| <span class="source-line-no">674</span><span id="line-674"> tableDescriptors.add(tableInfo.getTableDescriptor());</span> |
| <span class="source-line-no">675</span><span id="line-675"> }</span> |
| <span class="source-line-no">676</span><span id="line-676"> // Record tablenames for creating writer by favored nodes, and decoding compression,</span> |
| <span class="source-line-no">677</span><span id="line-677"> // block size and other attributes of columnfamily per table</span> |
| <span class="source-line-no">678</span><span id="line-678"> conf.set(OUTPUT_TABLE_NAME_CONF_KEY,</span> |
| <span class="source-line-no">679</span><span id="line-679"> StringUtils.join(allTableNames, Bytes.toString(tableSeparator)));</span> |
| <span class="source-line-no">680</span><span id="line-680"> List<ImmutableBytesWritable> startKeys =</span> |
| <span class="source-line-no">681</span><span id="line-681"> getRegionStartKeys(regionLocators, writeMultipleTables);</span> |
| <span class="source-line-no">682</span><span id="line-682"> // Use table's region boundaries for TOP split points.</span> |
| <span class="source-line-no">683</span><span id="line-683"> LOG.info("Configuring " + startKeys.size() + " reduce partitions "</span> |
| <span class="source-line-no">684</span><span id="line-684"> + "to match current region count for all tables");</span> |
| <span class="source-line-no">685</span><span id="line-685"> job.setNumReduceTasks(startKeys.size());</span> |
| <span class="source-line-no">686</span><span id="line-686"></span> |
| <span class="source-line-no">687</span><span id="line-687"> configurePartitioner(job, startKeys, writeMultipleTables);</span> |
| <span class="source-line-no">688</span><span id="line-688"> // Set compression algorithms based on column families</span> |
| <span class="source-line-no">689</span><span id="line-689"></span> |
| <span class="source-line-no">690</span><span id="line-690"> conf.set(COMPRESSION_FAMILIES_CONF_KEY,</span> |
| <span class="source-line-no">691</span><span id="line-691"> serializeColumnFamilyAttribute(compressionDetails, tableDescriptors));</span> |
| <span class="source-line-no">692</span><span id="line-692"> conf.set(BLOCK_SIZE_FAMILIES_CONF_KEY,</span> |
| <span class="source-line-no">693</span><span id="line-693"> serializeColumnFamilyAttribute(blockSizeDetails, tableDescriptors));</span> |
| <span class="source-line-no">694</span><span id="line-694"> conf.set(BLOOM_TYPE_FAMILIES_CONF_KEY,</span> |
| <span class="source-line-no">695</span><span id="line-695"> serializeColumnFamilyAttribute(bloomTypeDetails, tableDescriptors));</span> |
| <span class="source-line-no">696</span><span id="line-696"> conf.set(BLOOM_PARAM_FAMILIES_CONF_KEY,</span> |
| <span class="source-line-no">697</span><span id="line-697"> serializeColumnFamilyAttribute(bloomParamDetails, tableDescriptors));</span> |
| <span class="source-line-no">698</span><span id="line-698"> conf.set(DATABLOCK_ENCODING_FAMILIES_CONF_KEY,</span> |
| <span class="source-line-no">699</span><span id="line-699"> serializeColumnFamilyAttribute(dataBlockEncodingDetails, tableDescriptors));</span> |
| <span class="source-line-no">700</span><span id="line-700"></span> |
| <span class="source-line-no">701</span><span id="line-701"> TableMapReduceUtil.addDependencyJars(job);</span> |
| <span class="source-line-no">702</span><span id="line-702"> TableMapReduceUtil.initCredentials(job);</span> |
| <span class="source-line-no">703</span><span id="line-703"> LOG.info("Incremental output configured for tables: " + StringUtils.join(allTableNames, ","));</span> |
| <span class="source-line-no">704</span><span id="line-704"> }</span> |
| <span class="source-line-no">705</span><span id="line-705"></span> |
| <span class="source-line-no">706</span><span id="line-706"> private static void mergeSerializations(Configuration conf) {</span> |
| <span class="source-line-no">707</span><span id="line-707"> List<String> serializations = new ArrayList<>();</span> |
| <span class="source-line-no">708</span><span id="line-708"></span> |
| <span class="source-line-no">709</span><span id="line-709"> // add any existing values that have been set</span> |
| <span class="source-line-no">710</span><span id="line-710"> String[] existing = conf.getStrings("io.serializations");</span> |
| <span class="source-line-no">711</span><span id="line-711"> if (existing != null) {</span> |
| <span class="source-line-no">712</span><span id="line-712"> Collections.addAll(serializations, existing);</span> |
| <span class="source-line-no">713</span><span id="line-713"> }</span> |
| <span class="source-line-no">714</span><span id="line-714"></span> |
| <span class="source-line-no">715</span><span id="line-715"> serializations.add(MutationSerialization.class.getName());</span> |
| <span class="source-line-no">716</span><span id="line-716"> serializations.add(ResultSerialization.class.getName());</span> |
| <span class="source-line-no">717</span><span id="line-717"></span> |
| <span class="source-line-no">718</span><span id="line-718"> // Add ExtendedCellSerialization, if configured. Order matters here. Hadoop's</span> |
| <span class="source-line-no">719</span><span id="line-719"> // SerializationFactory runs through serializations in the order they are registered.</span> |
| <span class="source-line-no">720</span><span id="line-720"> // We want to register ExtendedCellSerialization before CellSerialization because both</span> |
| <span class="source-line-no">721</span><span id="line-721"> // work for ExtendedCells but only ExtendedCellSerialization handles them properly.</span> |
| <span class="source-line-no">722</span><span id="line-722"> if (</span> |
| <span class="source-line-no">723</span><span id="line-723"> conf.getBoolean(EXTENDED_CELL_SERIALIZATION_ENABLED_KEY,</span> |
| <span class="source-line-no">724</span><span id="line-724"> EXTENDED_CELL_SERIALIZATION_ENABLED_DEFULT)</span> |
| <span class="source-line-no">725</span><span id="line-725"> ) {</span> |
| <span class="source-line-no">726</span><span id="line-726"> serializations.add(ExtendedCellSerialization.class.getName());</span> |
| <span class="source-line-no">727</span><span id="line-727"> }</span> |
| <span class="source-line-no">728</span><span id="line-728"> serializations.add(CellSerialization.class.getName());</span> |
| <span class="source-line-no">729</span><span id="line-729"></span> |
| <span class="source-line-no">730</span><span id="line-730"> conf.setStrings("io.serializations", serializations.toArray(new String[0]));</span> |
| <span class="source-line-no">731</span><span id="line-731"> }</span> |
| <span class="source-line-no">732</span><span id="line-732"></span> |
| <span class="source-line-no">733</span><span id="line-733"> public static void configureIncrementalLoadMap(Job job, TableDescriptor tableDescriptor)</span> |
| <span class="source-line-no">734</span><span id="line-734"> throws IOException {</span> |
| <span class="source-line-no">735</span><span id="line-735"> Configuration conf = job.getConfiguration();</span> |
| <span class="source-line-no">736</span><span id="line-736"></span> |
| <span class="source-line-no">737</span><span id="line-737"> job.setOutputKeyClass(ImmutableBytesWritable.class);</span> |
| <span class="source-line-no">738</span><span id="line-738"> job.setOutputValueClass(MapReduceExtendedCell.class);</span> |
| <span class="source-line-no">739</span><span id="line-739"> job.setOutputFormatClass(HFileOutputFormat2.class);</span> |
| <span class="source-line-no">740</span><span id="line-740"></span> |
| <span class="source-line-no">741</span><span id="line-741"> ArrayList<TableDescriptor> singleTableDescriptor = new ArrayList<>(1);</span> |
| <span class="source-line-no">742</span><span id="line-742"> singleTableDescriptor.add(tableDescriptor);</span> |
| <span class="source-line-no">743</span><span id="line-743"></span> |
| <span class="source-line-no">744</span><span id="line-744"> conf.set(OUTPUT_TABLE_NAME_CONF_KEY, tableDescriptor.getTableName().getNameAsString());</span> |
| <span class="source-line-no">745</span><span id="line-745"> // Set compression algorithms based on column families</span> |
| <span class="source-line-no">746</span><span id="line-746"> conf.set(COMPRESSION_FAMILIES_CONF_KEY,</span> |
| <span class="source-line-no">747</span><span id="line-747"> serializeColumnFamilyAttribute(compressionDetails, singleTableDescriptor));</span> |
| <span class="source-line-no">748</span><span id="line-748"> conf.set(BLOCK_SIZE_FAMILIES_CONF_KEY,</span> |
| <span class="source-line-no">749</span><span id="line-749"> serializeColumnFamilyAttribute(blockSizeDetails, singleTableDescriptor));</span> |
| <span class="source-line-no">750</span><span id="line-750"> conf.set(BLOOM_TYPE_FAMILIES_CONF_KEY,</span> |
| <span class="source-line-no">751</span><span id="line-751"> serializeColumnFamilyAttribute(bloomTypeDetails, singleTableDescriptor));</span> |
| <span class="source-line-no">752</span><span id="line-752"> conf.set(BLOOM_PARAM_FAMILIES_CONF_KEY,</span> |
| <span class="source-line-no">753</span><span id="line-753"> serializeColumnFamilyAttribute(bloomParamDetails, singleTableDescriptor));</span> |
| <span class="source-line-no">754</span><span id="line-754"> conf.set(DATABLOCK_ENCODING_FAMILIES_CONF_KEY,</span> |
| <span class="source-line-no">755</span><span id="line-755"> serializeColumnFamilyAttribute(dataBlockEncodingDetails, singleTableDescriptor));</span> |
| <span class="source-line-no">756</span><span id="line-756"></span> |
| <span class="source-line-no">757</span><span id="line-757"> TableMapReduceUtil.addDependencyJars(job);</span> |
| <span class="source-line-no">758</span><span id="line-758"> TableMapReduceUtil.initCredentials(job);</span> |
| <span class="source-line-no">759</span><span id="line-759"> LOG.info("Incremental table " + tableDescriptor.getTableName() + " output configured.");</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"> /**</span> |
| <span class="source-line-no">763</span><span id="line-763"> * Configure HBase cluster key for remote cluster to load region location for locality-sensitive</span> |
| <span class="source-line-no">764</span><span id="line-764"> * if it's enabled. It's not necessary to call this method explicitly when the cluster key for</span> |
| <span class="source-line-no">765</span><span id="line-765"> * HBase cluster to be used to load region location is configured in the job configuration. Call</span> |
| <span class="source-line-no">766</span><span id="line-766"> * this method when another HBase cluster key is configured in the job configuration. For example,</span> |
| <span class="source-line-no">767</span><span id="line-767"> * you should call when you load data from HBase cluster A using {@link TableInputFormat} and</span> |
| <span class="source-line-no">768</span><span id="line-768"> * generate hfiles for HBase cluster B. Otherwise, HFileOutputFormat2 fetch location from cluster</span> |
| <span class="source-line-no">769</span><span id="line-769"> * A and locality-sensitive won't working correctly.</span> |
| <span class="source-line-no">770</span><span id="line-770"> * {@link #configureIncrementalLoad(Job, Table, RegionLocator)} calls this method using</span> |
| <span class="source-line-no">771</span><span id="line-771"> * {@link Table#getConfiguration} as clusterConf. See HBASE-25608.</span> |
| <span class="source-line-no">772</span><span id="line-772"> * @param job which has configuration to be updated</span> |
| <span class="source-line-no">773</span><span id="line-773"> * @param clusterConf which contains cluster key of the HBase cluster to be locality-sensitive</span> |
| <span class="source-line-no">774</span><span id="line-774"> * @see #configureIncrementalLoad(Job, Table, RegionLocator)</span> |
| <span class="source-line-no">775</span><span id="line-775"> * @see #LOCALITY_SENSITIVE_CONF_KEY</span> |
| <span class="source-line-no">776</span><span id="line-776"> * @see #REMOTE_CLUSTER_ZOOKEEPER_QUORUM_CONF_KEY</span> |
| <span class="source-line-no">777</span><span id="line-777"> * @see #REMOTE_CLUSTER_ZOOKEEPER_CLIENT_PORT_CONF_KEY</span> |
| <span class="source-line-no">778</span><span id="line-778"> * @see #REMOTE_CLUSTER_ZOOKEEPER_ZNODE_PARENT_CONF_KEY</span> |
| <span class="source-line-no">779</span><span id="line-779"> */</span> |
| <span class="source-line-no">780</span><span id="line-780"> public static void configureRemoteCluster(Job job, Configuration clusterConf) throws IOException {</span> |
| <span class="source-line-no">781</span><span id="line-781"> Configuration conf = job.getConfiguration();</span> |
| <span class="source-line-no">782</span><span id="line-782"></span> |
| <span class="source-line-no">783</span><span id="line-783"> if (!conf.getBoolean(LOCALITY_SENSITIVE_CONF_KEY, DEFAULT_LOCALITY_SENSITIVE)) {</span> |
| <span class="source-line-no">784</span><span id="line-784"> return;</span> |
| <span class="source-line-no">785</span><span id="line-785"> }</span> |
| <span class="source-line-no">786</span><span id="line-786"></span> |
| <span class="source-line-no">787</span><span id="line-787"> final String quorum = clusterConf.get(HConstants.ZOOKEEPER_QUORUM);</span> |
| <span class="source-line-no">788</span><span id="line-788"> final int clientPort = clusterConf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT,</span> |
| <span class="source-line-no">789</span><span id="line-789"> HConstants.DEFAULT_ZOOKEEPER_CLIENT_PORT);</span> |
| <span class="source-line-no">790</span><span id="line-790"> final String parent =</span> |
| <span class="source-line-no">791</span><span id="line-791"> clusterConf.get(HConstants.ZOOKEEPER_ZNODE_PARENT, HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);</span> |
| <span class="source-line-no">792</span><span id="line-792"></span> |
| <span class="source-line-no">793</span><span id="line-793"> conf.set(REMOTE_CLUSTER_ZOOKEEPER_QUORUM_CONF_KEY, quorum);</span> |
| <span class="source-line-no">794</span><span id="line-794"> conf.setInt(REMOTE_CLUSTER_ZOOKEEPER_CLIENT_PORT_CONF_KEY, clientPort);</span> |
| <span class="source-line-no">795</span><span id="line-795"> conf.set(REMOTE_CLUSTER_ZOOKEEPER_ZNODE_PARENT_CONF_KEY, parent);</span> |
| <span class="source-line-no">796</span><span id="line-796"></span> |
| <span class="source-line-no">797</span><span id="line-797"> TableMapReduceUtil.initCredentialsForCluster(job, clusterConf);</span> |
| <span class="source-line-no">798</span><span id="line-798"></span> |
| <span class="source-line-no">799</span><span id="line-799"> LOG.info("ZK configs for remote cluster of bulkload is configured: " + quorum + ":" + clientPort</span> |
| <span class="source-line-no">800</span><span id="line-800"> + "/" + parent);</span> |
| <span class="source-line-no">801</span><span id="line-801"> }</span> |
| <span class="source-line-no">802</span><span id="line-802"></span> |
| <span class="source-line-no">803</span><span id="line-803"> /**</span> |
| <span class="source-line-no">804</span><span id="line-804"> * Runs inside the task to deserialize column family to compression algorithm map from the</span> |
| <span class="source-line-no">805</span><span id="line-805"> * configuration.</span> |
| <span class="source-line-no">806</span><span id="line-806"> * @param conf to read the serialized values from</span> |
| <span class="source-line-no">807</span><span id="line-807"> * @return a map from column family to the configured compression algorithm</span> |
| <span class="source-line-no">808</span><span id="line-808"> */</span> |
| <span class="source-line-no">809</span><span id="line-809"> @InterfaceAudience.Private</span> |
| <span class="source-line-no">810</span><span id="line-810"> static Map<byte[], Algorithm> createFamilyCompressionMap(Configuration conf) {</span> |
| <span class="source-line-no">811</span><span id="line-811"> Map<byte[], String> stringMap = createFamilyConfValueMap(conf, COMPRESSION_FAMILIES_CONF_KEY);</span> |
| <span class="source-line-no">812</span><span id="line-812"> Map<byte[], Algorithm> compressionMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);</span> |
| <span class="source-line-no">813</span><span id="line-813"> for (Map.Entry<byte[], String> e : stringMap.entrySet()) {</span> |
| <span class="source-line-no">814</span><span id="line-814"> Algorithm algorithm = HFileWriterImpl.compressionByName(e.getValue());</span> |
| <span class="source-line-no">815</span><span id="line-815"> compressionMap.put(e.getKey(), algorithm);</span> |
| <span class="source-line-no">816</span><span id="line-816"> }</span> |
| <span class="source-line-no">817</span><span id="line-817"> return compressionMap;</span> |
| <span class="source-line-no">818</span><span id="line-818"> }</span> |
| <span class="source-line-no">819</span><span id="line-819"></span> |
| <span class="source-line-no">820</span><span id="line-820"> /**</span> |
| <span class="source-line-no">821</span><span id="line-821"> * Runs inside the task to deserialize column family to bloom filter type map from the</span> |
| <span class="source-line-no">822</span><span id="line-822"> * configuration.</span> |
| <span class="source-line-no">823</span><span id="line-823"> * @param conf to read the serialized values from</span> |
| <span class="source-line-no">824</span><span id="line-824"> * @return a map from column family to the the configured bloom filter type</span> |
| <span class="source-line-no">825</span><span id="line-825"> */</span> |
| <span class="source-line-no">826</span><span id="line-826"> @InterfaceAudience.Private</span> |
| <span class="source-line-no">827</span><span id="line-827"> static Map<byte[], BloomType> createFamilyBloomTypeMap(Configuration conf) {</span> |
| <span class="source-line-no">828</span><span id="line-828"> Map<byte[], String> stringMap = createFamilyConfValueMap(conf, BLOOM_TYPE_FAMILIES_CONF_KEY);</span> |
| <span class="source-line-no">829</span><span id="line-829"> Map<byte[], BloomType> bloomTypeMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);</span> |
| <span class="source-line-no">830</span><span id="line-830"> for (Map.Entry<byte[], String> e : stringMap.entrySet()) {</span> |
| <span class="source-line-no">831</span><span id="line-831"> BloomType bloomType = BloomType.valueOf(e.getValue());</span> |
| <span class="source-line-no">832</span><span id="line-832"> bloomTypeMap.put(e.getKey(), bloomType);</span> |
| <span class="source-line-no">833</span><span id="line-833"> }</span> |
| <span class="source-line-no">834</span><span id="line-834"> return bloomTypeMap;</span> |
| <span class="source-line-no">835</span><span id="line-835"> }</span> |
| <span class="source-line-no">836</span><span id="line-836"></span> |
| <span class="source-line-no">837</span><span id="line-837"> /**</span> |
| <span class="source-line-no">838</span><span id="line-838"> * Runs inside the task to deserialize column family to bloom filter param map from the</span> |
| <span class="source-line-no">839</span><span id="line-839"> * configuration.</span> |
| <span class="source-line-no">840</span><span id="line-840"> * @param conf to read the serialized values from</span> |
| <span class="source-line-no">841</span><span id="line-841"> * @return a map from column family to the the configured bloom filter param</span> |
| <span class="source-line-no">842</span><span id="line-842"> */</span> |
| <span class="source-line-no">843</span><span id="line-843"> @InterfaceAudience.Private</span> |
| <span class="source-line-no">844</span><span id="line-844"> static Map<byte[], String> createFamilyBloomParamMap(Configuration conf) {</span> |
| <span class="source-line-no">845</span><span id="line-845"> return createFamilyConfValueMap(conf, BLOOM_PARAM_FAMILIES_CONF_KEY);</span> |
| <span class="source-line-no">846</span><span id="line-846"> }</span> |
| <span class="source-line-no">847</span><span id="line-847"></span> |
| <span class="source-line-no">848</span><span id="line-848"> /**</span> |
| <span class="source-line-no">849</span><span id="line-849"> * Runs inside the task to deserialize column family to block size map from the configuration.</span> |
| <span class="source-line-no">850</span><span id="line-850"> * @param conf to read the serialized values from</span> |
| <span class="source-line-no">851</span><span id="line-851"> * @return a map from column family to the configured block size</span> |
| <span class="source-line-no">852</span><span id="line-852"> */</span> |
| <span class="source-line-no">853</span><span id="line-853"> @InterfaceAudience.Private</span> |
| <span class="source-line-no">854</span><span id="line-854"> static Map<byte[], Integer> createFamilyBlockSizeMap(Configuration conf) {</span> |
| <span class="source-line-no">855</span><span id="line-855"> Map<byte[], String> stringMap = createFamilyConfValueMap(conf, BLOCK_SIZE_FAMILIES_CONF_KEY);</span> |
| <span class="source-line-no">856</span><span id="line-856"> Map<byte[], Integer> blockSizeMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);</span> |
| <span class="source-line-no">857</span><span id="line-857"> for (Map.Entry<byte[], String> e : stringMap.entrySet()) {</span> |
| <span class="source-line-no">858</span><span id="line-858"> Integer blockSize = Integer.parseInt(e.getValue());</span> |
| <span class="source-line-no">859</span><span id="line-859"> blockSizeMap.put(e.getKey(), blockSize);</span> |
| <span class="source-line-no">860</span><span id="line-860"> }</span> |
| <span class="source-line-no">861</span><span id="line-861"> return blockSizeMap;</span> |
| <span class="source-line-no">862</span><span id="line-862"> }</span> |
| <span class="source-line-no">863</span><span id="line-863"></span> |
| <span class="source-line-no">864</span><span id="line-864"> /**</span> |
| <span class="source-line-no">865</span><span id="line-865"> * Runs inside the task to deserialize column family to data block encoding type map from the</span> |
| <span class="source-line-no">866</span><span id="line-866"> * configuration.</span> |
| <span class="source-line-no">867</span><span id="line-867"> * @param conf to read the serialized values from</span> |
| <span class="source-line-no">868</span><span id="line-868"> * @return a map from column family to HFileDataBlockEncoder for the configured data block type</span> |
| <span class="source-line-no">869</span><span id="line-869"> * for the family</span> |
| <span class="source-line-no">870</span><span id="line-870"> */</span> |
| <span class="source-line-no">871</span><span id="line-871"> @InterfaceAudience.Private</span> |
| <span class="source-line-no">872</span><span id="line-872"> static Map<byte[], DataBlockEncoding> createFamilyDataBlockEncodingMap(Configuration conf) {</span> |
| <span class="source-line-no">873</span><span id="line-873"> Map<byte[], String> stringMap =</span> |
| <span class="source-line-no">874</span><span id="line-874"> createFamilyConfValueMap(conf, DATABLOCK_ENCODING_FAMILIES_CONF_KEY);</span> |
| <span class="source-line-no">875</span><span id="line-875"> Map<byte[], DataBlockEncoding> encoderMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);</span> |
| <span class="source-line-no">876</span><span id="line-876"> for (Map.Entry<byte[], String> e : stringMap.entrySet()) {</span> |
| <span class="source-line-no">877</span><span id="line-877"> encoderMap.put(e.getKey(), DataBlockEncoding.valueOf((e.getValue())));</span> |
| <span class="source-line-no">878</span><span id="line-878"> }</span> |
| <span class="source-line-no">879</span><span id="line-879"> return encoderMap;</span> |
| <span class="source-line-no">880</span><span id="line-880"> }</span> |
| <span class="source-line-no">881</span><span id="line-881"></span> |
| <span class="source-line-no">882</span><span id="line-882"> /**</span> |
| <span class="source-line-no">883</span><span id="line-883"> * Run inside the task to deserialize column family to given conf value map.</span> |
| <span class="source-line-no">884</span><span id="line-884"> * @param conf to read the serialized values from</span> |
| <span class="source-line-no">885</span><span id="line-885"> * @param confName conf key to read from the configuration</span> |
| <span class="source-line-no">886</span><span id="line-886"> * @return a map of column family to the given configuration value</span> |
| <span class="source-line-no">887</span><span id="line-887"> */</span> |
| <span class="source-line-no">888</span><span id="line-888"> private static Map<byte[], String> createFamilyConfValueMap(Configuration conf, String confName) {</span> |
| <span class="source-line-no">889</span><span id="line-889"> Map<byte[], String> confValMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);</span> |
| <span class="source-line-no">890</span><span id="line-890"> String confVal = conf.get(confName, "");</span> |
| <span class="source-line-no">891</span><span id="line-891"> for (String familyConf : confVal.split("&")) {</span> |
| <span class="source-line-no">892</span><span id="line-892"> String[] familySplit = familyConf.split("=");</span> |
| <span class="source-line-no">893</span><span id="line-893"> if (familySplit.length != 2) {</span> |
| <span class="source-line-no">894</span><span id="line-894"> continue;</span> |
| <span class="source-line-no">895</span><span id="line-895"> }</span> |
| <span class="source-line-no">896</span><span id="line-896"> try {</span> |
| <span class="source-line-no">897</span><span id="line-897"> confValMap.put(Bytes.toBytes(URLDecoder.decode(familySplit[0], "UTF-8")),</span> |
| <span class="source-line-no">898</span><span id="line-898"> URLDecoder.decode(familySplit[1], "UTF-8"));</span> |
| <span class="source-line-no">899</span><span id="line-899"> } catch (UnsupportedEncodingException e) {</span> |
| <span class="source-line-no">900</span><span id="line-900"> // will not happen with UTF-8 encoding</span> |
| <span class="source-line-no">901</span><span id="line-901"> throw new AssertionError(e);</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"> return confValMap;</span> |
| <span class="source-line-no">905</span><span id="line-905"> }</span> |
| <span class="source-line-no">906</span><span id="line-906"></span> |
| <span class="source-line-no">907</span><span id="line-907"> /**</span> |
| <span class="source-line-no">908</span><span id="line-908"> * Configure <code>job</code> with a TotalOrderPartitioner, partitioning against</span> |
| <span class="source-line-no">909</span><span id="line-909"> * <code>splitPoints</code>. Cleans up the partitions file after job exists.</span> |
| <span class="source-line-no">910</span><span id="line-910"> */</span> |
| <span class="source-line-no">911</span><span id="line-911"> static void configurePartitioner(Job job, List<ImmutableBytesWritable> splitPoints,</span> |
| <span class="source-line-no">912</span><span id="line-912"> boolean writeMultipleTables) throws IOException {</span> |
| <span class="source-line-no">913</span><span id="line-913"> Configuration conf = job.getConfiguration();</span> |
| <span class="source-line-no">914</span><span id="line-914"> // create the partitions file</span> |
| <span class="source-line-no">915</span><span id="line-915"> FileSystem fs = FileSystem.get(conf);</span> |
| <span class="source-line-no">916</span><span id="line-916"> String hbaseTmpFsDir =</span> |
| <span class="source-line-no">917</span><span id="line-917"> conf.get(HConstants.TEMPORARY_FS_DIRECTORY_KEY, fs.getHomeDirectory() + "/hbase-staging");</span> |
| <span class="source-line-no">918</span><span id="line-918"> Path partitionsPath =</span> |
| <span class="source-line-no">919</span><span id="line-919"> fs.makeQualified(new Path(hbaseTmpFsDir, "partitions_" + UUID.randomUUID()));</span> |
| <span class="source-line-no">920</span><span id="line-920"> writePartitions(conf, partitionsPath, splitPoints, writeMultipleTables);</span> |
| <span class="source-line-no">921</span><span id="line-921"> fs.deleteOnExit(partitionsPath);</span> |
| <span class="source-line-no">922</span><span id="line-922"></span> |
| <span class="source-line-no">923</span><span id="line-923"> // configure job to use it</span> |
| <span class="source-line-no">924</span><span id="line-924"> job.setPartitionerClass(TotalOrderPartitioner.class);</span> |
| <span class="source-line-no">925</span><span id="line-925"> TotalOrderPartitioner.setPartitionFile(conf, partitionsPath);</span> |
| <span class="source-line-no">926</span><span id="line-926"> }</span> |
| <span class="source-line-no">927</span><span id="line-927"></span> |
| <span class="source-line-no">928</span><span id="line-928"> @edu.umd.cs.findbugs.annotations.SuppressWarnings(</span> |
| <span class="source-line-no">929</span><span id="line-929"> value = "RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")</span> |
| <span class="source-line-no">930</span><span id="line-930"> @InterfaceAudience.Private</span> |
| <span class="source-line-no">931</span><span id="line-931"> static String serializeColumnFamilyAttribute(Function<ColumnFamilyDescriptor, String> fn,</span> |
| <span class="source-line-no">932</span><span id="line-932"> List<TableDescriptor> allTables) throws UnsupportedEncodingException {</span> |
| <span class="source-line-no">933</span><span id="line-933"> StringBuilder attributeValue = new StringBuilder();</span> |
| <span class="source-line-no">934</span><span id="line-934"> int i = 0;</span> |
| <span class="source-line-no">935</span><span id="line-935"> for (TableDescriptor tableDescriptor : allTables) {</span> |
| <span class="source-line-no">936</span><span id="line-936"> if (tableDescriptor == null) {</span> |
| <span class="source-line-no">937</span><span id="line-937"> // could happen with mock table instance</span> |
| <span class="source-line-no">938</span><span id="line-938"> // CODEREVIEW: Can I set an empty string in conf if mock table instance?</span> |
| <span class="source-line-no">939</span><span id="line-939"> return "";</span> |
| <span class="source-line-no">940</span><span id="line-940"> }</span> |
| <span class="source-line-no">941</span><span id="line-941"> for (ColumnFamilyDescriptor familyDescriptor : tableDescriptor.getColumnFamilies()) {</span> |
| <span class="source-line-no">942</span><span id="line-942"> if (i++ > 0) {</span> |
| <span class="source-line-no">943</span><span id="line-943"> attributeValue.append('&');</span> |
| <span class="source-line-no">944</span><span id="line-944"> }</span> |
| <span class="source-line-no">945</span><span id="line-945"> attributeValue.append(URLEncoder</span> |
| <span class="source-line-no">946</span><span id="line-946"> .encode(Bytes.toString(combineTableNameSuffix(tableDescriptor.getTableName().getName(),</span> |
| <span class="source-line-no">947</span><span id="line-947"> familyDescriptor.getName())), "UTF-8"));</span> |
| <span class="source-line-no">948</span><span id="line-948"> attributeValue.append('=');</span> |
| <span class="source-line-no">949</span><span id="line-949"> attributeValue.append(URLEncoder.encode(fn.apply(familyDescriptor), "UTF-8"));</span> |
| <span class="source-line-no">950</span><span id="line-950"> }</span> |
| <span class="source-line-no">951</span><span id="line-951"> }</span> |
| <span class="source-line-no">952</span><span id="line-952"> // Get rid of the last ampersand</span> |
| <span class="source-line-no">953</span><span id="line-953"> return attributeValue.toString();</span> |
| <span class="source-line-no">954</span><span id="line-954"> }</span> |
| <span class="source-line-no">955</span><span id="line-955"></span> |
| <span class="source-line-no">956</span><span id="line-956"> /**</span> |
| <span class="source-line-no">957</span><span id="line-957"> * Serialize column family to compression algorithm map to configuration. Invoked while</span> |
| <span class="source-line-no">958</span><span id="line-958"> * configuring the MR job for incremental load.</span> |
| <span class="source-line-no">959</span><span id="line-959"> */</span> |
| <span class="source-line-no">960</span><span id="line-960"> @InterfaceAudience.Private</span> |
| <span class="source-line-no">961</span><span id="line-961"> static Function<ColumnFamilyDescriptor, String> compressionDetails =</span> |
| <span class="source-line-no">962</span><span id="line-962"> familyDescriptor -> familyDescriptor.getCompressionType().getName();</span> |
| <span class="source-line-no">963</span><span id="line-963"></span> |
| <span class="source-line-no">964</span><span id="line-964"> /**</span> |
| <span class="source-line-no">965</span><span id="line-965"> * Serialize column family to block size map to configuration. Invoked while configuring the MR</span> |
| <span class="source-line-no">966</span><span id="line-966"> * job for incremental load.</span> |
| <span class="source-line-no">967</span><span id="line-967"> */</span> |
| <span class="source-line-no">968</span><span id="line-968"> @InterfaceAudience.Private</span> |
| <span class="source-line-no">969</span><span id="line-969"> static Function<ColumnFamilyDescriptor, String> blockSizeDetails =</span> |
| <span class="source-line-no">970</span><span id="line-970"> familyDescriptor -> String.valueOf(familyDescriptor.getBlocksize());</span> |
| <span class="source-line-no">971</span><span id="line-971"></span> |
| <span class="source-line-no">972</span><span id="line-972"> /**</span> |
| <span class="source-line-no">973</span><span id="line-973"> * Serialize column family to bloom type map to configuration. Invoked while configuring the MR</span> |
| <span class="source-line-no">974</span><span id="line-974"> * job for incremental load.</span> |
| <span class="source-line-no">975</span><span id="line-975"> */</span> |
| <span class="source-line-no">976</span><span id="line-976"> @InterfaceAudience.Private</span> |
| <span class="source-line-no">977</span><span id="line-977"> static Function<ColumnFamilyDescriptor, String> bloomTypeDetails = familyDescriptor -> {</span> |
| <span class="source-line-no">978</span><span id="line-978"> String bloomType = familyDescriptor.getBloomFilterType().toString();</span> |
| <span class="source-line-no">979</span><span id="line-979"> if (bloomType == null) {</span> |
| <span class="source-line-no">980</span><span id="line-980"> bloomType = ColumnFamilyDescriptorBuilder.DEFAULT_BLOOMFILTER.name();</span> |
| <span class="source-line-no">981</span><span id="line-981"> }</span> |
| <span class="source-line-no">982</span><span id="line-982"> return bloomType;</span> |
| <span class="source-line-no">983</span><span id="line-983"> };</span> |
| <span class="source-line-no">984</span><span id="line-984"></span> |
| <span class="source-line-no">985</span><span id="line-985"> /**</span> |
| <span class="source-line-no">986</span><span id="line-986"> * Serialize column family to bloom param map to configuration. Invoked while configuring the MR</span> |
| <span class="source-line-no">987</span><span id="line-987"> * job for incremental load.</span> |
| <span class="source-line-no">988</span><span id="line-988"> */</span> |
| <span class="source-line-no">989</span><span id="line-989"> @InterfaceAudience.Private</span> |
| <span class="source-line-no">990</span><span id="line-990"> static Function<ColumnFamilyDescriptor, String> bloomParamDetails = familyDescriptor -> {</span> |
| <span class="source-line-no">991</span><span id="line-991"> BloomType bloomType = familyDescriptor.getBloomFilterType();</span> |
| <span class="source-line-no">992</span><span id="line-992"> String bloomParam = "";</span> |
| <span class="source-line-no">993</span><span id="line-993"> if (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH) {</span> |
| <span class="source-line-no">994</span><span id="line-994"> bloomParam = familyDescriptor.getConfigurationValue(BloomFilterUtil.PREFIX_LENGTH_KEY);</span> |
| <span class="source-line-no">995</span><span id="line-995"> }</span> |
| <span class="source-line-no">996</span><span id="line-996"> return bloomParam;</span> |
| <span class="source-line-no">997</span><span id="line-997"> };</span> |
| <span class="source-line-no">998</span><span id="line-998"></span> |
| <span class="source-line-no">999</span><span id="line-999"> /**</span> |
| <span class="source-line-no">1000</span><span id="line-1000"> * Serialize column family to data block encoding map to configuration. Invoked while configuring</span> |
| <span class="source-line-no">1001</span><span id="line-1001"> * the MR job for incremental load.</span> |
| <span class="source-line-no">1002</span><span id="line-1002"> */</span> |
| <span class="source-line-no">1003</span><span id="line-1003"> @InterfaceAudience.Private</span> |
| <span class="source-line-no">1004</span><span id="line-1004"> static Function<ColumnFamilyDescriptor, String> dataBlockEncodingDetails = familyDescriptor -> {</span> |
| <span class="source-line-no">1005</span><span id="line-1005"> DataBlockEncoding encoding = familyDescriptor.getDataBlockEncoding();</span> |
| <span class="source-line-no">1006</span><span id="line-1006"> if (encoding == null) {</span> |
| <span class="source-line-no">1007</span><span id="line-1007"> encoding = DataBlockEncoding.NONE;</span> |
| <span class="source-line-no">1008</span><span id="line-1008"> }</span> |
| <span class="source-line-no">1009</span><span id="line-1009"> return encoding.toString();</span> |
| <span class="source-line-no">1010</span><span id="line-1010"> };</span> |
| <span class="source-line-no">1011</span><span id="line-1011"></span> |
| <span class="source-line-no">1012</span><span id="line-1012">}</span> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </pre> |
| </div> |
| </main> |
| </body> |
| </html> |