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