| <!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 java.io.ByteArrayInputStream;<a name="line.20"></a> |
| <span class="sourceLineNo">021</span>import java.io.DataInput;<a name="line.21"></a> |
| <span class="sourceLineNo">022</span>import java.io.DataInputStream;<a name="line.22"></a> |
| <span class="sourceLineNo">023</span>import java.io.DataOutput;<a name="line.23"></a> |
| <span class="sourceLineNo">024</span>import java.io.IOException;<a name="line.24"></a> |
| <span class="sourceLineNo">025</span>import java.lang.reflect.InvocationTargetException;<a name="line.25"></a> |
| <span class="sourceLineNo">026</span>import java.lang.reflect.Method;<a name="line.26"></a> |
| <span class="sourceLineNo">027</span>import java.util.ArrayList;<a name="line.27"></a> |
| <span class="sourceLineNo">028</span>import java.util.Collections;<a name="line.28"></a> |
| <span class="sourceLineNo">029</span>import java.util.List;<a name="line.29"></a> |
| <span class="sourceLineNo">030</span>import java.util.Locale;<a name="line.30"></a> |
| <span class="sourceLineNo">031</span>import java.util.Map;<a name="line.31"></a> |
| <span class="sourceLineNo">032</span>import java.util.TreeMap;<a name="line.32"></a> |
| <span class="sourceLineNo">033</span>import java.util.UUID;<a name="line.33"></a> |
| <span class="sourceLineNo">034</span>import org.apache.hadoop.conf.Configuration;<a name="line.34"></a> |
| <span class="sourceLineNo">035</span>import org.apache.hadoop.conf.Configured;<a name="line.35"></a> |
| <span class="sourceLineNo">036</span>import org.apache.hadoop.fs.FileSystem;<a name="line.36"></a> |
| <span class="sourceLineNo">037</span>import org.apache.hadoop.fs.Path;<a name="line.37"></a> |
| <span class="sourceLineNo">038</span>import org.apache.hadoop.hbase.Cell;<a name="line.38"></a> |
| <span class="sourceLineNo">039</span>import org.apache.hadoop.hbase.CellComparator;<a name="line.39"></a> |
| <span class="sourceLineNo">040</span>import org.apache.hadoop.hbase.CellUtil;<a name="line.40"></a> |
| <span class="sourceLineNo">041</span>import org.apache.hadoop.hbase.HBaseConfiguration;<a name="line.41"></a> |
| <span class="sourceLineNo">042</span>import org.apache.hadoop.hbase.KeyValue;<a name="line.42"></a> |
| <span class="sourceLineNo">043</span>import org.apache.hadoop.hbase.KeyValueUtil;<a name="line.43"></a> |
| <span class="sourceLineNo">044</span>import org.apache.hadoop.hbase.PrivateCellUtil;<a name="line.44"></a> |
| <span class="sourceLineNo">045</span>import org.apache.hadoop.hbase.TableName;<a name="line.45"></a> |
| <span class="sourceLineNo">046</span>import org.apache.hadoop.hbase.Tag;<a name="line.46"></a> |
| <span class="sourceLineNo">047</span>import org.apache.hadoop.hbase.ZooKeeperConnectionException;<a name="line.47"></a> |
| <span class="sourceLineNo">048</span>import org.apache.hadoop.hbase.client.Admin;<a name="line.48"></a> |
| <span class="sourceLineNo">049</span>import org.apache.hadoop.hbase.client.Connection;<a name="line.49"></a> |
| <span class="sourceLineNo">050</span>import org.apache.hadoop.hbase.client.ConnectionFactory;<a name="line.50"></a> |
| <span class="sourceLineNo">051</span>import org.apache.hadoop.hbase.client.Delete;<a name="line.51"></a> |
| <span class="sourceLineNo">052</span>import org.apache.hadoop.hbase.client.Durability;<a name="line.52"></a> |
| <span class="sourceLineNo">053</span>import org.apache.hadoop.hbase.client.Mutation;<a name="line.53"></a> |
| <span class="sourceLineNo">054</span>import org.apache.hadoop.hbase.client.Put;<a name="line.54"></a> |
| <span class="sourceLineNo">055</span>import org.apache.hadoop.hbase.client.RegionLocator;<a name="line.55"></a> |
| <span class="sourceLineNo">056</span>import org.apache.hadoop.hbase.client.Result;<a name="line.56"></a> |
| <span class="sourceLineNo">057</span>import org.apache.hadoop.hbase.client.Table;<a name="line.57"></a> |
| <span class="sourceLineNo">058</span>import org.apache.hadoop.hbase.filter.Filter;<a name="line.58"></a> |
| <span class="sourceLineNo">059</span>import org.apache.hadoop.hbase.io.ImmutableBytesWritable;<a name="line.59"></a> |
| <span class="sourceLineNo">060</span>import org.apache.hadoop.hbase.util.Bytes;<a name="line.60"></a> |
| <span class="sourceLineNo">061</span>import org.apache.hadoop.hbase.util.MapReduceExtendedCell;<a name="line.61"></a> |
| <span class="sourceLineNo">062</span>import org.apache.hadoop.hbase.zookeeper.ZKClusterId;<a name="line.62"></a> |
| <span class="sourceLineNo">063</span>import org.apache.hadoop.hbase.zookeeper.ZKWatcher;<a name="line.63"></a> |
| <span class="sourceLineNo">064</span>import org.apache.hadoop.io.RawComparator;<a name="line.64"></a> |
| <span class="sourceLineNo">065</span>import org.apache.hadoop.io.WritableComparable;<a name="line.65"></a> |
| <span class="sourceLineNo">066</span>import org.apache.hadoop.io.WritableComparator;<a name="line.66"></a> |
| <span class="sourceLineNo">067</span>import org.apache.hadoop.mapreduce.Job;<a name="line.67"></a> |
| <span class="sourceLineNo">068</span>import org.apache.hadoop.mapreduce.Partitioner;<a name="line.68"></a> |
| <span class="sourceLineNo">069</span>import org.apache.hadoop.mapreduce.Reducer;<a name="line.69"></a> |
| <span class="sourceLineNo">070</span>import org.apache.hadoop.mapreduce.TaskCounter;<a name="line.70"></a> |
| <span class="sourceLineNo">071</span>import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;<a name="line.71"></a> |
| <span class="sourceLineNo">072</span>import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;<a name="line.72"></a> |
| <span class="sourceLineNo">073</span>import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;<a name="line.73"></a> |
| <span class="sourceLineNo">074</span>import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;<a name="line.74"></a> |
| <span class="sourceLineNo">075</span>import org.apache.hadoop.util.Tool;<a name="line.75"></a> |
| <span class="sourceLineNo">076</span>import org.apache.hadoop.util.ToolRunner;<a name="line.76"></a> |
| <span class="sourceLineNo">077</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.77"></a> |
| <span class="sourceLineNo">078</span>import org.apache.zookeeper.KeeperException;<a name="line.78"></a> |
| <span class="sourceLineNo">079</span>import org.slf4j.Logger;<a name="line.79"></a> |
| <span class="sourceLineNo">080</span>import org.slf4j.LoggerFactory;<a name="line.80"></a> |
| <span class="sourceLineNo">081</span><a name="line.81"></a> |
| <span class="sourceLineNo">082</span>/**<a name="line.82"></a> |
| <span class="sourceLineNo">083</span> * Import data written by {@link Export}.<a name="line.83"></a> |
| <span class="sourceLineNo">084</span> */<a name="line.84"></a> |
| <span class="sourceLineNo">085</span>@InterfaceAudience.Public<a name="line.85"></a> |
| <span class="sourceLineNo">086</span>public class Import extends Configured implements Tool {<a name="line.86"></a> |
| <span class="sourceLineNo">087</span> private static final Logger LOG = LoggerFactory.getLogger(Import.class);<a name="line.87"></a> |
| <span class="sourceLineNo">088</span> final static String NAME = "import";<a name="line.88"></a> |
| <span class="sourceLineNo">089</span> public final static String CF_RENAME_PROP = "HBASE_IMPORTER_RENAME_CFS";<a name="line.89"></a> |
| <span class="sourceLineNo">090</span> public final static String BULK_OUTPUT_CONF_KEY = "import.bulk.output";<a name="line.90"></a> |
| <span class="sourceLineNo">091</span> public final static String FILTER_CLASS_CONF_KEY = "import.filter.class";<a name="line.91"></a> |
| <span class="sourceLineNo">092</span> public final static String FILTER_ARGS_CONF_KEY = "import.filter.args";<a name="line.92"></a> |
| <span class="sourceLineNo">093</span> public final static String TABLE_NAME = "import.table.name";<a name="line.93"></a> |
| <span class="sourceLineNo">094</span> public final static String WAL_DURABILITY = "import.wal.durability";<a name="line.94"></a> |
| <span class="sourceLineNo">095</span> public final static String HAS_LARGE_RESULT = "import.bulk.hasLargeResult";<a name="line.95"></a> |
| <span class="sourceLineNo">096</span><a name="line.96"></a> |
| <span class="sourceLineNo">097</span> private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";<a name="line.97"></a> |
| <span class="sourceLineNo">098</span><a name="line.98"></a> |
| <span class="sourceLineNo">099</span> public static class CellWritableComparablePartitioner<a name="line.99"></a> |
| <span class="sourceLineNo">100</span> extends Partitioner<CellWritableComparable, Cell> {<a name="line.100"></a> |
| <span class="sourceLineNo">101</span> private static CellWritableComparable[] START_KEYS = null;<a name="line.101"></a> |
| <span class="sourceLineNo">102</span><a name="line.102"></a> |
| <span class="sourceLineNo">103</span> @Override<a name="line.103"></a> |
| <span class="sourceLineNo">104</span> public int getPartition(CellWritableComparable key, Cell value, int numPartitions) {<a name="line.104"></a> |
| <span class="sourceLineNo">105</span> for (int i = 0; i < START_KEYS.length; ++i) {<a name="line.105"></a> |
| <span class="sourceLineNo">106</span> if (key.compareTo(START_KEYS[i]) <= 0) {<a name="line.106"></a> |
| <span class="sourceLineNo">107</span> return i;<a name="line.107"></a> |
| <span class="sourceLineNo">108</span> }<a name="line.108"></a> |
| <span class="sourceLineNo">109</span> }<a name="line.109"></a> |
| <span class="sourceLineNo">110</span> return START_KEYS.length;<a name="line.110"></a> |
| <span class="sourceLineNo">111</span> }<a name="line.111"></a> |
| <span class="sourceLineNo">112</span><a name="line.112"></a> |
| <span class="sourceLineNo">113</span> }<a name="line.113"></a> |
| <span class="sourceLineNo">114</span><a name="line.114"></a> |
| <span class="sourceLineNo">115</span> public static class CellWritableComparable implements WritableComparable<CellWritableComparable> {<a name="line.115"></a> |
| <span class="sourceLineNo">116</span><a name="line.116"></a> |
| <span class="sourceLineNo">117</span> private Cell kv = null;<a name="line.117"></a> |
| <span class="sourceLineNo">118</span><a name="line.118"></a> |
| <span class="sourceLineNo">119</span> static {<a name="line.119"></a> |
| <span class="sourceLineNo">120</span> // register this comparator<a name="line.120"></a> |
| <span class="sourceLineNo">121</span> WritableComparator.define(CellWritableComparable.class, new CellWritableComparator());<a name="line.121"></a> |
| <span class="sourceLineNo">122</span> }<a name="line.122"></a> |
| <span class="sourceLineNo">123</span><a name="line.123"></a> |
| <span class="sourceLineNo">124</span> public CellWritableComparable() {<a name="line.124"></a> |
| <span class="sourceLineNo">125</span> }<a name="line.125"></a> |
| <span class="sourceLineNo">126</span><a name="line.126"></a> |
| <span class="sourceLineNo">127</span> public CellWritableComparable(Cell kv) {<a name="line.127"></a> |
| <span class="sourceLineNo">128</span> this.kv = kv;<a name="line.128"></a> |
| <span class="sourceLineNo">129</span> }<a name="line.129"></a> |
| <span class="sourceLineNo">130</span><a name="line.130"></a> |
| <span class="sourceLineNo">131</span> @Override<a name="line.131"></a> |
| <span class="sourceLineNo">132</span> public void write(DataOutput out) throws IOException {<a name="line.132"></a> |
| <span class="sourceLineNo">133</span> out.writeInt(PrivateCellUtil.estimatedSerializedSizeOfKey(kv));<a name="line.133"></a> |
| <span class="sourceLineNo">134</span> out.writeInt(0);<a name="line.134"></a> |
| <span class="sourceLineNo">135</span> PrivateCellUtil.writeFlatKey(kv, out);<a name="line.135"></a> |
| <span class="sourceLineNo">136</span> }<a name="line.136"></a> |
| <span class="sourceLineNo">137</span><a name="line.137"></a> |
| <span class="sourceLineNo">138</span> @Override<a name="line.138"></a> |
| <span class="sourceLineNo">139</span> public void readFields(DataInput in) throws IOException {<a name="line.139"></a> |
| <span class="sourceLineNo">140</span> kv = KeyValue.create(in);<a name="line.140"></a> |
| <span class="sourceLineNo">141</span> }<a name="line.141"></a> |
| <span class="sourceLineNo">142</span><a name="line.142"></a> |
| <span class="sourceLineNo">143</span> @Override<a name="line.143"></a> |
| <span class="sourceLineNo">144</span> @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "EQ_COMPARETO_USE_OBJECT_EQUALS",<a name="line.144"></a> |
| <span class="sourceLineNo">145</span> justification = "This is wrong, yes, but we should be purging Writables, not fixing them")<a name="line.145"></a> |
| <span class="sourceLineNo">146</span> public int compareTo(CellWritableComparable o) {<a name="line.146"></a> |
| <span class="sourceLineNo">147</span> return CellComparator.getInstance().compare(this.kv, o.kv);<a name="line.147"></a> |
| <span class="sourceLineNo">148</span> }<a name="line.148"></a> |
| <span class="sourceLineNo">149</span><a name="line.149"></a> |
| <span class="sourceLineNo">150</span> public static class CellWritableComparator extends WritableComparator {<a name="line.150"></a> |
| <span class="sourceLineNo">151</span><a name="line.151"></a> |
| <span class="sourceLineNo">152</span> @Override<a name="line.152"></a> |
| <span class="sourceLineNo">153</span> public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {<a name="line.153"></a> |
| <span class="sourceLineNo">154</span> try {<a name="line.154"></a> |
| <span class="sourceLineNo">155</span> CellWritableComparable kv1 = new CellWritableComparable();<a name="line.155"></a> |
| <span class="sourceLineNo">156</span> kv1.readFields(new DataInputStream(new ByteArrayInputStream(b1, s1, l1)));<a name="line.156"></a> |
| <span class="sourceLineNo">157</span> CellWritableComparable kv2 = new CellWritableComparable();<a name="line.157"></a> |
| <span class="sourceLineNo">158</span> kv2.readFields(new DataInputStream(new ByteArrayInputStream(b2, s2, l2)));<a name="line.158"></a> |
| <span class="sourceLineNo">159</span> return compare(kv1, kv2);<a name="line.159"></a> |
| <span class="sourceLineNo">160</span> } catch (IOException e) {<a name="line.160"></a> |
| <span class="sourceLineNo">161</span> throw new RuntimeException(e);<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><a name="line.164"></a> |
| <span class="sourceLineNo">165</span> }<a name="line.165"></a> |
| <span class="sourceLineNo">166</span><a name="line.166"></a> |
| <span class="sourceLineNo">167</span> }<a name="line.167"></a> |
| <span class="sourceLineNo">168</span><a name="line.168"></a> |
| <span class="sourceLineNo">169</span> public static class CellReducer<a name="line.169"></a> |
| <span class="sourceLineNo">170</span> extends Reducer<CellWritableComparable, Cell, ImmutableBytesWritable, Cell> {<a name="line.170"></a> |
| <span class="sourceLineNo">171</span> protected void reduce(CellWritableComparable row, Iterable<Cell> kvs,<a name="line.171"></a> |
| <span class="sourceLineNo">172</span> Reducer<CellWritableComparable, Cell, ImmutableBytesWritable, Cell>.Context context)<a name="line.172"></a> |
| <span class="sourceLineNo">173</span> throws java.io.IOException, InterruptedException {<a name="line.173"></a> |
| <span class="sourceLineNo">174</span> int index = 0;<a name="line.174"></a> |
| <span class="sourceLineNo">175</span> for (Cell kv : kvs) {<a name="line.175"></a> |
| <span class="sourceLineNo">176</span> context.write(new ImmutableBytesWritable(CellUtil.cloneRow(kv)),<a name="line.176"></a> |
| <span class="sourceLineNo">177</span> new MapReduceExtendedCell(kv));<a name="line.177"></a> |
| <span class="sourceLineNo">178</span> if (++index % 100 == 0) context.setStatus("Wrote " + index + " KeyValues, "<a name="line.178"></a> |
| <span class="sourceLineNo">179</span> + "and the rowkey whose is being wrote is " + Bytes.toString(kv.getRowArray()));<a name="line.179"></a> |
| <span class="sourceLineNo">180</span> }<a name="line.180"></a> |
| <span class="sourceLineNo">181</span> }<a name="line.181"></a> |
| <span class="sourceLineNo">182</span> }<a name="line.182"></a> |
| <span class="sourceLineNo">183</span><a name="line.183"></a> |
| <span class="sourceLineNo">184</span> public static class CellSortImporter extends TableMapper<CellWritableComparable, Cell> {<a name="line.184"></a> |
| <span class="sourceLineNo">185</span> private Map<byte[], byte[]> cfRenameMap;<a name="line.185"></a> |
| <span class="sourceLineNo">186</span> private Filter filter;<a name="line.186"></a> |
| <span class="sourceLineNo">187</span> private static final Logger LOG = LoggerFactory.getLogger(CellImporter.class);<a name="line.187"></a> |
| <span class="sourceLineNo">188</span><a name="line.188"></a> |
| <span class="sourceLineNo">189</span> /**<a name="line.189"></a> |
| <span class="sourceLineNo">190</span> * @param row The current table row key.<a name="line.190"></a> |
| <span class="sourceLineNo">191</span> * @param value The columns.<a name="line.191"></a> |
| <span class="sourceLineNo">192</span> * @param context The current context.<a name="line.192"></a> |
| <span class="sourceLineNo">193</span> * @throws IOException When something is broken with the data.<a name="line.193"></a> |
| <span class="sourceLineNo">194</span> */<a name="line.194"></a> |
| <span class="sourceLineNo">195</span> @Override<a name="line.195"></a> |
| <span class="sourceLineNo">196</span> public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException {<a name="line.196"></a> |
| <span class="sourceLineNo">197</span> try {<a name="line.197"></a> |
| <span class="sourceLineNo">198</span> if (LOG.isTraceEnabled()) {<a name="line.198"></a> |
| <span class="sourceLineNo">199</span> LOG.trace(<a name="line.199"></a> |
| <span class="sourceLineNo">200</span> "Considering the row." + Bytes.toString(row.get(), row.getOffset(), row.getLength()));<a name="line.200"></a> |
| <span class="sourceLineNo">201</span> }<a name="line.201"></a> |
| <span class="sourceLineNo">202</span> if (<a name="line.202"></a> |
| <span class="sourceLineNo">203</span> filter == null || !filter.filterRowKey(<a name="line.203"></a> |
| <span class="sourceLineNo">204</span> PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(), (short) row.getLength()))<a name="line.204"></a> |
| <span class="sourceLineNo">205</span> ) {<a name="line.205"></a> |
| <span class="sourceLineNo">206</span> for (Cell kv : value.rawCells()) {<a name="line.206"></a> |
| <span class="sourceLineNo">207</span> kv = filterKv(filter, kv);<a name="line.207"></a> |
| <span class="sourceLineNo">208</span> // skip if we filtered it out<a name="line.208"></a> |
| <span class="sourceLineNo">209</span> if (kv == null) continue;<a name="line.209"></a> |
| <span class="sourceLineNo">210</span> Cell ret = convertKv(kv, cfRenameMap);<a name="line.210"></a> |
| <span class="sourceLineNo">211</span> context.write(new CellWritableComparable(ret), ret);<a name="line.211"></a> |
| <span class="sourceLineNo">212</span> }<a name="line.212"></a> |
| <span class="sourceLineNo">213</span> }<a name="line.213"></a> |
| <span class="sourceLineNo">214</span> } catch (InterruptedException e) {<a name="line.214"></a> |
| <span class="sourceLineNo">215</span> LOG.error("Interrupted while emitting Cell", e);<a name="line.215"></a> |
| <span class="sourceLineNo">216</span> Thread.currentThread().interrupt();<a name="line.216"></a> |
| <span class="sourceLineNo">217</span> }<a name="line.217"></a> |
| <span class="sourceLineNo">218</span> }<a name="line.218"></a> |
| <span class="sourceLineNo">219</span><a name="line.219"></a> |
| <span class="sourceLineNo">220</span> @Override<a name="line.220"></a> |
| <span class="sourceLineNo">221</span> public void setup(Context context) throws IOException {<a name="line.221"></a> |
| <span class="sourceLineNo">222</span> cfRenameMap = createCfRenameMap(context.getConfiguration());<a name="line.222"></a> |
| <span class="sourceLineNo">223</span> filter = instantiateFilter(context.getConfiguration());<a name="line.223"></a> |
| <span class="sourceLineNo">224</span> int reduceNum = context.getNumReduceTasks();<a name="line.224"></a> |
| <span class="sourceLineNo">225</span> Configuration conf = context.getConfiguration();<a name="line.225"></a> |
| <span class="sourceLineNo">226</span> TableName tableName = TableName.valueOf(context.getConfiguration().get(TABLE_NAME));<a name="line.226"></a> |
| <span class="sourceLineNo">227</span> try (Connection conn = ConnectionFactory.createConnection(conf);<a name="line.227"></a> |
| <span class="sourceLineNo">228</span> RegionLocator regionLocator = conn.getRegionLocator(tableName)) {<a name="line.228"></a> |
| <span class="sourceLineNo">229</span> byte[][] startKeys = regionLocator.getStartKeys();<a name="line.229"></a> |
| <span class="sourceLineNo">230</span> if (startKeys.length != reduceNum) {<a name="line.230"></a> |
| <span class="sourceLineNo">231</span> throw new IOException("Region split after job initialization");<a name="line.231"></a> |
| <span class="sourceLineNo">232</span> }<a name="line.232"></a> |
| <span class="sourceLineNo">233</span> CellWritableComparable[] startKeyWraps = new CellWritableComparable[startKeys.length - 1];<a name="line.233"></a> |
| <span class="sourceLineNo">234</span> for (int i = 1; i < startKeys.length; ++i) {<a name="line.234"></a> |
| <span class="sourceLineNo">235</span> startKeyWraps[i - 1] =<a name="line.235"></a> |
| <span class="sourceLineNo">236</span> new CellWritableComparable(KeyValueUtil.createFirstOnRow(startKeys[i]));<a name="line.236"></a> |
| <span class="sourceLineNo">237</span> }<a name="line.237"></a> |
| <span class="sourceLineNo">238</span> CellWritableComparablePartitioner.START_KEYS = startKeyWraps;<a name="line.238"></a> |
| <span class="sourceLineNo">239</span> }<a name="line.239"></a> |
| <span class="sourceLineNo">240</span> }<a name="line.240"></a> |
| <span class="sourceLineNo">241</span> }<a name="line.241"></a> |
| <span class="sourceLineNo">242</span><a name="line.242"></a> |
| <span class="sourceLineNo">243</span> /**<a name="line.243"></a> |
| <span class="sourceLineNo">244</span> * A mapper that just writes out KeyValues.<a name="line.244"></a> |
| <span class="sourceLineNo">245</span> */<a name="line.245"></a> |
| <span class="sourceLineNo">246</span> @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "EQ_COMPARETO_USE_OBJECT_EQUALS",<a name="line.246"></a> |
| <span class="sourceLineNo">247</span> justification = "Writables are going away and this has been this way forever")<a name="line.247"></a> |
| <span class="sourceLineNo">248</span> public static class CellImporter extends TableMapper<ImmutableBytesWritable, Cell> {<a name="line.248"></a> |
| <span class="sourceLineNo">249</span> private Map<byte[], byte[]> cfRenameMap;<a name="line.249"></a> |
| <span class="sourceLineNo">250</span> private Filter filter;<a name="line.250"></a> |
| <span class="sourceLineNo">251</span> private static final Logger LOG = LoggerFactory.getLogger(CellImporter.class);<a name="line.251"></a> |
| <span class="sourceLineNo">252</span><a name="line.252"></a> |
| <span class="sourceLineNo">253</span> /**<a name="line.253"></a> |
| <span class="sourceLineNo">254</span> * @param row The current table row key.<a name="line.254"></a> |
| <span class="sourceLineNo">255</span> * @param value The columns.<a name="line.255"></a> |
| <span class="sourceLineNo">256</span> * @param context The current context.<a name="line.256"></a> |
| <span class="sourceLineNo">257</span> * @throws IOException When something is broken with the data.<a name="line.257"></a> |
| <span class="sourceLineNo">258</span> */<a name="line.258"></a> |
| <span class="sourceLineNo">259</span> @Override<a name="line.259"></a> |
| <span class="sourceLineNo">260</span> public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException {<a name="line.260"></a> |
| <span class="sourceLineNo">261</span> try {<a name="line.261"></a> |
| <span class="sourceLineNo">262</span> if (LOG.isTraceEnabled()) {<a name="line.262"></a> |
| <span class="sourceLineNo">263</span> LOG.trace(<a name="line.263"></a> |
| <span class="sourceLineNo">264</span> "Considering the row." + Bytes.toString(row.get(), row.getOffset(), row.getLength()));<a name="line.264"></a> |
| <span class="sourceLineNo">265</span> }<a name="line.265"></a> |
| <span class="sourceLineNo">266</span> if (<a name="line.266"></a> |
| <span class="sourceLineNo">267</span> filter == null || !filter.filterRowKey(<a name="line.267"></a> |
| <span class="sourceLineNo">268</span> PrivateCellUtil.createFirstOnRow(row.get(), row.getOffset(), (short) row.getLength()))<a name="line.268"></a> |
| <span class="sourceLineNo">269</span> ) {<a name="line.269"></a> |
| <span class="sourceLineNo">270</span> for (Cell kv : value.rawCells()) {<a name="line.270"></a> |
| <span class="sourceLineNo">271</span> kv = filterKv(filter, kv);<a name="line.271"></a> |
| <span class="sourceLineNo">272</span> // skip if we filtered it out<a name="line.272"></a> |
| <span class="sourceLineNo">273</span> if (kv == null) continue;<a name="line.273"></a> |
| <span class="sourceLineNo">274</span> context.write(row, new MapReduceExtendedCell(convertKv(kv, cfRenameMap)));<a name="line.274"></a> |
| <span class="sourceLineNo">275</span> }<a name="line.275"></a> |
| <span class="sourceLineNo">276</span> }<a name="line.276"></a> |
| <span class="sourceLineNo">277</span> } catch (InterruptedException e) {<a name="line.277"></a> |
| <span class="sourceLineNo">278</span> LOG.error("Interrupted while emitting Cell", e);<a name="line.278"></a> |
| <span class="sourceLineNo">279</span> Thread.currentThread().interrupt();<a name="line.279"></a> |
| <span class="sourceLineNo">280</span> }<a name="line.280"></a> |
| <span class="sourceLineNo">281</span> }<a name="line.281"></a> |
| <span class="sourceLineNo">282</span><a name="line.282"></a> |
| <span class="sourceLineNo">283</span> @Override<a name="line.283"></a> |
| <span class="sourceLineNo">284</span> public void setup(Context context) {<a name="line.284"></a> |
| <span class="sourceLineNo">285</span> cfRenameMap = createCfRenameMap(context.getConfiguration());<a name="line.285"></a> |
| <span class="sourceLineNo">286</span> filter = instantiateFilter(context.getConfiguration());<a name="line.286"></a> |
| <span class="sourceLineNo">287</span> }<a name="line.287"></a> |
| <span class="sourceLineNo">288</span> }<a name="line.288"></a> |
| <span class="sourceLineNo">289</span><a name="line.289"></a> |
| <span class="sourceLineNo">290</span> /**<a name="line.290"></a> |
| <span class="sourceLineNo">291</span> * Write table content out to files in hdfs.<a name="line.291"></a> |
| <span class="sourceLineNo">292</span> */<a name="line.292"></a> |
| <span class="sourceLineNo">293</span> public static class Importer extends TableMapper<ImmutableBytesWritable, Mutation> {<a name="line.293"></a> |
| <span class="sourceLineNo">294</span> private Map<byte[], byte[]> cfRenameMap;<a name="line.294"></a> |
| <span class="sourceLineNo">295</span> private List<UUID> clusterIds;<a name="line.295"></a> |
| <span class="sourceLineNo">296</span> private Filter filter;<a name="line.296"></a> |
| <span class="sourceLineNo">297</span> private Durability durability;<a name="line.297"></a> |
| <span class="sourceLineNo">298</span><a name="line.298"></a> |
| <span class="sourceLineNo">299</span> /**<a name="line.299"></a> |
| <span class="sourceLineNo">300</span> * @param row The current table row key.<a name="line.300"></a> |
| <span class="sourceLineNo">301</span> * @param value The columns.<a name="line.301"></a> |
| <span class="sourceLineNo">302</span> * @param context The current context.<a name="line.302"></a> |
| <span class="sourceLineNo">303</span> * @throws IOException When something is broken with the data.<a name="line.303"></a> |
| <span class="sourceLineNo">304</span> */<a name="line.304"></a> |
| <span class="sourceLineNo">305</span> @Override<a name="line.305"></a> |
| <span class="sourceLineNo">306</span> public void map(ImmutableBytesWritable row, Result value, Context context) throws IOException {<a name="line.306"></a> |
| <span class="sourceLineNo">307</span> try {<a name="line.307"></a> |
| <span class="sourceLineNo">308</span> writeResult(row, value, context);<a name="line.308"></a> |
| <span class="sourceLineNo">309</span> } catch (InterruptedException e) {<a name="line.309"></a> |
| <span class="sourceLineNo">310</span> LOG.error("Interrupted while writing result", e);<a name="line.310"></a> |
| <span class="sourceLineNo">311</span> Thread.currentThread().interrupt();<a name="line.311"></a> |
| <span class="sourceLineNo">312</span> }<a name="line.312"></a> |
| <span class="sourceLineNo">313</span> }<a name="line.313"></a> |
| <span class="sourceLineNo">314</span><a name="line.314"></a> |
| <span class="sourceLineNo">315</span> private void writeResult(ImmutableBytesWritable key, Result result, Context context)<a name="line.315"></a> |
| <span class="sourceLineNo">316</span> throws IOException, InterruptedException {<a name="line.316"></a> |
| <span class="sourceLineNo">317</span> Put put = null;<a name="line.317"></a> |
| <span class="sourceLineNo">318</span> Delete delete = null;<a name="line.318"></a> |
| <span class="sourceLineNo">319</span> if (LOG.isTraceEnabled()) {<a name="line.319"></a> |
| <span class="sourceLineNo">320</span> LOG.trace(<a name="line.320"></a> |
| <span class="sourceLineNo">321</span> "Considering the row." + Bytes.toString(key.get(), key.getOffset(), key.getLength()));<a name="line.321"></a> |
| <span class="sourceLineNo">322</span> }<a name="line.322"></a> |
| <span class="sourceLineNo">323</span> if (<a name="line.323"></a> |
| <span class="sourceLineNo">324</span> filter == null || !filter.filterRowKey(<a name="line.324"></a> |
| <span class="sourceLineNo">325</span> PrivateCellUtil.createFirstOnRow(key.get(), key.getOffset(), (short) key.getLength()))<a name="line.325"></a> |
| <span class="sourceLineNo">326</span> ) {<a name="line.326"></a> |
| <span class="sourceLineNo">327</span> processKV(key, result, context, put, delete);<a name="line.327"></a> |
| <span class="sourceLineNo">328</span> }<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> protected void processKV(ImmutableBytesWritable key, Result result, Context context, Put put,<a name="line.331"></a> |
| <span class="sourceLineNo">332</span> Delete delete) throws IOException, InterruptedException {<a name="line.332"></a> |
| <span class="sourceLineNo">333</span> for (Cell kv : result.rawCells()) {<a name="line.333"></a> |
| <span class="sourceLineNo">334</span> kv = filterKv(filter, kv);<a name="line.334"></a> |
| <span class="sourceLineNo">335</span> // skip if we filter it out<a name="line.335"></a> |
| <span class="sourceLineNo">336</span> if (kv == null) continue;<a name="line.336"></a> |
| <span class="sourceLineNo">337</span><a name="line.337"></a> |
| <span class="sourceLineNo">338</span> kv = convertKv(kv, cfRenameMap);<a name="line.338"></a> |
| <span class="sourceLineNo">339</span> // Deletes and Puts are gathered and written when finished<a name="line.339"></a> |
| <span class="sourceLineNo">340</span> /*<a name="line.340"></a> |
| <span class="sourceLineNo">341</span> * If there are sequence of mutations and tombstones in an Export, and after Import the same<a name="line.341"></a> |
| <span class="sourceLineNo">342</span> * sequence should be restored as it is. If we combine all Delete tombstones into single<a name="line.342"></a> |
| <span class="sourceLineNo">343</span> * request then there is chance of ignoring few DeleteFamily tombstones, because if we<a name="line.343"></a> |
| <span class="sourceLineNo">344</span> * submit multiple DeleteFamily tombstones in single Delete request then we are maintaining<a name="line.344"></a> |
| <span class="sourceLineNo">345</span> * only newest in hbase table and ignoring other. Check - HBASE-12065<a name="line.345"></a> |
| <span class="sourceLineNo">346</span> */<a name="line.346"></a> |
| <span class="sourceLineNo">347</span> if (PrivateCellUtil.isDeleteFamily(kv)) {<a name="line.347"></a> |
| <span class="sourceLineNo">348</span> Delete deleteFamily = new Delete(key.get());<a name="line.348"></a> |
| <span class="sourceLineNo">349</span> deleteFamily.add(kv);<a name="line.349"></a> |
| <span class="sourceLineNo">350</span> if (durability != null) {<a name="line.350"></a> |
| <span class="sourceLineNo">351</span> deleteFamily.setDurability(durability);<a name="line.351"></a> |
| <span class="sourceLineNo">352</span> }<a name="line.352"></a> |
| <span class="sourceLineNo">353</span> deleteFamily.setClusterIds(clusterIds);<a name="line.353"></a> |
| <span class="sourceLineNo">354</span> context.write(key, deleteFamily);<a name="line.354"></a> |
| <span class="sourceLineNo">355</span> } else if (CellUtil.isDelete(kv)) {<a name="line.355"></a> |
| <span class="sourceLineNo">356</span> if (delete == null) {<a name="line.356"></a> |
| <span class="sourceLineNo">357</span> delete = new Delete(key.get());<a name="line.357"></a> |
| <span class="sourceLineNo">358</span> }<a name="line.358"></a> |
| <span class="sourceLineNo">359</span> delete.add(kv);<a name="line.359"></a> |
| <span class="sourceLineNo">360</span> } else {<a name="line.360"></a> |
| <span class="sourceLineNo">361</span> if (put == null) {<a name="line.361"></a> |
| <span class="sourceLineNo">362</span> put = new Put(key.get());<a name="line.362"></a> |
| <span class="sourceLineNo">363</span> }<a name="line.363"></a> |
| <span class="sourceLineNo">364</span> addPutToKv(put, kv);<a name="line.364"></a> |
| <span class="sourceLineNo">365</span> }<a name="line.365"></a> |
| <span class="sourceLineNo">366</span> }<a name="line.366"></a> |
| <span class="sourceLineNo">367</span> if (put != null) {<a name="line.367"></a> |
| <span class="sourceLineNo">368</span> if (durability != null) {<a name="line.368"></a> |
| <span class="sourceLineNo">369</span> put.setDurability(durability);<a name="line.369"></a> |
| <span class="sourceLineNo">370</span> }<a name="line.370"></a> |
| <span class="sourceLineNo">371</span> put.setClusterIds(clusterIds);<a name="line.371"></a> |
| <span class="sourceLineNo">372</span> context.write(key, put);<a name="line.372"></a> |
| <span class="sourceLineNo">373</span> }<a name="line.373"></a> |
| <span class="sourceLineNo">374</span> if (delete != null) {<a name="line.374"></a> |
| <span class="sourceLineNo">375</span> if (durability != null) {<a name="line.375"></a> |
| <span class="sourceLineNo">376</span> delete.setDurability(durability);<a name="line.376"></a> |
| <span class="sourceLineNo">377</span> }<a name="line.377"></a> |
| <span class="sourceLineNo">378</span> delete.setClusterIds(clusterIds);<a name="line.378"></a> |
| <span class="sourceLineNo">379</span> context.write(key, delete);<a name="line.379"></a> |
| <span class="sourceLineNo">380</span> }<a name="line.380"></a> |
| <span class="sourceLineNo">381</span> }<a name="line.381"></a> |
| <span class="sourceLineNo">382</span><a name="line.382"></a> |
| <span class="sourceLineNo">383</span> protected void addPutToKv(Put put, Cell kv) throws IOException {<a name="line.383"></a> |
| <span class="sourceLineNo">384</span> put.add(kv);<a name="line.384"></a> |
| <span class="sourceLineNo">385</span> }<a name="line.385"></a> |
| <span class="sourceLineNo">386</span><a name="line.386"></a> |
| <span class="sourceLineNo">387</span> @Override<a name="line.387"></a> |
| <span class="sourceLineNo">388</span> public void setup(Context context) {<a name="line.388"></a> |
| <span class="sourceLineNo">389</span> LOG.info("Setting up " + getClass() + " mapper.");<a name="line.389"></a> |
| <span class="sourceLineNo">390</span> Configuration conf = context.getConfiguration();<a name="line.390"></a> |
| <span class="sourceLineNo">391</span> cfRenameMap = createCfRenameMap(conf);<a name="line.391"></a> |
| <span class="sourceLineNo">392</span> filter = instantiateFilter(conf);<a name="line.392"></a> |
| <span class="sourceLineNo">393</span> String durabilityStr = conf.get(WAL_DURABILITY);<a name="line.393"></a> |
| <span class="sourceLineNo">394</span> if (durabilityStr != null) {<a name="line.394"></a> |
| <span class="sourceLineNo">395</span> durability = Durability.valueOf(durabilityStr.toUpperCase(Locale.ROOT));<a name="line.395"></a> |
| <span class="sourceLineNo">396</span> LOG.info("setting WAL durability to " + durability);<a name="line.396"></a> |
| <span class="sourceLineNo">397</span> } else {<a name="line.397"></a> |
| <span class="sourceLineNo">398</span> LOG.info("setting WAL durability to default.");<a name="line.398"></a> |
| <span class="sourceLineNo">399</span> }<a name="line.399"></a> |
| <span class="sourceLineNo">400</span> // TODO: This is kind of ugly doing setup of ZKW just to read the clusterid.<a name="line.400"></a> |
| <span class="sourceLineNo">401</span> ZKWatcher zkw = null;<a name="line.401"></a> |
| <span class="sourceLineNo">402</span> Exception ex = null;<a name="line.402"></a> |
| <span class="sourceLineNo">403</span> try {<a name="line.403"></a> |
| <span class="sourceLineNo">404</span> zkw = new ZKWatcher(conf, context.getTaskAttemptID().toString(), null);<a name="line.404"></a> |
| <span class="sourceLineNo">405</span> clusterIds = Collections.singletonList(ZKClusterId.getUUIDForCluster(zkw));<a name="line.405"></a> |
| <span class="sourceLineNo">406</span> } catch (ZooKeeperConnectionException e) {<a name="line.406"></a> |
| <span class="sourceLineNo">407</span> ex = e;<a name="line.407"></a> |
| <span class="sourceLineNo">408</span> LOG.error("Problem connecting to ZooKeper during task setup", e);<a name="line.408"></a> |
| <span class="sourceLineNo">409</span> } catch (KeeperException e) {<a name="line.409"></a> |
| <span class="sourceLineNo">410</span> ex = e;<a name="line.410"></a> |
| <span class="sourceLineNo">411</span> LOG.error("Problem reading ZooKeeper data during task setup", e);<a name="line.411"></a> |
| <span class="sourceLineNo">412</span> } catch (IOException e) {<a name="line.412"></a> |
| <span class="sourceLineNo">413</span> ex = e;<a name="line.413"></a> |
| <span class="sourceLineNo">414</span> LOG.error("Problem setting up task", e);<a name="line.414"></a> |
| <span class="sourceLineNo">415</span> } finally {<a name="line.415"></a> |
| <span class="sourceLineNo">416</span> if (zkw != null) zkw.close();<a name="line.416"></a> |
| <span class="sourceLineNo">417</span> }<a name="line.417"></a> |
| <span class="sourceLineNo">418</span> if (clusterIds == null) {<a name="line.418"></a> |
| <span class="sourceLineNo">419</span> // exit early if setup fails<a name="line.419"></a> |
| <span class="sourceLineNo">420</span> throw new RuntimeException(ex);<a name="line.420"></a> |
| <span class="sourceLineNo">421</span> }<a name="line.421"></a> |
| <span class="sourceLineNo">422</span> }<a name="line.422"></a> |
| <span class="sourceLineNo">423</span> }<a name="line.423"></a> |
| <span class="sourceLineNo">424</span><a name="line.424"></a> |
| <span class="sourceLineNo">425</span> /**<a name="line.425"></a> |
| <span class="sourceLineNo">426</span> * Create a {@link Filter} to apply to all incoming keys ({@link KeyValue KeyValues}) to<a name="line.426"></a> |
| <span class="sourceLineNo">427</span> * optionally not include in the job output<a name="line.427"></a> |
| <span class="sourceLineNo">428</span> * @param conf {@link Configuration} from which to load the filter<a name="line.428"></a> |
| <span class="sourceLineNo">429</span> * @return the filter to use for the task, or <tt>null</tt> if no filter to should be used<a name="line.429"></a> |
| <span class="sourceLineNo">430</span> * @throws IllegalArgumentException if the filter is misconfigured<a name="line.430"></a> |
| <span class="sourceLineNo">431</span> */<a name="line.431"></a> |
| <span class="sourceLineNo">432</span> public static Filter instantiateFilter(Configuration conf) {<a name="line.432"></a> |
| <span class="sourceLineNo">433</span> // get the filter, if it was configured<a name="line.433"></a> |
| <span class="sourceLineNo">434</span> Class<? extends Filter> filterClass = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);<a name="line.434"></a> |
| <span class="sourceLineNo">435</span> if (filterClass == null) {<a name="line.435"></a> |
| <span class="sourceLineNo">436</span> LOG.debug("No configured filter class, accepting all keyvalues.");<a name="line.436"></a> |
| <span class="sourceLineNo">437</span> return null;<a name="line.437"></a> |
| <span class="sourceLineNo">438</span> }<a name="line.438"></a> |
| <span class="sourceLineNo">439</span> LOG.debug("Attempting to create filter:" + filterClass);<a name="line.439"></a> |
| <span class="sourceLineNo">440</span> String[] filterArgs = conf.getStrings(FILTER_ARGS_CONF_KEY);<a name="line.440"></a> |
| <span class="sourceLineNo">441</span> ArrayList<byte[]> quotedArgs = toQuotedByteArrays(filterArgs);<a name="line.441"></a> |
| <span class="sourceLineNo">442</span> try {<a name="line.442"></a> |
| <span class="sourceLineNo">443</span> Method m = filterClass.getMethod("createFilterFromArguments", ArrayList.class);<a name="line.443"></a> |
| <span class="sourceLineNo">444</span> return (Filter) m.invoke(null, quotedArgs);<a name="line.444"></a> |
| <span class="sourceLineNo">445</span> } catch (IllegalAccessException e) {<a name="line.445"></a> |
| <span class="sourceLineNo">446</span> LOG.error("Couldn't instantiate filter!", e);<a name="line.446"></a> |
| <span class="sourceLineNo">447</span> throw new RuntimeException(e);<a name="line.447"></a> |
| <span class="sourceLineNo">448</span> } catch (SecurityException e) {<a name="line.448"></a> |
| <span class="sourceLineNo">449</span> LOG.error("Couldn't instantiate filter!", e);<a name="line.449"></a> |
| <span class="sourceLineNo">450</span> throw new RuntimeException(e);<a name="line.450"></a> |
| <span class="sourceLineNo">451</span> } catch (NoSuchMethodException e) {<a name="line.451"></a> |
| <span class="sourceLineNo">452</span> LOG.error("Couldn't instantiate filter!", e);<a name="line.452"></a> |
| <span class="sourceLineNo">453</span> throw new RuntimeException(e);<a name="line.453"></a> |
| <span class="sourceLineNo">454</span> } catch (IllegalArgumentException e) {<a name="line.454"></a> |
| <span class="sourceLineNo">455</span> LOG.error("Couldn't instantiate filter!", e);<a name="line.455"></a> |
| <span class="sourceLineNo">456</span> throw new RuntimeException(e);<a name="line.456"></a> |
| <span class="sourceLineNo">457</span> } catch (InvocationTargetException e) {<a name="line.457"></a> |
| <span class="sourceLineNo">458</span> LOG.error("Couldn't instantiate filter!", e);<a name="line.458"></a> |
| <span class="sourceLineNo">459</span> throw new RuntimeException(e);<a name="line.459"></a> |
| <span class="sourceLineNo">460</span> }<a name="line.460"></a> |
| <span class="sourceLineNo">461</span> }<a name="line.461"></a> |
| <span class="sourceLineNo">462</span><a name="line.462"></a> |
| <span class="sourceLineNo">463</span> private static ArrayList<byte[]> toQuotedByteArrays(String... stringArgs) {<a name="line.463"></a> |
| <span class="sourceLineNo">464</span> ArrayList<byte[]> quotedArgs = new ArrayList<>();<a name="line.464"></a> |
| <span class="sourceLineNo">465</span> for (String stringArg : stringArgs) {<a name="line.465"></a> |
| <span class="sourceLineNo">466</span> // all the filters' instantiation methods expected quoted args since they are coming from<a name="line.466"></a> |
| <span class="sourceLineNo">467</span> // the shell, so add them here, though it shouldn't really be needed :-/<a name="line.467"></a> |
| <span class="sourceLineNo">468</span> quotedArgs.add(Bytes.toBytes("'" + stringArg + "'"));<a name="line.468"></a> |
| <span class="sourceLineNo">469</span> }<a name="line.469"></a> |
| <span class="sourceLineNo">470</span> return quotedArgs;<a name="line.470"></a> |
| <span class="sourceLineNo">471</span> }<a name="line.471"></a> |
| <span class="sourceLineNo">472</span><a name="line.472"></a> |
| <span class="sourceLineNo">473</span> /**<a name="line.473"></a> |
| <span class="sourceLineNo">474</span> * Attempt to filter out the keyvalue<a name="line.474"></a> |
| <span class="sourceLineNo">475</span> * @param c {@link Cell} on which to apply the filter<a name="line.475"></a> |
| <span class="sourceLineNo">476</span> * @return <tt>null</tt> if the key should not be written, otherwise returns the original<a name="line.476"></a> |
| <span class="sourceLineNo">477</span> * {@link Cell}<a name="line.477"></a> |
| <span class="sourceLineNo">478</span> */<a name="line.478"></a> |
| <span class="sourceLineNo">479</span> public static Cell filterKv(Filter filter, Cell c) throws IOException {<a name="line.479"></a> |
| <span class="sourceLineNo">480</span> // apply the filter and skip this kv if the filter doesn't apply<a name="line.480"></a> |
| <span class="sourceLineNo">481</span> if (filter != null) {<a name="line.481"></a> |
| <span class="sourceLineNo">482</span> Filter.ReturnCode code = filter.filterCell(c);<a name="line.482"></a> |
| <span class="sourceLineNo">483</span> if (LOG.isTraceEnabled()) {<a name="line.483"></a> |
| <span class="sourceLineNo">484</span> LOG.trace("Filter returned:" + code + " for the cell:" + c);<a name="line.484"></a> |
| <span class="sourceLineNo">485</span> }<a name="line.485"></a> |
| <span class="sourceLineNo">486</span> // if its not an accept type, then skip this kv<a name="line.486"></a> |
| <span class="sourceLineNo">487</span> if (<a name="line.487"></a> |
| <span class="sourceLineNo">488</span> !(code.equals(Filter.ReturnCode.INCLUDE)<a name="line.488"></a> |
| <span class="sourceLineNo">489</span> || code.equals(Filter.ReturnCode.INCLUDE_AND_NEXT_COL))<a name="line.489"></a> |
| <span class="sourceLineNo">490</span> ) {<a name="line.490"></a> |
| <span class="sourceLineNo">491</span> return null;<a name="line.491"></a> |
| <span class="sourceLineNo">492</span> }<a name="line.492"></a> |
| <span class="sourceLineNo">493</span> }<a name="line.493"></a> |
| <span class="sourceLineNo">494</span> return c;<a name="line.494"></a> |
| <span class="sourceLineNo">495</span> }<a name="line.495"></a> |
| <span class="sourceLineNo">496</span><a name="line.496"></a> |
| <span class="sourceLineNo">497</span> // helper: create a new KeyValue based on CF rename map<a name="line.497"></a> |
| <span class="sourceLineNo">498</span> private static Cell convertKv(Cell kv, Map<byte[], byte[]> cfRenameMap) {<a name="line.498"></a> |
| <span class="sourceLineNo">499</span> if (cfRenameMap != null) {<a name="line.499"></a> |
| <span class="sourceLineNo">500</span> // If there's a rename mapping for this CF, create a new KeyValue<a name="line.500"></a> |
| <span class="sourceLineNo">501</span> byte[] newCfName = cfRenameMap.get(CellUtil.cloneFamily(kv));<a name="line.501"></a> |
| <span class="sourceLineNo">502</span> if (newCfName != null) {<a name="line.502"></a> |
| <span class="sourceLineNo">503</span> List<Tag> tags = PrivateCellUtil.getTags(kv);<a name="line.503"></a> |
| <span class="sourceLineNo">504</span> kv = new KeyValue(kv.getRowArray(), // row buffer<a name="line.504"></a> |
| <span class="sourceLineNo">505</span> kv.getRowOffset(), // row offset<a name="line.505"></a> |
| <span class="sourceLineNo">506</span> kv.getRowLength(), // row length<a name="line.506"></a> |
| <span class="sourceLineNo">507</span> newCfName, // CF buffer<a name="line.507"></a> |
| <span class="sourceLineNo">508</span> 0, // CF offset<a name="line.508"></a> |
| <span class="sourceLineNo">509</span> newCfName.length, // CF length<a name="line.509"></a> |
| <span class="sourceLineNo">510</span> kv.getQualifierArray(), // qualifier buffer<a name="line.510"></a> |
| <span class="sourceLineNo">511</span> kv.getQualifierOffset(), // qualifier offset<a name="line.511"></a> |
| <span class="sourceLineNo">512</span> kv.getQualifierLength(), // qualifier length<a name="line.512"></a> |
| <span class="sourceLineNo">513</span> kv.getTimestamp(), // timestamp<a name="line.513"></a> |
| <span class="sourceLineNo">514</span> KeyValue.Type.codeToType(kv.getTypeByte()), // KV Type<a name="line.514"></a> |
| <span class="sourceLineNo">515</span> kv.getValueArray(), // value buffer<a name="line.515"></a> |
| <span class="sourceLineNo">516</span> kv.getValueOffset(), // value offset<a name="line.516"></a> |
| <span class="sourceLineNo">517</span> kv.getValueLength(), // value length<a name="line.517"></a> |
| <span class="sourceLineNo">518</span> tags.size() == 0 ? null : tags);<a name="line.518"></a> |
| <span class="sourceLineNo">519</span> }<a name="line.519"></a> |
| <span class="sourceLineNo">520</span> }<a name="line.520"></a> |
| <span class="sourceLineNo">521</span> return kv;<a name="line.521"></a> |
| <span class="sourceLineNo">522</span> }<a name="line.522"></a> |
| <span class="sourceLineNo">523</span><a name="line.523"></a> |
| <span class="sourceLineNo">524</span> // helper: make a map from sourceCfName to destCfName by parsing a config key<a name="line.524"></a> |
| <span class="sourceLineNo">525</span> private static Map<byte[], byte[]> createCfRenameMap(Configuration conf) {<a name="line.525"></a> |
| <span class="sourceLineNo">526</span> Map<byte[], byte[]> cfRenameMap = null;<a name="line.526"></a> |
| <span class="sourceLineNo">527</span> String allMappingsPropVal = conf.get(CF_RENAME_PROP);<a name="line.527"></a> |
| <span class="sourceLineNo">528</span> if (allMappingsPropVal != null) {<a name="line.528"></a> |
| <span class="sourceLineNo">529</span> // The conf value format should be sourceCf1:destCf1,sourceCf2:destCf2,...<a name="line.529"></a> |
| <span class="sourceLineNo">530</span> String[] allMappings = allMappingsPropVal.split(",");<a name="line.530"></a> |
| <span class="sourceLineNo">531</span> for (String mapping : allMappings) {<a name="line.531"></a> |
| <span class="sourceLineNo">532</span> if (cfRenameMap == null) {<a name="line.532"></a> |
| <span class="sourceLineNo">533</span> cfRenameMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);<a name="line.533"></a> |
| <span class="sourceLineNo">534</span> }<a name="line.534"></a> |
| <span class="sourceLineNo">535</span> String[] srcAndDest = mapping.split(":");<a name="line.535"></a> |
| <span class="sourceLineNo">536</span> if (srcAndDest.length != 2) {<a name="line.536"></a> |
| <span class="sourceLineNo">537</span> continue;<a name="line.537"></a> |
| <span class="sourceLineNo">538</span> }<a name="line.538"></a> |
| <span class="sourceLineNo">539</span> cfRenameMap.put(Bytes.toBytes(srcAndDest[0]), Bytes.toBytes(srcAndDest[1]));<a name="line.539"></a> |
| <span class="sourceLineNo">540</span> }<a name="line.540"></a> |
| <span class="sourceLineNo">541</span> }<a name="line.541"></a> |
| <span class="sourceLineNo">542</span> return cfRenameMap;<a name="line.542"></a> |
| <span class="sourceLineNo">543</span> }<a name="line.543"></a> |
| <span class="sourceLineNo">544</span><a name="line.544"></a> |
| <span class="sourceLineNo">545</span> /**<a name="line.545"></a> |
| <span class="sourceLineNo">546</span> * <p><a name="line.546"></a> |
| <span class="sourceLineNo">547</span> * Sets a configuration property with key {@link #CF_RENAME_PROP} in conf that tells the mapper<a name="line.547"></a> |
| <span class="sourceLineNo">548</span> * how to rename column families.<a name="line.548"></a> |
| <span class="sourceLineNo">549</span> * <p><a name="line.549"></a> |
| <span class="sourceLineNo">550</span> * Alternately, instead of calling this function, you could set the configuration key<a name="line.550"></a> |
| <span class="sourceLineNo">551</span> * {@link #CF_RENAME_PROP} yourself. The value should look like<a name="line.551"></a> |
| <span class="sourceLineNo">552</span> *<a name="line.552"></a> |
| <span class="sourceLineNo">553</span> * <pre><a name="line.553"></a> |
| <span class="sourceLineNo">554</span> * srcCf1:destCf1,srcCf2:destCf2,....<a name="line.554"></a> |
| <span class="sourceLineNo">555</span> * </pre><a name="line.555"></a> |
| <span class="sourceLineNo">556</span> *<a name="line.556"></a> |
| <span class="sourceLineNo">557</span> * . This would have the same effect on the mapper behavior.<a name="line.557"></a> |
| <span class="sourceLineNo">558</span> * @param conf the Configuration in which the {@link #CF_RENAME_PROP} key will be set<a name="line.558"></a> |
| <span class="sourceLineNo">559</span> * @param renameMap a mapping from source CF names to destination CF names<a name="line.559"></a> |
| <span class="sourceLineNo">560</span> */<a name="line.560"></a> |
| <span class="sourceLineNo">561</span> static public void configureCfRenaming(Configuration conf, Map<String, String> renameMap) {<a name="line.561"></a> |
| <span class="sourceLineNo">562</span> StringBuilder sb = new StringBuilder();<a name="line.562"></a> |
| <span class="sourceLineNo">563</span> for (Map.Entry<String, String> entry : renameMap.entrySet()) {<a name="line.563"></a> |
| <span class="sourceLineNo">564</span> String sourceCf = entry.getKey();<a name="line.564"></a> |
| <span class="sourceLineNo">565</span> String destCf = entry.getValue();<a name="line.565"></a> |
| <span class="sourceLineNo">566</span><a name="line.566"></a> |
| <span class="sourceLineNo">567</span> if (<a name="line.567"></a> |
| <span class="sourceLineNo">568</span> sourceCf.contains(":") || sourceCf.contains(",") || destCf.contains(":")<a name="line.568"></a> |
| <span class="sourceLineNo">569</span> || destCf.contains(",")<a name="line.569"></a> |
| <span class="sourceLineNo">570</span> ) {<a name="line.570"></a> |
| <span class="sourceLineNo">571</span> throw new IllegalArgumentException(<a name="line.571"></a> |
| <span class="sourceLineNo">572</span> "Illegal character in CF names: " + sourceCf + ", " + destCf);<a name="line.572"></a> |
| <span class="sourceLineNo">573</span> }<a name="line.573"></a> |
| <span class="sourceLineNo">574</span><a name="line.574"></a> |
| <span class="sourceLineNo">575</span> if (sb.length() != 0) {<a name="line.575"></a> |
| <span class="sourceLineNo">576</span> sb.append(",");<a name="line.576"></a> |
| <span class="sourceLineNo">577</span> }<a name="line.577"></a> |
| <span class="sourceLineNo">578</span> sb.append(sourceCf + ":" + destCf);<a name="line.578"></a> |
| <span class="sourceLineNo">579</span> }<a name="line.579"></a> |
| <span class="sourceLineNo">580</span> conf.set(CF_RENAME_PROP, sb.toString());<a name="line.580"></a> |
| <span class="sourceLineNo">581</span> }<a name="line.581"></a> |
| <span class="sourceLineNo">582</span><a name="line.582"></a> |
| <span class="sourceLineNo">583</span> /**<a name="line.583"></a> |
| <span class="sourceLineNo">584</span> * Add a Filter to be instantiated on import<a name="line.584"></a> |
| <span class="sourceLineNo">585</span> * @param conf Configuration to update (will be passed to the job)<a name="line.585"></a> |
| <span class="sourceLineNo">586</span> * @param clazz {@link Filter} subclass to instantiate on the server.<a name="line.586"></a> |
| <span class="sourceLineNo">587</span> * @param filterArgs List of arguments to pass to the filter on instantiation<a name="line.587"></a> |
| <span class="sourceLineNo">588</span> */<a name="line.588"></a> |
| <span class="sourceLineNo">589</span> public static void addFilterAndArguments(Configuration conf, Class<? extends Filter> clazz,<a name="line.589"></a> |
| <span class="sourceLineNo">590</span> List<String> filterArgs) throws IOException {<a name="line.590"></a> |
| <span class="sourceLineNo">591</span> conf.set(Import.FILTER_CLASS_CONF_KEY, clazz.getName());<a name="line.591"></a> |
| <span class="sourceLineNo">592</span> conf.setStrings(Import.FILTER_ARGS_CONF_KEY, filterArgs.toArray(new String[filterArgs.size()]));<a name="line.592"></a> |
| <span class="sourceLineNo">593</span> }<a name="line.593"></a> |
| <span class="sourceLineNo">594</span><a name="line.594"></a> |
| <span class="sourceLineNo">595</span> /**<a name="line.595"></a> |
| <span class="sourceLineNo">596</span> * Sets up the actual job.<a name="line.596"></a> |
| <span class="sourceLineNo">597</span> * @param conf The current configuration.<a name="line.597"></a> |
| <span class="sourceLineNo">598</span> * @param args The command line parameters.<a name="line.598"></a> |
| <span class="sourceLineNo">599</span> * @return The newly created job.<a name="line.599"></a> |
| <span class="sourceLineNo">600</span> * @throws IOException When setting up the job fails.<a name="line.600"></a> |
| <span class="sourceLineNo">601</span> */<a name="line.601"></a> |
| <span class="sourceLineNo">602</span> public static Job createSubmittableJob(Configuration conf, String[] args) throws IOException {<a name="line.602"></a> |
| <span class="sourceLineNo">603</span> TableName tableName = TableName.valueOf(args[0]);<a name="line.603"></a> |
| <span class="sourceLineNo">604</span> conf.set(TABLE_NAME, tableName.getNameAsString());<a name="line.604"></a> |
| <span class="sourceLineNo">605</span> Path inputDir = new Path(args[1]);<a name="line.605"></a> |
| <span class="sourceLineNo">606</span> Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));<a name="line.606"></a> |
| <span class="sourceLineNo">607</span> job.setJarByClass(Importer.class);<a name="line.607"></a> |
| <span class="sourceLineNo">608</span> FileInputFormat.setInputPaths(job, inputDir);<a name="line.608"></a> |
| <span class="sourceLineNo">609</span> job.setInputFormatClass(SequenceFileInputFormat.class);<a name="line.609"></a> |
| <span class="sourceLineNo">610</span> String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);<a name="line.610"></a> |
| <span class="sourceLineNo">611</span><a name="line.611"></a> |
| <span class="sourceLineNo">612</span> // make sure we get the filter in the jars<a name="line.612"></a> |
| <span class="sourceLineNo">613</span> try {<a name="line.613"></a> |
| <span class="sourceLineNo">614</span> Class<? extends Filter> filter = conf.getClass(FILTER_CLASS_CONF_KEY, null, Filter.class);<a name="line.614"></a> |
| <span class="sourceLineNo">615</span> if (filter != null) {<a name="line.615"></a> |
| <span class="sourceLineNo">616</span> TableMapReduceUtil.addDependencyJarsForClasses(conf, filter);<a name="line.616"></a> |
| <span class="sourceLineNo">617</span> }<a name="line.617"></a> |
| <span class="sourceLineNo">618</span> } catch (Exception e) {<a name="line.618"></a> |
| <span class="sourceLineNo">619</span> throw new IOException(e);<a name="line.619"></a> |
| <span class="sourceLineNo">620</span> }<a name="line.620"></a> |
| <span class="sourceLineNo">621</span><a name="line.621"></a> |
| <span class="sourceLineNo">622</span> if (hfileOutPath != null && conf.getBoolean(HAS_LARGE_RESULT, false)) {<a name="line.622"></a> |
| <span class="sourceLineNo">623</span> LOG.info("Use Large Result!!");<a name="line.623"></a> |
| <span class="sourceLineNo">624</span> try (Connection conn = ConnectionFactory.createConnection(conf);<a name="line.624"></a> |
| <span class="sourceLineNo">625</span> Table table = conn.getTable(tableName);<a name="line.625"></a> |
| <span class="sourceLineNo">626</span> RegionLocator regionLocator = conn.getRegionLocator(tableName)) {<a name="line.626"></a> |
| <span class="sourceLineNo">627</span> HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);<a name="line.627"></a> |
| <span class="sourceLineNo">628</span> job.setMapperClass(CellSortImporter.class);<a name="line.628"></a> |
| <span class="sourceLineNo">629</span> job.setReducerClass(CellReducer.class);<a name="line.629"></a> |
| <span class="sourceLineNo">630</span> Path outputDir = new Path(hfileOutPath);<a name="line.630"></a> |
| <span class="sourceLineNo">631</span> FileOutputFormat.setOutputPath(job, outputDir);<a name="line.631"></a> |
| <span class="sourceLineNo">632</span> job.setMapOutputKeyClass(CellWritableComparable.class);<a name="line.632"></a> |
| <span class="sourceLineNo">633</span> job.setMapOutputValueClass(MapReduceExtendedCell.class);<a name="line.633"></a> |
| <span class="sourceLineNo">634</span> job.getConfiguration().setClass("mapreduce.job.output.key.comparator.class",<a name="line.634"></a> |
| <span class="sourceLineNo">635</span> CellWritableComparable.CellWritableComparator.class, RawComparator.class);<a name="line.635"></a> |
| <span class="sourceLineNo">636</span> Path partitionsPath =<a name="line.636"></a> |
| <span class="sourceLineNo">637</span> new Path(TotalOrderPartitioner.getPartitionFile(job.getConfiguration()));<a name="line.637"></a> |
| <span class="sourceLineNo">638</span> FileSystem fs = FileSystem.get(job.getConfiguration());<a name="line.638"></a> |
| <span class="sourceLineNo">639</span> fs.deleteOnExit(partitionsPath);<a name="line.639"></a> |
| <span class="sourceLineNo">640</span> job.setPartitionerClass(CellWritableComparablePartitioner.class);<a name="line.640"></a> |
| <span class="sourceLineNo">641</span> job.setNumReduceTasks(regionLocator.getStartKeys().length);<a name="line.641"></a> |
| <span class="sourceLineNo">642</span> TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),<a name="line.642"></a> |
| <span class="sourceLineNo">643</span> org.apache.hbase.thirdparty.com.google.common.base.Preconditions.class);<a name="line.643"></a> |
| <span class="sourceLineNo">644</span> }<a name="line.644"></a> |
| <span class="sourceLineNo">645</span> } else if (hfileOutPath != null) {<a name="line.645"></a> |
| <span class="sourceLineNo">646</span> LOG.info("writing to hfiles for bulk load.");<a name="line.646"></a> |
| <span class="sourceLineNo">647</span> job.setMapperClass(CellImporter.class);<a name="line.647"></a> |
| <span class="sourceLineNo">648</span> try (Connection conn = ConnectionFactory.createConnection(conf);<a name="line.648"></a> |
| <span class="sourceLineNo">649</span> Table table = conn.getTable(tableName);<a name="line.649"></a> |
| <span class="sourceLineNo">650</span> RegionLocator regionLocator = conn.getRegionLocator(tableName)) {<a name="line.650"></a> |
| <span class="sourceLineNo">651</span> job.setReducerClass(CellSortReducer.class);<a name="line.651"></a> |
| <span class="sourceLineNo">652</span> Path outputDir = new Path(hfileOutPath);<a name="line.652"></a> |
| <span class="sourceLineNo">653</span> FileOutputFormat.setOutputPath(job, outputDir);<a name="line.653"></a> |
| <span class="sourceLineNo">654</span> job.setMapOutputKeyClass(ImmutableBytesWritable.class);<a name="line.654"></a> |
| <span class="sourceLineNo">655</span> job.setMapOutputValueClass(MapReduceExtendedCell.class);<a name="line.655"></a> |
| <span class="sourceLineNo">656</span> HFileOutputFormat2.configureIncrementalLoad(job, table.getDescriptor(), regionLocator);<a name="line.656"></a> |
| <span class="sourceLineNo">657</span> TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),<a name="line.657"></a> |
| <span class="sourceLineNo">658</span> org.apache.hbase.thirdparty.com.google.common.base.Preconditions.class);<a name="line.658"></a> |
| <span class="sourceLineNo">659</span> }<a name="line.659"></a> |
| <span class="sourceLineNo">660</span> } else {<a name="line.660"></a> |
| <span class="sourceLineNo">661</span> LOG.info("writing directly to table from Mapper.");<a name="line.661"></a> |
| <span class="sourceLineNo">662</span> // No reducers. Just write straight to table. Call initTableReducerJob<a name="line.662"></a> |
| <span class="sourceLineNo">663</span> // because it sets up the TableOutputFormat.<a name="line.663"></a> |
| <span class="sourceLineNo">664</span> job.setMapperClass(Importer.class);<a name="line.664"></a> |
| <span class="sourceLineNo">665</span> TableMapReduceUtil.initTableReducerJob(tableName.getNameAsString(), null, job);<a name="line.665"></a> |
| <span class="sourceLineNo">666</span> job.setNumReduceTasks(0);<a name="line.666"></a> |
| <span class="sourceLineNo">667</span> }<a name="line.667"></a> |
| <span class="sourceLineNo">668</span> return job;<a name="line.668"></a> |
| <span class="sourceLineNo">669</span> }<a name="line.669"></a> |
| <span class="sourceLineNo">670</span><a name="line.670"></a> |
| <span class="sourceLineNo">671</span> /*<a name="line.671"></a> |
| <span class="sourceLineNo">672</span> * @param errorMsg Error message. Can be null.<a name="line.672"></a> |
| <span class="sourceLineNo">673</span> */<a name="line.673"></a> |
| <span class="sourceLineNo">674</span> private static void usage(final String errorMsg) {<a name="line.674"></a> |
| <span class="sourceLineNo">675</span> if (errorMsg != null && errorMsg.length() > 0) {<a name="line.675"></a> |
| <span class="sourceLineNo">676</span> System.err.println("ERROR: " + errorMsg);<a name="line.676"></a> |
| <span class="sourceLineNo">677</span> }<a name="line.677"></a> |
| <span class="sourceLineNo">678</span> System.err.println("Usage: Import [options] <tablename> <inputdir>");<a name="line.678"></a> |
| <span class="sourceLineNo">679</span> System.err.println("By default Import will load data directly into HBase. To instead generate");<a name="line.679"></a> |
| <span class="sourceLineNo">680</span> System.err.println("HFiles of data to prepare for a bulk data load, pass the option:");<a name="line.680"></a> |
| <span class="sourceLineNo">681</span> System.err.println(" -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");<a name="line.681"></a> |
| <span class="sourceLineNo">682</span> System.err.println("If there is a large result that includes too much Cell "<a name="line.682"></a> |
| <span class="sourceLineNo">683</span> + "whitch can occur OOME caused by the memery sort in reducer, pass the option:");<a name="line.683"></a> |
| <span class="sourceLineNo">684</span> System.err.println(" -D" + HAS_LARGE_RESULT + "=true");<a name="line.684"></a> |
| <span class="sourceLineNo">685</span> System.err<a name="line.685"></a> |
| <span class="sourceLineNo">686</span> .println(" To apply a generic org.apache.hadoop.hbase.filter.Filter to the input, use");<a name="line.686"></a> |
| <span class="sourceLineNo">687</span> System.err.println(" -D" + FILTER_CLASS_CONF_KEY + "=<name of filter class>");<a name="line.687"></a> |
| <span class="sourceLineNo">688</span> System.err.println(" -D" + FILTER_ARGS_CONF_KEY + "=<comma separated list of args for filter");<a name="line.688"></a> |
| <span class="sourceLineNo">689</span> System.err.println(" NOTE: The filter will be applied BEFORE doing key renames via the "<a name="line.689"></a> |
| <span class="sourceLineNo">690</span> + CF_RENAME_PROP + " property. Futher, filters will only use the"<a name="line.690"></a> |
| <span class="sourceLineNo">691</span> + " Filter#filterRowKey(byte[] buffer, int offset, int length) method to identify "<a name="line.691"></a> |
| <span class="sourceLineNo">692</span> + " whether the current row needs to be ignored completely for processing and "<a name="line.692"></a> |
| <span class="sourceLineNo">693</span> + " Filter#filterCell(Cell) method to determine if the Cell should be added;"<a name="line.693"></a> |
| <span class="sourceLineNo">694</span> + " Filter.ReturnCode#INCLUDE and #INCLUDE_AND_NEXT_COL will be considered as including"<a name="line.694"></a> |
| <span class="sourceLineNo">695</span> + " the Cell.");<a name="line.695"></a> |
| <span class="sourceLineNo">696</span> System.err.println("To import data exported from HBase 0.94, use");<a name="line.696"></a> |
| <span class="sourceLineNo">697</span> System.err.println(" -Dhbase.import.version=0.94");<a name="line.697"></a> |
| <span class="sourceLineNo">698</span> System.err.println(" -D " + JOB_NAME_CONF_KEY<a name="line.698"></a> |
| <span class="sourceLineNo">699</span> + "=jobName - use the specified mapreduce job name for the import");<a name="line.699"></a> |
| <span class="sourceLineNo">700</span> System.err.println("For performance consider the following options:\n"<a name="line.700"></a> |
| <span class="sourceLineNo">701</span> + " -Dmapreduce.map.speculative=false\n" + " -Dmapreduce.reduce.speculative=false\n"<a name="line.701"></a> |
| <span class="sourceLineNo">702</span> + " -D" + WAL_DURABILITY + "=<Used while writing data to hbase."<a name="line.702"></a> |
| <span class="sourceLineNo">703</span> + " Allowed values are the supported durability values"<a name="line.703"></a> |
| <span class="sourceLineNo">704</span> + " like SKIP_WAL/ASYNC_WAL/SYNC_WAL/...>");<a name="line.704"></a> |
| <span class="sourceLineNo">705</span> }<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> * If the durability is set to {@link Durability#SKIP_WAL} and the data is imported to hbase, we<a name="line.708"></a> |
| <span class="sourceLineNo">709</span> * need to flush all the regions of the table as the data is held in memory and is also not<a name="line.709"></a> |
| <span class="sourceLineNo">710</span> * present in the Write Ahead Log to replay in scenarios of a crash. This method flushes all the<a name="line.710"></a> |
| <span class="sourceLineNo">711</span> * regions of the table in the scenarios of import data to hbase with {@link Durability#SKIP_WAL}<a name="line.711"></a> |
| <span class="sourceLineNo">712</span> */<a name="line.712"></a> |
| <span class="sourceLineNo">713</span> public static void flushRegionsIfNecessary(Configuration conf)<a name="line.713"></a> |
| <span class="sourceLineNo">714</span> throws IOException, InterruptedException {<a name="line.714"></a> |
| <span class="sourceLineNo">715</span> String tableName = conf.get(TABLE_NAME);<a name="line.715"></a> |
| <span class="sourceLineNo">716</span> Admin hAdmin = null;<a name="line.716"></a> |
| <span class="sourceLineNo">717</span> Connection connection = null;<a name="line.717"></a> |
| <span class="sourceLineNo">718</span> String durability = conf.get(WAL_DURABILITY);<a name="line.718"></a> |
| <span class="sourceLineNo">719</span> // Need to flush if the data is written to hbase and skip wal is enabled.<a name="line.719"></a> |
| <span class="sourceLineNo">720</span> if (<a name="line.720"></a> |
| <span class="sourceLineNo">721</span> conf.get(BULK_OUTPUT_CONF_KEY) == null && durability != null<a name="line.721"></a> |
| <span class="sourceLineNo">722</span> && Durability.SKIP_WAL.name().equalsIgnoreCase(durability)<a name="line.722"></a> |
| <span class="sourceLineNo">723</span> ) {<a name="line.723"></a> |
| <span class="sourceLineNo">724</span> LOG.info("Flushing all data that skipped the WAL.");<a name="line.724"></a> |
| <span class="sourceLineNo">725</span> try {<a name="line.725"></a> |
| <span class="sourceLineNo">726</span> connection = ConnectionFactory.createConnection(conf);<a name="line.726"></a> |
| <span class="sourceLineNo">727</span> hAdmin = connection.getAdmin();<a name="line.727"></a> |
| <span class="sourceLineNo">728</span> hAdmin.flush(TableName.valueOf(tableName));<a name="line.728"></a> |
| <span class="sourceLineNo">729</span> } finally {<a name="line.729"></a> |
| <span class="sourceLineNo">730</span> if (hAdmin != null) {<a name="line.730"></a> |
| <span class="sourceLineNo">731</span> hAdmin.close();<a name="line.731"></a> |
| <span class="sourceLineNo">732</span> }<a name="line.732"></a> |
| <span class="sourceLineNo">733</span> if (connection != null) {<a name="line.733"></a> |
| <span class="sourceLineNo">734</span> connection.close();<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> }<a name="line.738"></a> |
| <span class="sourceLineNo">739</span><a name="line.739"></a> |
| <span class="sourceLineNo">740</span> @Override<a name="line.740"></a> |
| <span class="sourceLineNo">741</span> public int run(String[] args) throws Exception {<a name="line.741"></a> |
| <span class="sourceLineNo">742</span> if (args.length < 2) {<a name="line.742"></a> |
| <span class="sourceLineNo">743</span> usage("Wrong number of arguments: " + args.length);<a name="line.743"></a> |
| <span class="sourceLineNo">744</span> return -1;<a name="line.744"></a> |
| <span class="sourceLineNo">745</span> }<a name="line.745"></a> |
| <span class="sourceLineNo">746</span> String inputVersionString = System.getProperty(ResultSerialization.IMPORT_FORMAT_VER);<a name="line.746"></a> |
| <span class="sourceLineNo">747</span> if (inputVersionString != null) {<a name="line.747"></a> |
| <span class="sourceLineNo">748</span> getConf().set(ResultSerialization.IMPORT_FORMAT_VER, inputVersionString);<a name="line.748"></a> |
| <span class="sourceLineNo">749</span> }<a name="line.749"></a> |
| <span class="sourceLineNo">750</span> Job job = createSubmittableJob(getConf(), args);<a name="line.750"></a> |
| <span class="sourceLineNo">751</span> boolean isJobSuccessful = job.waitForCompletion(true);<a name="line.751"></a> |
| <span class="sourceLineNo">752</span> if (isJobSuccessful) {<a name="line.752"></a> |
| <span class="sourceLineNo">753</span> // Flush all the regions of the table<a name="line.753"></a> |
| <span class="sourceLineNo">754</span> flushRegionsIfNecessary(getConf());<a name="line.754"></a> |
| <span class="sourceLineNo">755</span> }<a name="line.755"></a> |
| <span class="sourceLineNo">756</span> long inputRecords = job.getCounters().findCounter(TaskCounter.MAP_INPUT_RECORDS).getValue();<a name="line.756"></a> |
| <span class="sourceLineNo">757</span> long outputRecords = job.getCounters().findCounter(TaskCounter.MAP_OUTPUT_RECORDS).getValue();<a name="line.757"></a> |
| <span class="sourceLineNo">758</span> if (outputRecords < inputRecords) {<a name="line.758"></a> |
| <span class="sourceLineNo">759</span> System.err.println("Warning, not all records were imported (maybe filtered out).");<a name="line.759"></a> |
| <span class="sourceLineNo">760</span> if (outputRecords == 0) {<a name="line.760"></a> |
| <span class="sourceLineNo">761</span> System.err.println("If the data was exported from HBase 0.94 "<a name="line.761"></a> |
| <span class="sourceLineNo">762</span> + "consider using -Dhbase.import.version=0.94.");<a name="line.762"></a> |
| <span class="sourceLineNo">763</span> }<a name="line.763"></a> |
| <span class="sourceLineNo">764</span> }<a name="line.764"></a> |
| <span class="sourceLineNo">765</span><a name="line.765"></a> |
| <span class="sourceLineNo">766</span> return (isJobSuccessful ? 0 : 1);<a name="line.766"></a> |
| <span class="sourceLineNo">767</span> }<a name="line.767"></a> |
| <span class="sourceLineNo">768</span><a name="line.768"></a> |
| <span class="sourceLineNo">769</span> /**<a name="line.769"></a> |
| <span class="sourceLineNo">770</span> * Main entry point.<a name="line.770"></a> |
| <span class="sourceLineNo">771</span> * @param args The command line parameters.<a name="line.771"></a> |
| <span class="sourceLineNo">772</span> * @throws Exception When running the job fails.<a name="line.772"></a> |
| <span class="sourceLineNo">773</span> */<a name="line.773"></a> |
| <span class="sourceLineNo">774</span> public static void main(String[] args) throws Exception {<a name="line.774"></a> |
| <span class="sourceLineNo">775</span> int errCode = ToolRunner.run(HBaseConfiguration.create(), new Import(), args);<a name="line.775"></a> |
| <span class="sourceLineNo">776</span> System.exit(errCode);<a name="line.776"></a> |
| <span class="sourceLineNo">777</span> }<a name="line.777"></a> |
| <span class="sourceLineNo">778</span><a name="line.778"></a> |
| <span class="sourceLineNo">779</span>}<a name="line.779"></a> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </pre> |
| </div> |
| </body> |
| </html> |