blob: 59b04412bad99cd9e8c97f1b6d17e20c2d7b9a6a [file] [log] [blame]
<!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.IOException;<a name="line.20"></a>
<span class="sourceLineNo">021</span>import java.util.ArrayList;<a name="line.21"></a>
<span class="sourceLineNo">022</span>import java.util.HashMap;<a name="line.22"></a>
<span class="sourceLineNo">023</span>import java.util.Iterator;<a name="line.23"></a>
<span class="sourceLineNo">024</span>import java.util.List;<a name="line.24"></a>
<span class="sourceLineNo">025</span>import java.util.Map;<a name="line.25"></a>
<span class="sourceLineNo">026</span>import org.apache.commons.lang3.StringUtils;<a name="line.26"></a>
<span class="sourceLineNo">027</span>import org.apache.hadoop.conf.Configuration;<a name="line.27"></a>
<span class="sourceLineNo">028</span>import org.apache.hadoop.hbase.HBaseConfiguration;<a name="line.28"></a>
<span class="sourceLineNo">029</span>import org.apache.hadoop.hbase.HConstants;<a name="line.29"></a>
<span class="sourceLineNo">030</span>import org.apache.hadoop.mapreduce.InputSplit;<a name="line.30"></a>
<span class="sourceLineNo">031</span>import org.apache.hadoop.mapreduce.JobContext;<a name="line.31"></a>
<span class="sourceLineNo">032</span>import org.apache.hadoop.mapreduce.JobID;<a name="line.32"></a>
<span class="sourceLineNo">033</span>import org.apache.hadoop.mapreduce.task.JobContextImpl;<a name="line.33"></a>
<span class="sourceLineNo">034</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.34"></a>
<span class="sourceLineNo">035</span><a name="line.35"></a>
<span class="sourceLineNo">036</span>/**<a name="line.36"></a>
<span class="sourceLineNo">037</span> * Process the return from super-class {@link TableInputFormat} (TIF) so as to undo any clumping of<a name="line.37"></a>
<span class="sourceLineNo">038</span> * {@link InputSplit}s around RegionServers. Spread splits broadly to distribute read-load over<a name="line.38"></a>
<span class="sourceLineNo">039</span> * RegionServers in the cluster. The super-class TIF returns splits in hbase:meta table order.<a name="line.39"></a>
<span class="sourceLineNo">040</span> * Adjacent or near-adjacent hbase:meta Regions can be hosted on the same RegionServer -- nothing<a name="line.40"></a>
<span class="sourceLineNo">041</span> * prevents this. This hbase:maeta ordering of InputSplit placement can be lumpy making it so some<a name="line.41"></a>
<span class="sourceLineNo">042</span> * RegionServers end up hosting lots of InputSplit scans while contemporaneously other RegionServers<a name="line.42"></a>
<span class="sourceLineNo">043</span> * host few or none. This class does a pass over the return from the super-class to better spread<a name="line.43"></a>
<span class="sourceLineNo">044</span> * the load. See the below helpful Flipkart blog post for a description and from where the base of<a name="line.44"></a>
<span class="sourceLineNo">045</span> * this code comes from (with permission).<a name="line.45"></a>
<span class="sourceLineNo">046</span> * @see https://tech.flipkart.com/is-data-locality-always-out-of-the-box-in-hadoop-not-really-2ae9c95163cb<a name="line.46"></a>
<span class="sourceLineNo">047</span> */<a name="line.47"></a>
<span class="sourceLineNo">048</span>@InterfaceAudience.Public<a name="line.48"></a>
<span class="sourceLineNo">049</span>public class RoundRobinTableInputFormat extends TableInputFormat {<a name="line.49"></a>
<span class="sourceLineNo">050</span> private Boolean hbaseRegionsizecalculatorEnableOriginalValue = null;<a name="line.50"></a>
<span class="sourceLineNo">051</span> /**<a name="line.51"></a>
<span class="sourceLineNo">052</span> * Boolean config for whether superclass should produce InputSplits with 'lengths'. If true, TIF<a name="line.52"></a>
<span class="sourceLineNo">053</span> * will query every RegionServer to get the 'size' of all involved Regions and this 'size' will be<a name="line.53"></a>
<span class="sourceLineNo">054</span> * used the the InputSplit length. If false, we skip this query and the super-classes returned<a name="line.54"></a>
<span class="sourceLineNo">055</span> * InputSplits will have lenghths of zero. This override will set the flag to false. All returned<a name="line.55"></a>
<span class="sourceLineNo">056</span> * lengths will be zero. Makes it so sorting on 'length' becomes a noop. The sort returned by this<a name="line.56"></a>
<span class="sourceLineNo">057</span> * override will prevail. Thats what we want.<a name="line.57"></a>
<span class="sourceLineNo">058</span> */<a name="line.58"></a>
<span class="sourceLineNo">059</span> static String HBASE_REGIONSIZECALCULATOR_ENABLE = "hbase.regionsizecalculator.enable";<a name="line.59"></a>
<span class="sourceLineNo">060</span><a name="line.60"></a>
<span class="sourceLineNo">061</span> @Override<a name="line.61"></a>
<span class="sourceLineNo">062</span> public List&lt;InputSplit&gt; getSplits(JobContext context) throws IOException {<a name="line.62"></a>
<span class="sourceLineNo">063</span> try {<a name="line.63"></a>
<span class="sourceLineNo">064</span> // Do a round robin on what we get back from the super-class.<a name="line.64"></a>
<span class="sourceLineNo">065</span> configure();<a name="line.65"></a>
<span class="sourceLineNo">066</span> return roundRobin(getSuperSplits(context));<a name="line.66"></a>
<span class="sourceLineNo">067</span> } finally {<a name="line.67"></a>
<span class="sourceLineNo">068</span> unconfigure();<a name="line.68"></a>
<span class="sourceLineNo">069</span> }<a name="line.69"></a>
<span class="sourceLineNo">070</span> }<a name="line.70"></a>
<span class="sourceLineNo">071</span><a name="line.71"></a>
<span class="sourceLineNo">072</span> /**<a name="line.72"></a>
<span class="sourceLineNo">073</span> * Call super-classes' getSplits. Have it out here as its own method so can be overridden.<a name="line.73"></a>
<span class="sourceLineNo">074</span> */<a name="line.74"></a>
<span class="sourceLineNo">075</span> List&lt;InputSplit&gt; getSuperSplits(JobContext context) throws IOException {<a name="line.75"></a>
<span class="sourceLineNo">076</span> return super.getSplits(context);<a name="line.76"></a>
<span class="sourceLineNo">077</span> }<a name="line.77"></a>
<span class="sourceLineNo">078</span><a name="line.78"></a>
<span class="sourceLineNo">079</span> /**<a name="line.79"></a>
<span class="sourceLineNo">080</span> * Spread the splits list so as to avoid clumping on RegionServers. Order splits so every server<a name="line.80"></a>
<span class="sourceLineNo">081</span> * gets one split before a server gets a second, and so on; i.e. round-robin the splits amongst<a name="line.81"></a>
<span class="sourceLineNo">082</span> * the servers in the cluster.<a name="line.82"></a>
<span class="sourceLineNo">083</span> */<a name="line.83"></a>
<span class="sourceLineNo">084</span> List&lt;InputSplit&gt; roundRobin(List&lt;InputSplit&gt; inputs) throws IOException {<a name="line.84"></a>
<span class="sourceLineNo">085</span> if ((inputs == null) || inputs.isEmpty()) {<a name="line.85"></a>
<span class="sourceLineNo">086</span> return inputs;<a name="line.86"></a>
<span class="sourceLineNo">087</span> }<a name="line.87"></a>
<span class="sourceLineNo">088</span> List&lt;InputSplit&gt; result = new ArrayList&lt;&gt;(inputs.size());<a name="line.88"></a>
<span class="sourceLineNo">089</span> // Prepare a hashmap with each region server as key and list of Input Splits as value<a name="line.89"></a>
<span class="sourceLineNo">090</span> Map&lt;String, List&lt;InputSplit&gt;&gt; regionServerSplits = new HashMap&lt;&gt;();<a name="line.90"></a>
<span class="sourceLineNo">091</span> for (InputSplit is : inputs) {<a name="line.91"></a>
<span class="sourceLineNo">092</span> if (is instanceof TableSplit) {<a name="line.92"></a>
<span class="sourceLineNo">093</span> String regionServer = ((TableSplit) is).getRegionLocation();<a name="line.93"></a>
<span class="sourceLineNo">094</span> if (regionServer != null &amp;&amp; !StringUtils.isBlank(regionServer)) {<a name="line.94"></a>
<span class="sourceLineNo">095</span> regionServerSplits.computeIfAbsent(regionServer, k -&gt; new ArrayList&lt;&gt;()).add(is);<a name="line.95"></a>
<span class="sourceLineNo">096</span> continue;<a name="line.96"></a>
<span class="sourceLineNo">097</span> }<a name="line.97"></a>
<span class="sourceLineNo">098</span> }<a name="line.98"></a>
<span class="sourceLineNo">099</span> // If TableSplit or region server not found, add it anyways.<a name="line.99"></a>
<span class="sourceLineNo">100</span> result.add(is);<a name="line.100"></a>
<span class="sourceLineNo">101</span> }<a name="line.101"></a>
<span class="sourceLineNo">102</span> // Write out splits in a manner that spreads splits for a RegionServer to avoid 'clumping'.<a name="line.102"></a>
<span class="sourceLineNo">103</span> while (!regionServerSplits.isEmpty()) {<a name="line.103"></a>
<span class="sourceLineNo">104</span> Iterator&lt;List&lt;InputSplit&gt;&gt; iter = regionServerSplits.values().iterator();<a name="line.104"></a>
<span class="sourceLineNo">105</span> while (iter.hasNext()) {<a name="line.105"></a>
<span class="sourceLineNo">106</span> List&lt;InputSplit&gt; inputSplitListForRegion = iter.next();<a name="line.106"></a>
<span class="sourceLineNo">107</span> if (!inputSplitListForRegion.isEmpty()) {<a name="line.107"></a>
<span class="sourceLineNo">108</span> result.add(inputSplitListForRegion.remove(0));<a name="line.108"></a>
<span class="sourceLineNo">109</span> }<a name="line.109"></a>
<span class="sourceLineNo">110</span> if (inputSplitListForRegion.isEmpty()) {<a name="line.110"></a>
<span class="sourceLineNo">111</span> iter.remove();<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> return result;<a name="line.115"></a>
<span class="sourceLineNo">116</span> }<a name="line.116"></a>
<span class="sourceLineNo">117</span><a name="line.117"></a>
<span class="sourceLineNo">118</span> /**<a name="line.118"></a>
<span class="sourceLineNo">119</span> * Adds a configuration to the Context disabling remote rpc'ing to figure Region size when<a name="line.119"></a>
<span class="sourceLineNo">120</span> * calculating InputSplits. See up in super-class TIF where we rpc to every server to find the<a name="line.120"></a>
<span class="sourceLineNo">121</span> * size of all involved Regions. Here we disable this super-class action. This means InputSplits<a name="line.121"></a>
<span class="sourceLineNo">122</span> * will have a length of zero. If all InputSplits have zero-length InputSplits, the ordering done<a name="line.122"></a>
<span class="sourceLineNo">123</span> * in here will 'pass-through' Hadoop's length-first sort. The superclass TIF will ask every node<a name="line.123"></a>
<span class="sourceLineNo">124</span> * for the current size of each of the participating Table Regions. It does this because it wants<a name="line.124"></a>
<span class="sourceLineNo">125</span> * to schedule the biggest Regions first (This fixation comes of hadoop itself -- see JobSubmitter<a name="line.125"></a>
<span class="sourceLineNo">126</span> * where it sorts inputs by size). This extra diligence takes time and is of no utility in this<a name="line.126"></a>
<span class="sourceLineNo">127</span> * RRTIF where spread is of more import than size-first. Also, if a rolling restart is happening<a name="line.127"></a>
<span class="sourceLineNo">128</span> * when we go to launch the job, the job launch may fail because the request for Region size fails<a name="line.128"></a>
<span class="sourceLineNo">129</span> * -- even after retries -- because rolled RegionServer may take a while to come online: e.g. it<a name="line.129"></a>
<span class="sourceLineNo">130</span> * takes java 90 seconds to allocate a 160G. RegionServer is offline during this time. The job<a name="line.130"></a>
<span class="sourceLineNo">131</span> * launch will fail with 'Connection rejected'. So, we set 'hbase.regionsizecalculator.enable' to<a name="line.131"></a>
<span class="sourceLineNo">132</span> * false here in RRTIF.<a name="line.132"></a>
<span class="sourceLineNo">133</span> * @see #unconfigure()<a name="line.133"></a>
<span class="sourceLineNo">134</span> */<a name="line.134"></a>
<span class="sourceLineNo">135</span> void configure() {<a name="line.135"></a>
<span class="sourceLineNo">136</span> if (getConf().get(HBASE_REGIONSIZECALCULATOR_ENABLE) != null) {<a name="line.136"></a>
<span class="sourceLineNo">137</span> this.hbaseRegionsizecalculatorEnableOriginalValue =<a name="line.137"></a>
<span class="sourceLineNo">138</span> getConf().getBoolean(HBASE_REGIONSIZECALCULATOR_ENABLE, true);<a name="line.138"></a>
<span class="sourceLineNo">139</span> }<a name="line.139"></a>
<span class="sourceLineNo">140</span> getConf().setBoolean(HBASE_REGIONSIZECALCULATOR_ENABLE, false);<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> /**<a name="line.143"></a>
<span class="sourceLineNo">144</span> * @see #configure()<a name="line.144"></a>
<span class="sourceLineNo">145</span> */<a name="line.145"></a>
<span class="sourceLineNo">146</span> void unconfigure() {<a name="line.146"></a>
<span class="sourceLineNo">147</span> if (this.hbaseRegionsizecalculatorEnableOriginalValue == null) {<a name="line.147"></a>
<span class="sourceLineNo">148</span> getConf().unset(HBASE_REGIONSIZECALCULATOR_ENABLE);<a name="line.148"></a>
<span class="sourceLineNo">149</span> } else {<a name="line.149"></a>
<span class="sourceLineNo">150</span> getConf().setBoolean(HBASE_REGIONSIZECALCULATOR_ENABLE,<a name="line.150"></a>
<span class="sourceLineNo">151</span> this.hbaseRegionsizecalculatorEnableOriginalValue);<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><a name="line.154"></a>
<span class="sourceLineNo">155</span> /**<a name="line.155"></a>
<span class="sourceLineNo">156</span> * Pass table name as argument. Set the zk ensemble to use with the System property<a name="line.156"></a>
<span class="sourceLineNo">157</span> * 'hbase.zookeeper.quorum'<a name="line.157"></a>
<span class="sourceLineNo">158</span> */<a name="line.158"></a>
<span class="sourceLineNo">159</span> public static void main(String[] args) throws IOException {<a name="line.159"></a>
<span class="sourceLineNo">160</span> TableInputFormat tif = new RoundRobinTableInputFormat();<a name="line.160"></a>
<span class="sourceLineNo">161</span> final Configuration configuration = HBaseConfiguration.create();<a name="line.161"></a>
<span class="sourceLineNo">162</span> configuration.setBoolean("hbase.regionsizecalculator.enable", false);<a name="line.162"></a>
<span class="sourceLineNo">163</span> configuration.set(HConstants.ZOOKEEPER_QUORUM,<a name="line.163"></a>
<span class="sourceLineNo">164</span> System.getProperty(HConstants.ZOOKEEPER_QUORUM, "localhost"));<a name="line.164"></a>
<span class="sourceLineNo">165</span> configuration.set(TableInputFormat.INPUT_TABLE, args[0]);<a name="line.165"></a>
<span class="sourceLineNo">166</span> tif.setConf(configuration);<a name="line.166"></a>
<span class="sourceLineNo">167</span> List&lt;InputSplit&gt; splits = tif.getSplits(new JobContextImpl(configuration, new JobID()));<a name="line.167"></a>
<span class="sourceLineNo">168</span> for (InputSplit split : splits) {<a name="line.168"></a>
<span class="sourceLineNo">169</span> System.out.println(split);<a name="line.169"></a>
<span class="sourceLineNo">170</span> }<a name="line.170"></a>
<span class="sourceLineNo">171</span> }<a name="line.171"></a>
<span class="sourceLineNo">172</span>}<a name="line.172"></a>
</pre>
</div>
</body>
</html>