blob: 019394a3a77d5940aa99e4d849bf496935c05df6 [file] [log] [blame]
<!DOCTYPE HTML>
<html lang="en">
<head>
<!-- Generated by javadoc (17) -->
<title>Source code</title>
<meta name="viewport" content="width=device-width, initial-scale=1">
<meta name="description" content="source: package: org.apache.hadoop.hbase.regionserver, class: InputStreamBlockDistribution">
<meta name="generator" content="javadoc/SourceToHTMLConverter">
<link rel="stylesheet" type="text/css" href="../../../../../../stylesheet.css" title="Style">
</head>
<body class="source-page">
<main role="main">
<div class="source-container">
<pre><span class="source-line-no">001</span><span id="line-1">/*</span>
<span class="source-line-no">002</span><span id="line-2"> * Licensed to the Apache Software Foundation (ASF) under one</span>
<span class="source-line-no">003</span><span id="line-3"> * or more contributor license agreements. See the NOTICE file</span>
<span class="source-line-no">004</span><span id="line-4"> * distributed with this work for additional information</span>
<span class="source-line-no">005</span><span id="line-5"> * regarding copyright ownership. The ASF licenses this file</span>
<span class="source-line-no">006</span><span id="line-6"> * to you under the Apache License, Version 2.0 (the</span>
<span class="source-line-no">007</span><span id="line-7"> * "License"); you may not use this file except in compliance</span>
<span class="source-line-no">008</span><span id="line-8"> * with the License. You may obtain a copy of the License at</span>
<span class="source-line-no">009</span><span id="line-9"> *</span>
<span class="source-line-no">010</span><span id="line-10"> * http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="source-line-no">011</span><span id="line-11"> *</span>
<span class="source-line-no">012</span><span id="line-12"> * Unless required by applicable law or agreed to in writing, software</span>
<span class="source-line-no">013</span><span id="line-13"> * distributed under the License is distributed on an "AS IS" BASIS,</span>
<span class="source-line-no">014</span><span id="line-14"> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="source-line-no">015</span><span id="line-15"> * See the License for the specific language governing permissions and</span>
<span class="source-line-no">016</span><span id="line-16"> * limitations under the License.</span>
<span class="source-line-no">017</span><span id="line-17"> */</span>
<span class="source-line-no">018</span><span id="line-18">package org.apache.hadoop.hbase.regionserver;</span>
<span class="source-line-no">019</span><span id="line-19"></span>
<span class="source-line-no">020</span><span id="line-20">import com.google.errorprone.annotations.RestrictedApi;</span>
<span class="source-line-no">021</span><span id="line-21">import java.io.IOException;</span>
<span class="source-line-no">022</span><span id="line-22">import org.apache.hadoop.conf.Configuration;</span>
<span class="source-line-no">023</span><span id="line-23">import org.apache.hadoop.fs.FSDataInputStream;</span>
<span class="source-line-no">024</span><span id="line-24">import org.apache.hadoop.hbase.HDFSBlocksDistribution;</span>
<span class="source-line-no">025</span><span id="line-25">import org.apache.hadoop.hbase.io.FileLink;</span>
<span class="source-line-no">026</span><span id="line-26">import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;</span>
<span class="source-line-no">027</span><span id="line-27">import org.apache.hadoop.hbase.util.FSUtils;</span>
<span class="source-line-no">028</span><span id="line-28">import org.apache.hadoop.hdfs.client.HdfsDataInputStream;</span>
<span class="source-line-no">029</span><span id="line-29">import org.apache.yetus.audience.InterfaceAudience;</span>
<span class="source-line-no">030</span><span id="line-30">import org.slf4j.Logger;</span>
<span class="source-line-no">031</span><span id="line-31">import org.slf4j.LoggerFactory;</span>
<span class="source-line-no">032</span><span id="line-32"></span>
<span class="source-line-no">033</span><span id="line-33">/**</span>
<span class="source-line-no">034</span><span id="line-34"> * Computes the HDFSBlockDistribution for a file based on the underlying located blocks for an</span>
<span class="source-line-no">035</span><span id="line-35"> * HdfsDataInputStream reading that file. The backing DFSInputStream.getAllBlocks involves</span>
<span class="source-line-no">036</span><span id="line-36"> * allocating an array of numBlocks size per call. It may also involve calling the namenode, if the</span>
<span class="source-line-no">037</span><span id="line-37"> * DFSInputStream has not fetched all the blocks yet. In order to avoid allocation pressure, we</span>
<span class="source-line-no">038</span><span id="line-38"> * cache the computed distribution for a configurable period of time.</span>
<span class="source-line-no">039</span><span id="line-39"> * &lt;p&gt;</span>
<span class="source-line-no">040</span><span id="line-40"> * This class only gets instantiated for the &lt;b&gt;first&lt;/b&gt; FSDataInputStream of each StoreFile (i.e.</span>
<span class="source-line-no">041</span><span id="line-41"> * the one backing {@link HStoreFile#initialReader}). It's then used to dynamically update the value</span>
<span class="source-line-no">042</span><span id="line-42"> * returned by {@link HStoreFile#getHDFSBlockDistribution()}.</span>
<span class="source-line-no">043</span><span id="line-43"> * &lt;p&gt;</span>
<span class="source-line-no">044</span><span id="line-44"> * Once the backing FSDataInputStream is closed, we should not expect the distribution result to</span>
<span class="source-line-no">045</span><span id="line-45"> * change anymore. This is ok becuase the initialReader's InputStream is only closed when the</span>
<span class="source-line-no">046</span><span id="line-46"> * StoreFile itself is closed, at which point nothing will be querying getHDFSBlockDistribution</span>
<span class="source-line-no">047</span><span id="line-47"> * anymore. If/When the StoreFile is reopened, a new {@link InputStreamBlockDistribution} will be</span>
<span class="source-line-no">048</span><span id="line-48"> * created for the new initialReader.</span>
<span class="source-line-no">049</span><span id="line-49"> */</span>
<span class="source-line-no">050</span><span id="line-50">@InterfaceAudience.Private</span>
<span class="source-line-no">051</span><span id="line-51">public class InputStreamBlockDistribution {</span>
<span class="source-line-no">052</span><span id="line-52"> private static final Logger LOG = LoggerFactory.getLogger(InputStreamBlockDistribution.class);</span>
<span class="source-line-no">053</span><span id="line-53"></span>
<span class="source-line-no">054</span><span id="line-54"> private static final String HBASE_LOCALITY_INPUTSTREAM_DERIVE_ENABLED =</span>
<span class="source-line-no">055</span><span id="line-55"> "hbase.locality.inputstream.derive.enabled";</span>
<span class="source-line-no">056</span><span id="line-56"> private static final boolean DEFAULT_HBASE_LOCALITY_INPUTSTREAM_DERIVE_ENABLED = false;</span>
<span class="source-line-no">057</span><span id="line-57"></span>
<span class="source-line-no">058</span><span id="line-58"> private static final String HBASE_LOCALITY_INPUTSTREAM_DERIVE_CACHE_PERIOD =</span>
<span class="source-line-no">059</span><span id="line-59"> "hbase.locality.inputstream.derive.cache.period";</span>
<span class="source-line-no">060</span><span id="line-60"> private static final int DEFAULT_HBASE_LOCALITY_INPUTSTREAM_DERIVE_CACHE_PERIOD = 60_000;</span>
<span class="source-line-no">061</span><span id="line-61"></span>
<span class="source-line-no">062</span><span id="line-62"> private final FSDataInputStream stream;</span>
<span class="source-line-no">063</span><span id="line-63"> private final StoreFileInfo fileInfo;</span>
<span class="source-line-no">064</span><span id="line-64"> private final int cachePeriodMs;</span>
<span class="source-line-no">065</span><span id="line-65"></span>
<span class="source-line-no">066</span><span id="line-66"> private HDFSBlocksDistribution hdfsBlocksDistribution;</span>
<span class="source-line-no">067</span><span id="line-67"> private long lastCachedAt;</span>
<span class="source-line-no">068</span><span id="line-68"> private boolean streamUnsupported;</span>
<span class="source-line-no">069</span><span id="line-69"></span>
<span class="source-line-no">070</span><span id="line-70"> /**</span>
<span class="source-line-no">071</span><span id="line-71"> * This should only be called for the first FSDataInputStream of a StoreFile, in</span>
<span class="source-line-no">072</span><span id="line-72"> * {@link HStoreFile#open()}.</span>
<span class="source-line-no">073</span><span id="line-73"> * @see InputStreamBlockDistribution</span>
<span class="source-line-no">074</span><span id="line-74"> * @param stream the input stream to derive locality from</span>
<span class="source-line-no">075</span><span id="line-75"> * @param fileInfo the StoreFileInfo for the related store file</span>
<span class="source-line-no">076</span><span id="line-76"> */</span>
<span class="source-line-no">077</span><span id="line-77"> public InputStreamBlockDistribution(FSDataInputStream stream, StoreFileInfo fileInfo) {</span>
<span class="source-line-no">078</span><span id="line-78"> this.stream = stream;</span>
<span class="source-line-no">079</span><span id="line-79"> this.fileInfo = fileInfo;</span>
<span class="source-line-no">080</span><span id="line-80"> this.cachePeriodMs = fileInfo.getConf().getInt(HBASE_LOCALITY_INPUTSTREAM_DERIVE_CACHE_PERIOD,</span>
<span class="source-line-no">081</span><span id="line-81"> DEFAULT_HBASE_LOCALITY_INPUTSTREAM_DERIVE_CACHE_PERIOD);</span>
<span class="source-line-no">082</span><span id="line-82"> this.lastCachedAt = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">083</span><span id="line-83"> this.streamUnsupported = false;</span>
<span class="source-line-no">084</span><span id="line-84"> this.hdfsBlocksDistribution = fileInfo.getHDFSBlockDistribution();</span>
<span class="source-line-no">085</span><span id="line-85"> }</span>
<span class="source-line-no">086</span><span id="line-86"></span>
<span class="source-line-no">087</span><span id="line-87"> /**</span>
<span class="source-line-no">088</span><span id="line-88"> * True if we should derive StoreFile HDFSBlockDistribution from the underlying input stream</span>
<span class="source-line-no">089</span><span id="line-89"> */</span>
<span class="source-line-no">090</span><span id="line-90"> public static boolean isEnabled(Configuration conf) {</span>
<span class="source-line-no">091</span><span id="line-91"> return conf.getBoolean(HBASE_LOCALITY_INPUTSTREAM_DERIVE_ENABLED,</span>
<span class="source-line-no">092</span><span id="line-92"> DEFAULT_HBASE_LOCALITY_INPUTSTREAM_DERIVE_ENABLED);</span>
<span class="source-line-no">093</span><span id="line-93"> }</span>
<span class="source-line-no">094</span><span id="line-94"></span>
<span class="source-line-no">095</span><span id="line-95"> /**</span>
<span class="source-line-no">096</span><span id="line-96"> * Get the HDFSBlocksDistribution derived from the StoreFile input stream, re-computing if cache</span>
<span class="source-line-no">097</span><span id="line-97"> * is expired.</span>
<span class="source-line-no">098</span><span id="line-98"> */</span>
<span class="source-line-no">099</span><span id="line-99"> public synchronized HDFSBlocksDistribution getHDFSBlockDistribution() {</span>
<span class="source-line-no">100</span><span id="line-100"> if (EnvironmentEdgeManager.currentTime() - lastCachedAt &gt; cachePeriodMs) {</span>
<span class="source-line-no">101</span><span id="line-101"> try {</span>
<span class="source-line-no">102</span><span id="line-102"> LOG.debug("Refreshing HDFSBlockDistribution for {}", fileInfo);</span>
<span class="source-line-no">103</span><span id="line-103"> computeBlockDistribution();</span>
<span class="source-line-no">104</span><span id="line-104"> } catch (IOException e) {</span>
<span class="source-line-no">105</span><span id="line-105"> LOG.warn("Failed to recompute block distribution for {}. Falling back on cached value.",</span>
<span class="source-line-no">106</span><span id="line-106"> fileInfo, e);</span>
<span class="source-line-no">107</span><span id="line-107"> }</span>
<span class="source-line-no">108</span><span id="line-108"> }</span>
<span class="source-line-no">109</span><span id="line-109"> return hdfsBlocksDistribution;</span>
<span class="source-line-no">110</span><span id="line-110"> }</span>
<span class="source-line-no">111</span><span id="line-111"></span>
<span class="source-line-no">112</span><span id="line-112"> private void computeBlockDistribution() throws IOException {</span>
<span class="source-line-no">113</span><span id="line-113"> lastCachedAt = EnvironmentEdgeManager.currentTime();</span>
<span class="source-line-no">114</span><span id="line-114"></span>
<span class="source-line-no">115</span><span id="line-115"> FSDataInputStream stream;</span>
<span class="source-line-no">116</span><span id="line-116"> if (fileInfo.isLink()) {</span>
<span class="source-line-no">117</span><span id="line-117"> stream = FileLink.getUnderlyingFileLinkInputStream(this.stream);</span>
<span class="source-line-no">118</span><span id="line-118"> } else {</span>
<span class="source-line-no">119</span><span id="line-119"> stream = this.stream;</span>
<span class="source-line-no">120</span><span id="line-120"> }</span>
<span class="source-line-no">121</span><span id="line-121"></span>
<span class="source-line-no">122</span><span id="line-122"> if (!(stream instanceof HdfsDataInputStream)) {</span>
<span class="source-line-no">123</span><span id="line-123"> if (!streamUnsupported) {</span>
<span class="source-line-no">124</span><span id="line-124"> LOG.warn(</span>
<span class="source-line-no">125</span><span id="line-125"> "{} for storeFileInfo={}, isLink={}, is not an HdfsDataInputStream so cannot be "</span>
<span class="source-line-no">126</span><span id="line-126"> + "used to derive locality. Falling back on cached value.",</span>
<span class="source-line-no">127</span><span id="line-127"> stream, fileInfo, fileInfo.isLink());</span>
<span class="source-line-no">128</span><span id="line-128"> streamUnsupported = true;</span>
<span class="source-line-no">129</span><span id="line-129"> }</span>
<span class="source-line-no">130</span><span id="line-130"> return;</span>
<span class="source-line-no">131</span><span id="line-131"> }</span>
<span class="source-line-no">132</span><span id="line-132"></span>
<span class="source-line-no">133</span><span id="line-133"> streamUnsupported = false;</span>
<span class="source-line-no">134</span><span id="line-134"> hdfsBlocksDistribution = FSUtils.computeHDFSBlocksDistribution((HdfsDataInputStream) stream);</span>
<span class="source-line-no">135</span><span id="line-135"> }</span>
<span class="source-line-no">136</span><span id="line-136"></span>
<span class="source-line-no">137</span><span id="line-137"> /**</span>
<span class="source-line-no">138</span><span id="line-138"> * For tests only, sets lastCachedAt so we can force a refresh</span>
<span class="source-line-no">139</span><span id="line-139"> */</span>
<span class="source-line-no">140</span><span id="line-140"> @RestrictedApi(explanation = "Should only be called in tests", link = "",</span>
<span class="source-line-no">141</span><span id="line-141"> allowedOnPath = ".*/src/test/.*")</span>
<span class="source-line-no">142</span><span id="line-142"> synchronized void setLastCachedAt(long timestamp) {</span>
<span class="source-line-no">143</span><span id="line-143"> lastCachedAt = timestamp;</span>
<span class="source-line-no">144</span><span id="line-144"> }</span>
<span class="source-line-no">145</span><span id="line-145"></span>
<span class="source-line-no">146</span><span id="line-146"> /**</span>
<span class="source-line-no">147</span><span id="line-147"> * For tests only, returns the configured cache period</span>
<span class="source-line-no">148</span><span id="line-148"> */</span>
<span class="source-line-no">149</span><span id="line-149"> @RestrictedApi(explanation = "Should only be called in tests", link = "",</span>
<span class="source-line-no">150</span><span id="line-150"> allowedOnPath = ".*/src/test/.*")</span>
<span class="source-line-no">151</span><span id="line-151"> long getCachePeriodMs() {</span>
<span class="source-line-no">152</span><span id="line-152"> return cachePeriodMs;</span>
<span class="source-line-no">153</span><span id="line-153"> }</span>
<span class="source-line-no">154</span><span id="line-154"></span>
<span class="source-line-no">155</span><span id="line-155"> /**</span>
<span class="source-line-no">156</span><span id="line-156"> * For tests only, returns whether the passed stream is supported</span>
<span class="source-line-no">157</span><span id="line-157"> */</span>
<span class="source-line-no">158</span><span id="line-158"> @RestrictedApi(explanation = "Should only be called in tests", link = "",</span>
<span class="source-line-no">159</span><span id="line-159"> allowedOnPath = ".*/src/test/.*")</span>
<span class="source-line-no">160</span><span id="line-160"> boolean isStreamUnsupported() {</span>
<span class="source-line-no">161</span><span id="line-161"> return streamUnsupported;</span>
<span class="source-line-no">162</span><span id="line-162"> }</span>
<span class="source-line-no">163</span><span id="line-163">}</span>
</pre>
</div>
</main>
</body>
</html>