| <!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> *<a name="line.2"></a> |
| <span class="sourceLineNo">003</span> * Licensed to the Apache Software Foundation (ASF) under one<a name="line.3"></a> |
| <span class="sourceLineNo">004</span> * or more contributor license agreements. See the NOTICE file<a name="line.4"></a> |
| <span class="sourceLineNo">005</span> * distributed with this work for additional information<a name="line.5"></a> |
| <span class="sourceLineNo">006</span> * regarding copyright ownership. The ASF licenses this file<a name="line.6"></a> |
| <span class="sourceLineNo">007</span> * to you under the Apache License, Version 2.0 (the<a name="line.7"></a> |
| <span class="sourceLineNo">008</span> * "License"); you may not use this file except in compliance<a name="line.8"></a> |
| <span class="sourceLineNo">009</span> * with the License. You may obtain a copy of the License at<a name="line.9"></a> |
| <span class="sourceLineNo">010</span> *<a name="line.10"></a> |
| <span class="sourceLineNo">011</span> * http://www.apache.org/licenses/LICENSE-2.0<a name="line.11"></a> |
| <span class="sourceLineNo">012</span> *<a name="line.12"></a> |
| <span class="sourceLineNo">013</span> * Unless required by applicable law or agreed to in writing, software<a name="line.13"></a> |
| <span class="sourceLineNo">014</span> * distributed under the License is distributed on an "AS IS" BASIS,<a name="line.14"></a> |
| <span class="sourceLineNo">015</span> * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.<a name="line.15"></a> |
| <span class="sourceLineNo">016</span> * See the License for the specific language governing permissions and<a name="line.16"></a> |
| <span class="sourceLineNo">017</span> * limitations under the License.<a name="line.17"></a> |
| <span class="sourceLineNo">018</span> */<a name="line.18"></a> |
| <span class="sourceLineNo">019</span>package org.apache.hadoop.hbase.regionserver;<a name="line.19"></a> |
| <span class="sourceLineNo">020</span><a name="line.20"></a> |
| <span class="sourceLineNo">021</span>import java.io.IOException;<a name="line.21"></a> |
| <span class="sourceLineNo">022</span>import java.io.InterruptedIOException;<a name="line.22"></a> |
| <span class="sourceLineNo">023</span>import java.util.ArrayList;<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.NavigableSet;<a name="line.25"></a> |
| <span class="sourceLineNo">026</span>import java.util.concurrent.CountDownLatch;<a name="line.26"></a> |
| <span class="sourceLineNo">027</span>import java.util.concurrent.locks.ReentrantLock;<a name="line.27"></a> |
| <span class="sourceLineNo">028</span><a name="line.28"></a> |
| <span class="sourceLineNo">029</span>import org.apache.hadoop.hbase.Cell;<a name="line.29"></a> |
| <span class="sourceLineNo">030</span>import org.apache.hadoop.hbase.CellComparator;<a name="line.30"></a> |
| <span class="sourceLineNo">031</span>import org.apache.hadoop.hbase.CellUtil;<a name="line.31"></a> |
| <span class="sourceLineNo">032</span>import org.apache.hadoop.hbase.DoNotRetryIOException;<a name="line.32"></a> |
| <span class="sourceLineNo">033</span>import org.apache.hadoop.hbase.HConstants;<a name="line.33"></a> |
| <span class="sourceLineNo">034</span>import org.apache.hadoop.hbase.PrivateCellUtil;<a name="line.34"></a> |
| <span class="sourceLineNo">035</span>import org.apache.hadoop.hbase.KeyValue;<a name="line.35"></a> |
| <span class="sourceLineNo">036</span>import org.apache.hadoop.hbase.KeyValueUtil;<a name="line.36"></a> |
| <span class="sourceLineNo">037</span>import org.apache.hadoop.hbase.client.IsolationLevel;<a name="line.37"></a> |
| <span class="sourceLineNo">038</span>import org.apache.hadoop.hbase.client.Scan;<a name="line.38"></a> |
| <span class="sourceLineNo">039</span>import org.apache.hadoop.hbase.executor.ExecutorService;<a name="line.39"></a> |
| <span class="sourceLineNo">040</span>import org.apache.hadoop.hbase.filter.Filter;<a name="line.40"></a> |
| <span class="sourceLineNo">041</span>import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;<a name="line.41"></a> |
| <span class="sourceLineNo">042</span>import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;<a name="line.42"></a> |
| <span class="sourceLineNo">043</span>import org.apache.hadoop.hbase.regionserver.handler.ParallelSeekHandler;<a name="line.43"></a> |
| <span class="sourceLineNo">044</span>import org.apache.hadoop.hbase.regionserver.querymatcher.CompactionScanQueryMatcher;<a name="line.44"></a> |
| <span class="sourceLineNo">045</span>import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;<a name="line.45"></a> |
| <span class="sourceLineNo">046</span>import org.apache.hadoop.hbase.regionserver.querymatcher.UserScanQueryMatcher;<a name="line.46"></a> |
| <span class="sourceLineNo">047</span>import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;<a name="line.47"></a> |
| <span class="sourceLineNo">048</span>import org.apache.yetus.audience.InterfaceAudience;<a name="line.48"></a> |
| <span class="sourceLineNo">049</span><a name="line.49"></a> |
| <span class="sourceLineNo">050</span>import org.slf4j.Logger;<a name="line.50"></a> |
| <span class="sourceLineNo">051</span>import org.slf4j.LoggerFactory;<a name="line.51"></a> |
| <span class="sourceLineNo">052</span><a name="line.52"></a> |
| <span class="sourceLineNo">053</span>import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;<a name="line.53"></a> |
| <span class="sourceLineNo">054</span>import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;<a name="line.54"></a> |
| <span class="sourceLineNo">055</span>import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;<a name="line.55"></a> |
| <span class="sourceLineNo">056</span><a name="line.56"></a> |
| <span class="sourceLineNo">057</span>/**<a name="line.57"></a> |
| <span class="sourceLineNo">058</span> * Scanner scans both the memstore and the Store. Coalesce KeyValue stream into List&lt;KeyValue&gt;<a name="line.58"></a> |
| <span class="sourceLineNo">059</span> * for a single row.<a name="line.59"></a> |
| <span class="sourceLineNo">060</span> * <p><a name="line.60"></a> |
| <span class="sourceLineNo">061</span> * The implementation is not thread safe. So there will be no race between next and close. The only<a name="line.61"></a> |
| <span class="sourceLineNo">062</span> * exception is updateReaders, it will be called in the memstore flush thread to indicate that there<a name="line.62"></a> |
| <span class="sourceLineNo">063</span> * is a flush.<a name="line.63"></a> |
| <span class="sourceLineNo">064</span> */<a name="line.64"></a> |
| <span class="sourceLineNo">065</span>@InterfaceAudience.Private<a name="line.65"></a> |
| <span class="sourceLineNo">066</span>public class StoreScanner extends NonReversedNonLazyKeyValueScanner<a name="line.66"></a> |
| <span class="sourceLineNo">067</span> implements KeyValueScanner, InternalScanner, ChangedReadersObserver {<a name="line.67"></a> |
| <span class="sourceLineNo">068</span> private static final Logger LOG = LoggerFactory.getLogger(StoreScanner.class);<a name="line.68"></a> |
| <span class="sourceLineNo">069</span> // In unit tests, the store could be null<a name="line.69"></a> |
| <span class="sourceLineNo">070</span> protected final HStore store;<a name="line.70"></a> |
| <span class="sourceLineNo">071</span> private final CellComparator comparator;<a name="line.71"></a> |
| <span class="sourceLineNo">072</span> private ScanQueryMatcher matcher;<a name="line.72"></a> |
| <span class="sourceLineNo">073</span> protected KeyValueHeap heap;<a name="line.73"></a> |
| <span class="sourceLineNo">074</span> private boolean cacheBlocks;<a name="line.74"></a> |
| <span class="sourceLineNo">075</span><a name="line.75"></a> |
| <span class="sourceLineNo">076</span> private long countPerRow = 0;<a name="line.76"></a> |
| <span class="sourceLineNo">077</span> private int storeLimit = -1;<a name="line.77"></a> |
| <span class="sourceLineNo">078</span> private int storeOffset = 0;<a name="line.78"></a> |
| <span class="sourceLineNo">079</span><a name="line.79"></a> |
| <span class="sourceLineNo">080</span> // Used to indicate that the scanner has closed (see HBASE-1107)<a name="line.80"></a> |
| <span class="sourceLineNo">081</span> private volatile boolean closing = false;<a name="line.81"></a> |
| <span class="sourceLineNo">082</span> private final boolean get;<a name="line.82"></a> |
| <span class="sourceLineNo">083</span> private final boolean explicitColumnQuery;<a name="line.83"></a> |
| <span class="sourceLineNo">084</span> private final boolean useRowColBloom;<a name="line.84"></a> |
| <span class="sourceLineNo">085</span> /**<a name="line.85"></a> |
| <span class="sourceLineNo">086</span> * A flag that enables StoreFileScanner parallel-seeking<a name="line.86"></a> |
| <span class="sourceLineNo">087</span> */<a name="line.87"></a> |
| <span class="sourceLineNo">088</span> private boolean parallelSeekEnabled = false;<a name="line.88"></a> |
| <span class="sourceLineNo">089</span> private ExecutorService executor;<a name="line.89"></a> |
| <span class="sourceLineNo">090</span> private final Scan scan;<a name="line.90"></a> |
| <span class="sourceLineNo">091</span> private final long oldestUnexpiredTS;<a name="line.91"></a> |
| <span class="sourceLineNo">092</span> private final long now;<a name="line.92"></a> |
| <span class="sourceLineNo">093</span> private final int minVersions;<a name="line.93"></a> |
| <span class="sourceLineNo">094</span> private final long maxRowSize;<a name="line.94"></a> |
| <span class="sourceLineNo">095</span> private final long cellsPerHeartbeatCheck;<a name="line.95"></a> |
| <span class="sourceLineNo">096</span> @VisibleForTesting<a name="line.96"></a> |
| <span class="sourceLineNo">097</span> long memstoreOnlyReads;<a name="line.97"></a> |
| <span class="sourceLineNo">098</span> @VisibleForTesting<a name="line.98"></a> |
| <span class="sourceLineNo">099</span> long mixedReads;<a name="line.99"></a> |
| <span class="sourceLineNo">100</span><a name="line.100"></a> |
| <span class="sourceLineNo">101</span> // 1) Collects all the KVHeap that are eagerly getting closed during the<a name="line.101"></a> |
| <span class="sourceLineNo">102</span> // course of a scan<a name="line.102"></a> |
| <span class="sourceLineNo">103</span> // 2) Collects the unused memstore scanners. If we close the memstore scanners<a name="line.103"></a> |
| <span class="sourceLineNo">104</span> // before sending data to client, the chunk may be reclaimed by other<a name="line.104"></a> |
| <span class="sourceLineNo">105</span> // updates and the data will be corrupt.<a name="line.105"></a> |
| <span class="sourceLineNo">106</span> private final List<KeyValueScanner> scannersForDelayedClose = new ArrayList<>();<a name="line.106"></a> |
| <span class="sourceLineNo">107</span><a name="line.107"></a> |
| <span class="sourceLineNo">108</span> /**<a name="line.108"></a> |
| <span class="sourceLineNo">109</span> * The number of KVs seen by the scanner. Includes explicitly skipped KVs, but not<a name="line.109"></a> |
| <span class="sourceLineNo">110</span> * KVs skipped via seeking to next row/column. TODO: estimate them?<a name="line.110"></a> |
| <span class="sourceLineNo">111</span> */<a name="line.111"></a> |
| <span class="sourceLineNo">112</span> private long kvsScanned = 0;<a name="line.112"></a> |
| <span class="sourceLineNo">113</span> private Cell prevCell = null;<a name="line.113"></a> |
| <span class="sourceLineNo">114</span><a name="line.114"></a> |
| <span class="sourceLineNo">115</span> private final long preadMaxBytes;<a name="line.115"></a> |
| <span class="sourceLineNo">116</span> private long bytesRead;<a name="line.116"></a> |
| <span class="sourceLineNo">117</span><a name="line.117"></a> |
| <span class="sourceLineNo">118</span> /** We don't ever expect to change this, the constant is just for clarity. */<a name="line.118"></a> |
| <span class="sourceLineNo">119</span> static final boolean LAZY_SEEK_ENABLED_BY_DEFAULT = true;<a name="line.119"></a> |
| <span class="sourceLineNo">120</span> public static final String STORESCANNER_PARALLEL_SEEK_ENABLE =<a name="line.120"></a> |
| <span class="sourceLineNo">121</span> "hbase.storescanner.parallel.seek.enable";<a name="line.121"></a> |
| <span class="sourceLineNo">122</span><a name="line.122"></a> |
| <span class="sourceLineNo">123</span> /** Used during unit testing to ensure that lazy seek does save seek ops */<a name="line.123"></a> |
| <span class="sourceLineNo">124</span> private static boolean lazySeekEnabledGlobally = LAZY_SEEK_ENABLED_BY_DEFAULT;<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> * The number of cells scanned in between timeout checks. Specifying a larger value means that<a name="line.127"></a> |
| <span class="sourceLineNo">128</span> * timeout checks will occur less frequently. Specifying a small value will lead to more frequent<a name="line.128"></a> |
| <span class="sourceLineNo">129</span> * timeout checks.<a name="line.129"></a> |
| <span class="sourceLineNo">130</span> */<a name="line.130"></a> |
| <span class="sourceLineNo">131</span> public static final String HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK =<a name="line.131"></a> |
| <span class="sourceLineNo">132</span> "hbase.cells.scanned.per.heartbeat.check";<a name="line.132"></a> |
| <span class="sourceLineNo">133</span><a name="line.133"></a> |
| <span class="sourceLineNo">134</span> /**<a name="line.134"></a> |
| <span class="sourceLineNo">135</span> * Default value of {@link #HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK}.<a name="line.135"></a> |
| <span class="sourceLineNo">136</span> */<a name="line.136"></a> |
| <span class="sourceLineNo">137</span> public static final long DEFAULT_HBASE_CELLS_SCANNED_PER_HEARTBEAT_CHECK = 10000;<a name="line.137"></a> |
| <span class="sourceLineNo">138</span><a name="line.138"></a> |
| <span class="sourceLineNo">139</span> /**<a name="line.139"></a> |
| <span class="sourceLineNo">140</span> * If the read type if Scan.ReadType.DEFAULT, we will start with pread, and if the kvs we scanned<a name="line.140"></a> |
| <span class="sourceLineNo">141</span> * reaches this limit, we will reopen the scanner with stream. The default value is 4 times of<a name="line.141"></a> |
| <span class="sourceLineNo">142</span> * block size for this store.<a name="line.142"></a> |
| <span class="sourceLineNo">143</span> */<a name="line.143"></a> |
| <span class="sourceLineNo">144</span> public static final String STORESCANNER_PREAD_MAX_BYTES = "hbase.storescanner.pread.max.bytes";<a name="line.144"></a> |
| <span class="sourceLineNo">145</span><a name="line.145"></a> |
| <span class="sourceLineNo">146</span> private final Scan.ReadType readType;<a name="line.146"></a> |
| <span class="sourceLineNo">147</span><a name="line.147"></a> |
| <span class="sourceLineNo">148</span> // A flag whether use pread for scan<a name="line.148"></a> |
| <span class="sourceLineNo">149</span> // it maybe changed if we use Scan.ReadType.DEFAULT and we have read lots of data.<a name="line.149"></a> |
| <span class="sourceLineNo">150</span> private boolean scanUsePread;<a name="line.150"></a> |
| <span class="sourceLineNo">151</span> // Indicates whether there was flush during the course of the scan<a name="line.151"></a> |
| <span class="sourceLineNo">152</span> private volatile boolean flushed = false;<a name="line.152"></a> |
| <span class="sourceLineNo">153</span> // generally we get one file from a flush<a name="line.153"></a> |
| <span class="sourceLineNo">154</span> private final List<KeyValueScanner> flushedstoreFileScanners = new ArrayList<>(1);<a name="line.154"></a> |
| <span class="sourceLineNo">155</span> // Since CompactingMemstore is now default, we get three memstore scanners from a flush<a name="line.155"></a> |
| <span class="sourceLineNo">156</span> private final List<KeyValueScanner> memStoreScannersAfterFlush = new ArrayList<>(3);<a name="line.156"></a> |
| <span class="sourceLineNo">157</span> // The current list of scanners<a name="line.157"></a> |
| <span class="sourceLineNo">158</span> @VisibleForTesting<a name="line.158"></a> |
| <span class="sourceLineNo">159</span> final List<KeyValueScanner> currentScanners = new ArrayList<>();<a name="line.159"></a> |
| <span class="sourceLineNo">160</span> // flush update lock<a name="line.160"></a> |
| <span class="sourceLineNo">161</span> private final ReentrantLock flushLock = new ReentrantLock();<a name="line.161"></a> |
| <span class="sourceLineNo">162</span> // lock for closing.<a name="line.162"></a> |
| <span class="sourceLineNo">163</span> private final ReentrantLock closeLock = new ReentrantLock();<a name="line.163"></a> |
| <span class="sourceLineNo">164</span><a name="line.164"></a> |
| <span class="sourceLineNo">165</span> protected final long readPt;<a name="line.165"></a> |
| <span class="sourceLineNo">166</span> private boolean topChanged = false;<a name="line.166"></a> |
| <span class="sourceLineNo">167</span><a name="line.167"></a> |
| <span class="sourceLineNo">168</span> /** An internal constructor. */<a name="line.168"></a> |
| <span class="sourceLineNo">169</span> private StoreScanner(HStore store, Scan scan, ScanInfo scanInfo,<a name="line.169"></a> |
| <span class="sourceLineNo">170</span> int numColumns, long readPt, boolean cacheBlocks, ScanType scanType) {<a name="line.170"></a> |
| <span class="sourceLineNo">171</span> this.readPt = readPt;<a name="line.171"></a> |
| <span class="sourceLineNo">172</span> this.store = store;<a name="line.172"></a> |
| <span class="sourceLineNo">173</span> this.cacheBlocks = cacheBlocks;<a name="line.173"></a> |
| <span class="sourceLineNo">174</span> this.comparator = Preconditions.checkNotNull(scanInfo.getComparator());<a name="line.174"></a> |
| <span class="sourceLineNo">175</span> get = scan.isGetScan();<a name="line.175"></a> |
| <span class="sourceLineNo">176</span> explicitColumnQuery = numColumns > 0;<a name="line.176"></a> |
| <span class="sourceLineNo">177</span> this.scan = scan;<a name="line.177"></a> |
| <span class="sourceLineNo">178</span> this.now = EnvironmentEdgeManager.currentTime();<a name="line.178"></a> |
| <span class="sourceLineNo">179</span> this.oldestUnexpiredTS = scan.isRaw() ? 0L : now - scanInfo.getTtl();<a name="line.179"></a> |
| <span class="sourceLineNo">180</span> this.minVersions = scanInfo.getMinVersions();<a name="line.180"></a> |
| <span class="sourceLineNo">181</span><a name="line.181"></a> |
| <span class="sourceLineNo">182</span> // We look up row-column Bloom filters for multi-column queries as part of<a name="line.182"></a> |
| <span class="sourceLineNo">183</span> // the seek operation. However, we also look the row-column Bloom filter<a name="line.183"></a> |
| <span class="sourceLineNo">184</span> // for multi-row (non-"get") scans because this is not done in<a name="line.184"></a> |
| <span class="sourceLineNo">185</span> // StoreFile.passesBloomFilter(Scan, SortedSet<byte[]>).<a name="line.185"></a> |
| <span class="sourceLineNo">186</span> this.useRowColBloom = numColumns > 1 || (!get && numColumns == 1)<a name="line.186"></a> |
| <span class="sourceLineNo">187</span> && (store == null || store.getColumnFamilyDescriptor().getBloomFilterType() == BloomType.ROWCOL);<a name="line.187"></a> |
| <span class="sourceLineNo">188</span> this.maxRowSize = scanInfo.getTableMaxRowSize();<a name="line.188"></a> |
| <span class="sourceLineNo">189</span> if (get) {<a name="line.189"></a> |
| <span class="sourceLineNo">190</span> this.readType = Scan.ReadType.PREAD;<a name="line.190"></a> |
| <span class="sourceLineNo">191</span> this.scanUsePread = true;<a name="line.191"></a> |
| <span class="sourceLineNo">192</span> } else if (scanType != ScanType.USER_SCAN) {<a name="line.192"></a> |
| <span class="sourceLineNo">193</span> // For compaction scanners never use Pread as already we have stream based scanners on the<a name="line.193"></a> |
| <span class="sourceLineNo">194</span> // store files to be compacted<a name="line.194"></a> |
| <span class="sourceLineNo">195</span> this.readType = Scan.ReadType.STREAM;<a name="line.195"></a> |
| <span class="sourceLineNo">196</span> this.scanUsePread = false;<a name="line.196"></a> |
| <span class="sourceLineNo">197</span> } else {<a name="line.197"></a> |
| <span class="sourceLineNo">198</span> if (scan.getReadType() == Scan.ReadType.DEFAULT) {<a name="line.198"></a> |
| <span class="sourceLineNo">199</span> this.readType = scanInfo.isUsePread() ? Scan.ReadType.PREAD : Scan.ReadType.DEFAULT;<a name="line.199"></a> |
| <span class="sourceLineNo">200</span> } else {<a name="line.200"></a> |
| <span class="sourceLineNo">201</span> this.readType = scan.getReadType();<a name="line.201"></a> |
| <span class="sourceLineNo">202</span> }<a name="line.202"></a> |
| <span class="sourceLineNo">203</span> // Always start with pread unless user specific stream. Will change to stream later if<a name="line.203"></a> |
| <span class="sourceLineNo">204</span> // readType is default if the scan keeps running for a long time.<a name="line.204"></a> |
| <span class="sourceLineNo">205</span> this.scanUsePread = this.readType != Scan.ReadType.STREAM;<a name="line.205"></a> |
| <span class="sourceLineNo">206</span> }<a name="line.206"></a> |
| <span class="sourceLineNo">207</span> this.preadMaxBytes = scanInfo.getPreadMaxBytes();<a name="line.207"></a> |
| <span class="sourceLineNo">208</span> this.cellsPerHeartbeatCheck = scanInfo.getCellsPerTimeoutCheck();<a name="line.208"></a> |
| <span class="sourceLineNo">209</span> // Parallel seeking is on if the config allows and more there is more than one store file.<a name="line.209"></a> |
| <span class="sourceLineNo">210</span> if (store != null && store.getStorefilesCount() > 1) {<a name="line.210"></a> |
| <span class="sourceLineNo">211</span> RegionServerServices rsService = store.getHRegion().getRegionServerServices();<a name="line.211"></a> |
| <span class="sourceLineNo">212</span> if (rsService != null && scanInfo.isParallelSeekEnabled()) {<a name="line.212"></a> |
| <span class="sourceLineNo">213</span> this.parallelSeekEnabled = true;<a name="line.213"></a> |
| <span class="sourceLineNo">214</span> this.executor = rsService.getExecutorService();<a name="line.214"></a> |
| <span class="sourceLineNo">215</span> }<a name="line.215"></a> |
| <span class="sourceLineNo">216</span> }<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> private void addCurrentScanners(List<? extends KeyValueScanner> scanners) {<a name="line.219"></a> |
| <span class="sourceLineNo">220</span> this.currentScanners.addAll(scanners);<a name="line.220"></a> |
| <span class="sourceLineNo">221</span> }<a name="line.221"></a> |
| <span class="sourceLineNo">222</span><a name="line.222"></a> |
| <span class="sourceLineNo">223</span> /**<a name="line.223"></a> |
| <span class="sourceLineNo">224</span> * Opens a scanner across memstore, snapshot, and all StoreFiles. Assumes we<a name="line.224"></a> |
| <span class="sourceLineNo">225</span> * are not in a compaction.<a name="line.225"></a> |
| <span class="sourceLineNo">226</span> *<a name="line.226"></a> |
| <span class="sourceLineNo">227</span> * @param store who we scan<a name="line.227"></a> |
| <span class="sourceLineNo">228</span> * @param scan the spec<a name="line.228"></a> |
| <span class="sourceLineNo">229</span> * @param columns which columns we are scanning<a name="line.229"></a> |
| <span class="sourceLineNo">230</span> * @throws IOException<a name="line.230"></a> |
| <span class="sourceLineNo">231</span> */<a name="line.231"></a> |
| <span class="sourceLineNo">232</span> public StoreScanner(HStore store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,<a name="line.232"></a> |
| <span class="sourceLineNo">233</span> long readPt) throws IOException {<a name="line.233"></a> |
| <span class="sourceLineNo">234</span> this(store, scan, scanInfo, columns != null ? columns.size() : 0, readPt,<a name="line.234"></a> |
| <span class="sourceLineNo">235</span> scan.getCacheBlocks(), ScanType.USER_SCAN);<a name="line.235"></a> |
| <span class="sourceLineNo">236</span> if (columns != null && scan.isRaw()) {<a name="line.236"></a> |
| <span class="sourceLineNo">237</span> throw new DoNotRetryIOException("Cannot specify any column for a raw scan");<a name="line.237"></a> |
| <span class="sourceLineNo">238</span> }<a name="line.238"></a> |
| <span class="sourceLineNo">239</span> matcher = UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now,<a name="line.239"></a> |
| <span class="sourceLineNo">240</span> store.getCoprocessorHost());<a name="line.240"></a> |
| <span class="sourceLineNo">241</span><a name="line.241"></a> |
| <span class="sourceLineNo">242</span> store.addChangedReaderObserver(this);<a name="line.242"></a> |
| <span class="sourceLineNo">243</span><a name="line.243"></a> |
| <span class="sourceLineNo">244</span> List<KeyValueScanner> scanners = null;<a name="line.244"></a> |
| <span class="sourceLineNo">245</span> try {<a name="line.245"></a> |
| <span class="sourceLineNo">246</span> // Pass columns to try to filter out unnecessary StoreFiles.<a name="line.246"></a> |
| <span class="sourceLineNo">247</span> scanners = selectScannersFrom(store,<a name="line.247"></a> |
| <span class="sourceLineNo">248</span> store.getScanners(cacheBlocks, scanUsePread, false, matcher, scan.getStartRow(),<a name="line.248"></a> |
| <span class="sourceLineNo">249</span> scan.includeStartRow(), scan.getStopRow(), scan.includeStopRow(), this.readPt));<a name="line.249"></a> |
| <span class="sourceLineNo">250</span><a name="line.250"></a> |
| <span class="sourceLineNo">251</span> // Seek all scanners to the start of the Row (or if the exact matching row<a name="line.251"></a> |
| <span class="sourceLineNo">252</span> // key does not exist, then to the start of the next matching Row).<a name="line.252"></a> |
| <span class="sourceLineNo">253</span> // Always check bloom filter to optimize the top row seek for delete<a name="line.253"></a> |
| <span class="sourceLineNo">254</span> // family marker.<a name="line.254"></a> |
| <span class="sourceLineNo">255</span> seekScanners(scanners, matcher.getStartKey(), explicitColumnQuery && lazySeekEnabledGlobally,<a name="line.255"></a> |
| <span class="sourceLineNo">256</span> parallelSeekEnabled);<a name="line.256"></a> |
| <span class="sourceLineNo">257</span><a name="line.257"></a> |
| <span class="sourceLineNo">258</span> // set storeLimit<a name="line.258"></a> |
| <span class="sourceLineNo">259</span> this.storeLimit = scan.getMaxResultsPerColumnFamily();<a name="line.259"></a> |
| <span class="sourceLineNo">260</span><a name="line.260"></a> |
| <span class="sourceLineNo">261</span> // set rowOffset<a name="line.261"></a> |
| <span class="sourceLineNo">262</span> this.storeOffset = scan.getRowOffsetPerColumnFamily();<a name="line.262"></a> |
| <span class="sourceLineNo">263</span> addCurrentScanners(scanners);<a name="line.263"></a> |
| <span class="sourceLineNo">264</span> // Combine all seeked scanners with a heap<a name="line.264"></a> |
| <span class="sourceLineNo">265</span> resetKVHeap(scanners, comparator);<a name="line.265"></a> |
| <span class="sourceLineNo">266</span> } catch (IOException e) {<a name="line.266"></a> |
| <span class="sourceLineNo">267</span> clearAndClose(scanners);<a name="line.267"></a> |
| <span class="sourceLineNo">268</span> // remove us from the HStore#changedReaderObservers here or we'll have no chance to<a name="line.268"></a> |
| <span class="sourceLineNo">269</span> // and might cause memory leak<a name="line.269"></a> |
| <span class="sourceLineNo">270</span> store.deleteChangedReaderObserver(this);<a name="line.270"></a> |
| <span class="sourceLineNo">271</span> throw e;<a name="line.271"></a> |
| <span class="sourceLineNo">272</span> }<a name="line.272"></a> |
| <span class="sourceLineNo">273</span> }<a name="line.273"></a> |
| <span class="sourceLineNo">274</span><a name="line.274"></a> |
| <span class="sourceLineNo">275</span> // a dummy scan instance for compaction.<a name="line.275"></a> |
| <span class="sourceLineNo">276</span> private static final Scan SCAN_FOR_COMPACTION = new Scan();<a name="line.276"></a> |
| <span class="sourceLineNo">277</span><a name="line.277"></a> |
| <span class="sourceLineNo">278</span> /**<a name="line.278"></a> |
| <span class="sourceLineNo">279</span> * Used for store file compaction and memstore compaction.<a name="line.279"></a> |
| <span class="sourceLineNo">280</span> * <p><a name="line.280"></a> |
| <span class="sourceLineNo">281</span> * Opens a scanner across specified StoreFiles/MemStoreSegments.<a name="line.281"></a> |
| <span class="sourceLineNo">282</span> * @param store who we scan<a name="line.282"></a> |
| <span class="sourceLineNo">283</span> * @param scanners ancillary scanners<a name="line.283"></a> |
| <span class="sourceLineNo">284</span> * @param smallestReadPoint the readPoint that we should use for tracking versions<a name="line.284"></a> |
| <span class="sourceLineNo">285</span> */<a name="line.285"></a> |
| <span class="sourceLineNo">286</span> public StoreScanner(HStore store, ScanInfo scanInfo, List<? extends KeyValueScanner> scanners,<a name="line.286"></a> |
| <span class="sourceLineNo">287</span> ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {<a name="line.287"></a> |
| <span class="sourceLineNo">288</span> this(store, scanInfo, scanners, scanType, smallestReadPoint, earliestPutTs, null, null);<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> /**<a name="line.291"></a> |
| <span class="sourceLineNo">292</span> * Used for compactions that drop deletes from a limited range of rows.<a name="line.292"></a> |
| <span class="sourceLineNo">293</span> * <p><a name="line.293"></a> |
| <span class="sourceLineNo">294</span> * Opens a scanner across specified StoreFiles.<a name="line.294"></a> |
| <span class="sourceLineNo">295</span> * @param store who we scan<a name="line.295"></a> |
| <span class="sourceLineNo">296</span> * @param scanners ancillary scanners<a name="line.296"></a> |
| <span class="sourceLineNo">297</span> * @param smallestReadPoint the readPoint that we should use for tracking versions<a name="line.297"></a> |
| <span class="sourceLineNo">298</span> * @param dropDeletesFromRow The inclusive left bound of the range; can be EMPTY_START_ROW.<a name="line.298"></a> |
| <span class="sourceLineNo">299</span> * @param dropDeletesToRow The exclusive right bound of the range; can be EMPTY_END_ROW.<a name="line.299"></a> |
| <span class="sourceLineNo">300</span> */<a name="line.300"></a> |
| <span class="sourceLineNo">301</span> public StoreScanner(HStore store, ScanInfo scanInfo, List<? extends KeyValueScanner> scanners,<a name="line.301"></a> |
| <span class="sourceLineNo">302</span> long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,<a name="line.302"></a> |
| <span class="sourceLineNo">303</span> byte[] dropDeletesToRow) throws IOException {<a name="line.303"></a> |
| <span class="sourceLineNo">304</span> this(store, scanInfo, scanners, ScanType.COMPACT_RETAIN_DELETES, smallestReadPoint,<a name="line.304"></a> |
| <span class="sourceLineNo">305</span> earliestPutTs, dropDeletesFromRow, dropDeletesToRow);<a name="line.305"></a> |
| <span class="sourceLineNo">306</span> }<a name="line.306"></a> |
| <span class="sourceLineNo">307</span><a name="line.307"></a> |
| <span class="sourceLineNo">308</span> private StoreScanner(HStore store, ScanInfo scanInfo, List<? extends KeyValueScanner> scanners,<a name="line.308"></a> |
| <span class="sourceLineNo">309</span> ScanType scanType, long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,<a name="line.309"></a> |
| <span class="sourceLineNo">310</span> byte[] dropDeletesToRow) throws IOException {<a name="line.310"></a> |
| <span class="sourceLineNo">311</span> this(store, SCAN_FOR_COMPACTION, scanInfo, 0,<a name="line.311"></a> |
| <span class="sourceLineNo">312</span> store.getHRegion().getReadPoint(IsolationLevel.READ_COMMITTED), false, scanType);<a name="line.312"></a> |
| <span class="sourceLineNo">313</span> assert scanType != ScanType.USER_SCAN;<a name="line.313"></a> |
| <span class="sourceLineNo">314</span> matcher =<a name="line.314"></a> |
| <span class="sourceLineNo">315</span> CompactionScanQueryMatcher.create(scanInfo, scanType, smallestReadPoint, earliestPutTs,<a name="line.315"></a> |
| <span class="sourceLineNo">316</span> oldestUnexpiredTS, now, dropDeletesFromRow, dropDeletesToRow, store.getCoprocessorHost());<a name="line.316"></a> |
| <span class="sourceLineNo">317</span><a name="line.317"></a> |
| <span class="sourceLineNo">318</span> // Filter the list of scanners using Bloom filters, time range, TTL, etc.<a name="line.318"></a> |
| <span class="sourceLineNo">319</span> scanners = selectScannersFrom(store, scanners);<a name="line.319"></a> |
| <span class="sourceLineNo">320</span><a name="line.320"></a> |
| <span class="sourceLineNo">321</span> // Seek all scanners to the initial key<a name="line.321"></a> |
| <span class="sourceLineNo">322</span> seekScanners(scanners, matcher.getStartKey(), false, parallelSeekEnabled);<a name="line.322"></a> |
| <span class="sourceLineNo">323</span> addCurrentScanners(scanners);<a name="line.323"></a> |
| <span class="sourceLineNo">324</span> // Combine all seeked scanners with a heap<a name="line.324"></a> |
| <span class="sourceLineNo">325</span> resetKVHeap(scanners, comparator);<a name="line.325"></a> |
| <span class="sourceLineNo">326</span> }<a name="line.326"></a> |
| <span class="sourceLineNo">327</span><a name="line.327"></a> |
| <span class="sourceLineNo">328</span> private void seekAllScanner(ScanInfo scanInfo, List<? extends KeyValueScanner> scanners)<a name="line.328"></a> |
| <span class="sourceLineNo">329</span> throws IOException {<a name="line.329"></a> |
| <span class="sourceLineNo">330</span> // Seek all scanners to the initial key<a name="line.330"></a> |
| <span class="sourceLineNo">331</span> seekScanners(scanners, matcher.getStartKey(), false, parallelSeekEnabled);<a name="line.331"></a> |
| <span class="sourceLineNo">332</span> addCurrentScanners(scanners);<a name="line.332"></a> |
| <span class="sourceLineNo">333</span> resetKVHeap(scanners, comparator);<a name="line.333"></a> |
| <span class="sourceLineNo">334</span> }<a name="line.334"></a> |
| <span class="sourceLineNo">335</span><a name="line.335"></a> |
| <span class="sourceLineNo">336</span> // For mob compaction only as we do not have a Store instance when doing mob compaction.<a name="line.336"></a> |
| <span class="sourceLineNo">337</span> public StoreScanner(ScanInfo scanInfo, ScanType scanType,<a name="line.337"></a> |
| <span class="sourceLineNo">338</span> List<? extends KeyValueScanner> scanners) throws IOException {<a name="line.338"></a> |
| <span class="sourceLineNo">339</span> this(null, SCAN_FOR_COMPACTION, scanInfo, 0, Long.MAX_VALUE, false, scanType);<a name="line.339"></a> |
| <span class="sourceLineNo">340</span> assert scanType != ScanType.USER_SCAN;<a name="line.340"></a> |
| <span class="sourceLineNo">341</span> this.matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, Long.MAX_VALUE, 0L,<a name="line.341"></a> |
| <span class="sourceLineNo">342</span> oldestUnexpiredTS, now, null, null, null);<a name="line.342"></a> |
| <span class="sourceLineNo">343</span> seekAllScanner(scanInfo, scanners);<a name="line.343"></a> |
| <span class="sourceLineNo">344</span> }<a name="line.344"></a> |
| <span class="sourceLineNo">345</span><a name="line.345"></a> |
| <span class="sourceLineNo">346</span> // Used to instantiate a scanner for user scan in test<a name="line.346"></a> |
| <span class="sourceLineNo">347</span> @VisibleForTesting<a name="line.347"></a> |
| <span class="sourceLineNo">348</span> StoreScanner(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> columns,<a name="line.348"></a> |
| <span class="sourceLineNo">349</span> List<? extends KeyValueScanner> scanners, ScanType scanType) throws IOException {<a name="line.349"></a> |
| <span class="sourceLineNo">350</span> // 0 is passed as readpoint because the test bypasses Store<a name="line.350"></a> |
| <span class="sourceLineNo">351</span> this(null, scan, scanInfo, columns != null ? columns.size() : 0, 0L, scan.getCacheBlocks(),<a name="line.351"></a> |
| <span class="sourceLineNo">352</span> scanType);<a name="line.352"></a> |
| <span class="sourceLineNo">353</span> if (scanType == ScanType.USER_SCAN) {<a name="line.353"></a> |
| <span class="sourceLineNo">354</span> this.matcher =<a name="line.354"></a> |
| <span class="sourceLineNo">355</span> UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now, null);<a name="line.355"></a> |
| <span class="sourceLineNo">356</span> } else {<a name="line.356"></a> |
| <span class="sourceLineNo">357</span> this.matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, Long.MAX_VALUE,<a name="line.357"></a> |
| <span class="sourceLineNo">358</span> HConstants.OLDEST_TIMESTAMP, oldestUnexpiredTS, now, null, null, null);<a name="line.358"></a> |
| <span class="sourceLineNo">359</span> }<a name="line.359"></a> |
| <span class="sourceLineNo">360</span> seekAllScanner(scanInfo, scanners);<a name="line.360"></a> |
| <span class="sourceLineNo">361</span> }<a name="line.361"></a> |
| <span class="sourceLineNo">362</span><a name="line.362"></a> |
| <span class="sourceLineNo">363</span> // Used to instantiate a scanner for user scan in test<a name="line.363"></a> |
| <span class="sourceLineNo">364</span> @VisibleForTesting<a name="line.364"></a> |
| <span class="sourceLineNo">365</span> StoreScanner(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> columns,<a name="line.365"></a> |
| <span class="sourceLineNo">366</span> List<? extends KeyValueScanner> scanners) throws IOException {<a name="line.366"></a> |
| <span class="sourceLineNo">367</span> // 0 is passed as readpoint because the test bypasses Store<a name="line.367"></a> |
| <span class="sourceLineNo">368</span> this(null, scan, scanInfo, columns != null ? columns.size() : 0, 0L, scan.getCacheBlocks(),<a name="line.368"></a> |
| <span class="sourceLineNo">369</span> ScanType.USER_SCAN);<a name="line.369"></a> |
| <span class="sourceLineNo">370</span> this.matcher =<a name="line.370"></a> |
| <span class="sourceLineNo">371</span> UserScanQueryMatcher.create(scan, scanInfo, columns, oldestUnexpiredTS, now, null);<a name="line.371"></a> |
| <span class="sourceLineNo">372</span> seekAllScanner(scanInfo, scanners);<a name="line.372"></a> |
| <span class="sourceLineNo">373</span> }<a name="line.373"></a> |
| <span class="sourceLineNo">374</span><a name="line.374"></a> |
| <span class="sourceLineNo">375</span> // Used to instantiate a scanner for compaction in test<a name="line.375"></a> |
| <span class="sourceLineNo">376</span> @VisibleForTesting<a name="line.376"></a> |
| <span class="sourceLineNo">377</span> StoreScanner(ScanInfo scanInfo, int maxVersions, ScanType scanType,<a name="line.377"></a> |
| <span class="sourceLineNo">378</span> List<? extends KeyValueScanner> scanners) throws IOException {<a name="line.378"></a> |
| <span class="sourceLineNo">379</span> // 0 is passed as readpoint because the test bypasses Store<a name="line.379"></a> |
| <span class="sourceLineNo">380</span> this(null, maxVersions > 0 ? new Scan().readVersions(maxVersions)<a name="line.380"></a> |
| <span class="sourceLineNo">381</span> : SCAN_FOR_COMPACTION, scanInfo, 0, 0L, false, scanType);<a name="line.381"></a> |
| <span class="sourceLineNo">382</span> this.matcher = CompactionScanQueryMatcher.create(scanInfo, scanType, Long.MAX_VALUE,<a name="line.382"></a> |
| <span class="sourceLineNo">383</span> HConstants.OLDEST_TIMESTAMP, oldestUnexpiredTS, now, null, null, null);<a name="line.383"></a> |
| <span class="sourceLineNo">384</span> seekAllScanner(scanInfo, scanners);<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> @VisibleForTesting<a name="line.387"></a> |
| <span class="sourceLineNo">388</span> boolean isScanUsePread() {<a name="line.388"></a> |
| <span class="sourceLineNo">389</span> return this.scanUsePread;<a name="line.389"></a> |
| <span class="sourceLineNo">390</span> }<a name="line.390"></a> |
| <span class="sourceLineNo">391</span> /**<a name="line.391"></a> |
| <span class="sourceLineNo">392</span> * Seek the specified scanners with the given key<a name="line.392"></a> |
| <span class="sourceLineNo">393</span> * @param scanners<a name="line.393"></a> |
| <span class="sourceLineNo">394</span> * @param seekKey<a name="line.394"></a> |
| <span class="sourceLineNo">395</span> * @param isLazy true if using lazy seek<a name="line.395"></a> |
| <span class="sourceLineNo">396</span> * @param isParallelSeek true if using parallel seek<a name="line.396"></a> |
| <span class="sourceLineNo">397</span> * @throws IOException<a name="line.397"></a> |
| <span class="sourceLineNo">398</span> */<a name="line.398"></a> |
| <span class="sourceLineNo">399</span> protected void seekScanners(List<? extends KeyValueScanner> scanners,<a name="line.399"></a> |
| <span class="sourceLineNo">400</span> Cell seekKey, boolean isLazy, boolean isParallelSeek)<a name="line.400"></a> |
| <span class="sourceLineNo">401</span> throws IOException {<a name="line.401"></a> |
| <span class="sourceLineNo">402</span> // Seek all scanners to the start of the Row (or if the exact matching row<a name="line.402"></a> |
| <span class="sourceLineNo">403</span> // key does not exist, then to the start of the next matching Row).<a name="line.403"></a> |
| <span class="sourceLineNo">404</span> // Always check bloom filter to optimize the top row seek for delete<a name="line.404"></a> |
| <span class="sourceLineNo">405</span> // family marker.<a name="line.405"></a> |
| <span class="sourceLineNo">406</span> if (isLazy) {<a name="line.406"></a> |
| <span class="sourceLineNo">407</span> for (KeyValueScanner scanner : scanners) {<a name="line.407"></a> |
| <span class="sourceLineNo">408</span> scanner.requestSeek(seekKey, false, true);<a name="line.408"></a> |
| <span class="sourceLineNo">409</span> }<a name="line.409"></a> |
| <span class="sourceLineNo">410</span> } else {<a name="line.410"></a> |
| <span class="sourceLineNo">411</span> if (!isParallelSeek) {<a name="line.411"></a> |
| <span class="sourceLineNo">412</span> long totalScannersSoughtBytes = 0;<a name="line.412"></a> |
| <span class="sourceLineNo">413</span> for (KeyValueScanner scanner : scanners) {<a name="line.413"></a> |
| <span class="sourceLineNo">414</span> if (matcher.isUserScan() && totalScannersSoughtBytes >= maxRowSize) {<a name="line.414"></a> |
| <span class="sourceLineNo">415</span> throw new RowTooBigException("Max row size allowed: " + maxRowSize<a name="line.415"></a> |
| <span class="sourceLineNo">416</span> + ", but row is bigger than that");<a name="line.416"></a> |
| <span class="sourceLineNo">417</span> }<a name="line.417"></a> |
| <span class="sourceLineNo">418</span> scanner.seek(seekKey);<a name="line.418"></a> |
| <span class="sourceLineNo">419</span> Cell c = scanner.peek();<a name="line.419"></a> |
| <span class="sourceLineNo">420</span> if (c != null) {<a name="line.420"></a> |
| <span class="sourceLineNo">421</span> totalScannersSoughtBytes += PrivateCellUtil.estimatedSerializedSizeOf(c);<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> } else {<a name="line.424"></a> |
| <span class="sourceLineNo">425</span> parallelSeek(scanners, seekKey);<a name="line.425"></a> |
| <span class="sourceLineNo">426</span> }<a name="line.426"></a> |
| <span class="sourceLineNo">427</span> }<a name="line.427"></a> |
| <span class="sourceLineNo">428</span> }<a name="line.428"></a> |
| <span class="sourceLineNo">429</span><a name="line.429"></a> |
| <span class="sourceLineNo">430</span> @VisibleForTesting<a name="line.430"></a> |
| <span class="sourceLineNo">431</span> protected void resetKVHeap(List<? extends KeyValueScanner> scanners,<a name="line.431"></a> |
| <span class="sourceLineNo">432</span> CellComparator comparator) throws IOException {<a name="line.432"></a> |
| <span class="sourceLineNo">433</span> // Combine all seeked scanners with a heap<a name="line.433"></a> |
| <span class="sourceLineNo">434</span> heap = newKVHeap(scanners, comparator);<a name="line.434"></a> |
| <span class="sourceLineNo">435</span> }<a name="line.435"></a> |
| <span class="sourceLineNo">436</span><a name="line.436"></a> |
| <span class="sourceLineNo">437</span> protected KeyValueHeap newKVHeap(List<? extends KeyValueScanner> scanners,<a name="line.437"></a> |
| <span class="sourceLineNo">438</span> CellComparator comparator) throws IOException {<a name="line.438"></a> |
| <span class="sourceLineNo">439</span> return new KeyValueHeap(scanners, comparator);<a name="line.439"></a> |
| <span class="sourceLineNo">440</span> }<a name="line.440"></a> |
| <span class="sourceLineNo">441</span><a name="line.441"></a> |
| <span class="sourceLineNo">442</span> /**<a name="line.442"></a> |
| <span class="sourceLineNo">443</span> * Filters the given list of scanners using Bloom filter, time range, and TTL.<a name="line.443"></a> |
| <span class="sourceLineNo">444</span> * <p><a name="line.444"></a> |
| <span class="sourceLineNo">445</span> * Will be overridden by testcase so declared as protected.<a name="line.445"></a> |
| <span class="sourceLineNo">446</span> */<a name="line.446"></a> |
| <span class="sourceLineNo">447</span> @VisibleForTesting<a name="line.447"></a> |
| <span class="sourceLineNo">448</span> protected List<KeyValueScanner> selectScannersFrom(HStore store,<a name="line.448"></a> |
| <span class="sourceLineNo">449</span> List<? extends KeyValueScanner> allScanners) {<a name="line.449"></a> |
| <span class="sourceLineNo">450</span> boolean memOnly;<a name="line.450"></a> |
| <span class="sourceLineNo">451</span> boolean filesOnly;<a name="line.451"></a> |
| <span class="sourceLineNo">452</span> if (scan instanceof InternalScan) {<a name="line.452"></a> |
| <span class="sourceLineNo">453</span> InternalScan iscan = (InternalScan) scan;<a name="line.453"></a> |
| <span class="sourceLineNo">454</span> memOnly = iscan.isCheckOnlyMemStore();<a name="line.454"></a> |
| <span class="sourceLineNo">455</span> filesOnly = iscan.isCheckOnlyStoreFiles();<a name="line.455"></a> |
| <span class="sourceLineNo">456</span> } else {<a name="line.456"></a> |
| <span class="sourceLineNo">457</span> memOnly = false;<a name="line.457"></a> |
| <span class="sourceLineNo">458</span> filesOnly = false;<a name="line.458"></a> |
| <span class="sourceLineNo">459</span> }<a name="line.459"></a> |
| <span class="sourceLineNo">460</span><a name="line.460"></a> |
| <span class="sourceLineNo">461</span> List<KeyValueScanner> scanners = new ArrayList<>(allScanners.size());<a name="line.461"></a> |
| <span class="sourceLineNo">462</span><a name="line.462"></a> |
| <span class="sourceLineNo">463</span> // We can only exclude store files based on TTL if minVersions is set to 0.<a name="line.463"></a> |
| <span class="sourceLineNo">464</span> // Otherwise, we might have to return KVs that have technically expired.<a name="line.464"></a> |
| <span class="sourceLineNo">465</span> long expiredTimestampCutoff = minVersions == 0 ? oldestUnexpiredTS : Long.MIN_VALUE;<a name="line.465"></a> |
| <span class="sourceLineNo">466</span><a name="line.466"></a> |
| <span class="sourceLineNo">467</span> // include only those scan files which pass all filters<a name="line.467"></a> |
| <span class="sourceLineNo">468</span> for (KeyValueScanner kvs : allScanners) {<a name="line.468"></a> |
| <span class="sourceLineNo">469</span> boolean isFile = kvs.isFileScanner();<a name="line.469"></a> |
| <span class="sourceLineNo">470</span> if ((!isFile && filesOnly) || (isFile && memOnly)) {<a name="line.470"></a> |
| <span class="sourceLineNo">471</span> continue;<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> if (kvs.shouldUseScanner(scan, store, expiredTimestampCutoff)) {<a name="line.474"></a> |
| <span class="sourceLineNo">475</span> scanners.add(kvs);<a name="line.475"></a> |
| <span class="sourceLineNo">476</span> } else {<a name="line.476"></a> |
| <span class="sourceLineNo">477</span> kvs.close();<a name="line.477"></a> |
| <span class="sourceLineNo">478</span> }<a name="line.478"></a> |
| <span class="sourceLineNo">479</span> }<a name="line.479"></a> |
| <span class="sourceLineNo">480</span> return scanners;<a name="line.480"></a> |
| <span class="sourceLineNo">481</span> }<a name="line.481"></a> |
| <span class="sourceLineNo">482</span><a name="line.482"></a> |
| <span class="sourceLineNo">483</span> @Override<a name="line.483"></a> |
| <span class="sourceLineNo">484</span> public Cell peek() {<a name="line.484"></a> |
| <span class="sourceLineNo">485</span> return heap != null ? heap.peek() : null;<a name="line.485"></a> |
| <span class="sourceLineNo">486</span> }<a name="line.486"></a> |
| <span class="sourceLineNo">487</span><a name="line.487"></a> |
| <span class="sourceLineNo">488</span> @Override<a name="line.488"></a> |
| <span class="sourceLineNo">489</span> public KeyValue next() {<a name="line.489"></a> |
| <span class="sourceLineNo">490</span> // throw runtime exception perhaps?<a name="line.490"></a> |
| <span class="sourceLineNo">491</span> throw new RuntimeException("Never call StoreScanner.next()");<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> @Override<a name="line.494"></a> |
| <span class="sourceLineNo">495</span> public void close() {<a name="line.495"></a> |
| <span class="sourceLineNo">496</span> close(true);<a name="line.496"></a> |
| <span class="sourceLineNo">497</span> }<a name="line.497"></a> |
| <span class="sourceLineNo">498</span><a name="line.498"></a> |
| <span class="sourceLineNo">499</span> private void close(boolean withDelayedScannersClose) {<a name="line.499"></a> |
| <span class="sourceLineNo">500</span> closeLock.lock();<a name="line.500"></a> |
| <span class="sourceLineNo">501</span> // If the closeLock is acquired then any subsequent updateReaders()<a name="line.501"></a> |
| <span class="sourceLineNo">502</span> // call is ignored.<a name="line.502"></a> |
| <span class="sourceLineNo">503</span> try {<a name="line.503"></a> |
| <span class="sourceLineNo">504</span> if (this.closing) {<a name="line.504"></a> |
| <span class="sourceLineNo">505</span> return;<a name="line.505"></a> |
| <span class="sourceLineNo">506</span> }<a name="line.506"></a> |
| <span class="sourceLineNo">507</span> if (withDelayedScannersClose) {<a name="line.507"></a> |
| <span class="sourceLineNo">508</span> this.closing = true;<a name="line.508"></a> |
| <span class="sourceLineNo">509</span> }<a name="line.509"></a> |
| <span class="sourceLineNo">510</span> // For mob compaction, we do not have a store.<a name="line.510"></a> |
| <span class="sourceLineNo">511</span> if (this.store != null) {<a name="line.511"></a> |
| <span class="sourceLineNo">512</span> this.store.deleteChangedReaderObserver(this);<a name="line.512"></a> |
| <span class="sourceLineNo">513</span> }<a name="line.513"></a> |
| <span class="sourceLineNo">514</span> if (withDelayedScannersClose) {<a name="line.514"></a> |
| <span class="sourceLineNo">515</span> clearAndClose(scannersForDelayedClose);<a name="line.515"></a> |
| <span class="sourceLineNo">516</span> clearAndClose(memStoreScannersAfterFlush);<a name="line.516"></a> |
| <span class="sourceLineNo">517</span> clearAndClose(flushedstoreFileScanners);<a name="line.517"></a> |
| <span class="sourceLineNo">518</span> if (this.heap != null) {<a name="line.518"></a> |
| <span class="sourceLineNo">519</span> this.heap.close();<a name="line.519"></a> |
| <span class="sourceLineNo">520</span> this.currentScanners.clear();<a name="line.520"></a> |
| <span class="sourceLineNo">521</span> this.heap = null; // CLOSED!<a name="line.521"></a> |
| <span class="sourceLineNo">522</span> }<a name="line.522"></a> |
| <span class="sourceLineNo">523</span> } else {<a name="line.523"></a> |
| <span class="sourceLineNo">524</span> if (this.heap != null) {<a name="line.524"></a> |
| <span class="sourceLineNo">525</span> this.scannersForDelayedClose.add(this.heap);<a name="line.525"></a> |
| <span class="sourceLineNo">526</span> this.currentScanners.clear();<a name="line.526"></a> |
| <span class="sourceLineNo">527</span> this.heap = null;<a name="line.527"></a> |
| <span class="sourceLineNo">528</span> }<a name="line.528"></a> |
| <span class="sourceLineNo">529</span> }<a name="line.529"></a> |
| <span class="sourceLineNo">530</span> } finally {<a name="line.530"></a> |
| <span class="sourceLineNo">531</span> closeLock.unlock();<a name="line.531"></a> |
| <span class="sourceLineNo">532</span> }<a name="line.532"></a> |
| <span class="sourceLineNo">533</span> }<a name="line.533"></a> |
| <span class="sourceLineNo">534</span><a name="line.534"></a> |
| <span class="sourceLineNo">535</span> @Override<a name="line.535"></a> |
| <span class="sourceLineNo">536</span> public boolean seek(Cell key) throws IOException {<a name="line.536"></a> |
| <span class="sourceLineNo">537</span> if (checkFlushed()) {<a name="line.537"></a> |
| <span class="sourceLineNo">538</span> reopenAfterFlush();<a name="line.538"></a> |
| <span class="sourceLineNo">539</span> }<a name="line.539"></a> |
| <span class="sourceLineNo">540</span> return this.heap.seek(key);<a name="line.540"></a> |
| <span class="sourceLineNo">541</span> }<a name="line.541"></a> |
| <span class="sourceLineNo">542</span><a name="line.542"></a> |
| <span class="sourceLineNo">543</span> /**<a name="line.543"></a> |
| <span class="sourceLineNo">544</span> * Get the next row of values from this Store.<a name="line.544"></a> |
| <span class="sourceLineNo">545</span> * @param outResult<a name="line.545"></a> |
| <span class="sourceLineNo">546</span> * @param scannerContext<a name="line.546"></a> |
| <span class="sourceLineNo">547</span> * @return true if there are more rows, false if scanner is done<a name="line.547"></a> |
| <span class="sourceLineNo">548</span> */<a name="line.548"></a> |
| <span class="sourceLineNo">549</span> @Override<a name="line.549"></a> |
| <span class="sourceLineNo">550</span> public boolean next(List<Cell> outResult, ScannerContext scannerContext) throws IOException {<a name="line.550"></a> |
| <span class="sourceLineNo">551</span> if (scannerContext == null) {<a name="line.551"></a> |
| <span class="sourceLineNo">552</span> throw new IllegalArgumentException("Scanner context cannot be null");<a name="line.552"></a> |
| <span class="sourceLineNo">553</span> }<a name="line.553"></a> |
| <span class="sourceLineNo">554</span> if (checkFlushed() && reopenAfterFlush()) {<a name="line.554"></a> |
| <span class="sourceLineNo">555</span> return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();<a name="line.555"></a> |
| <span class="sourceLineNo">556</span> }<a name="line.556"></a> |
| <span class="sourceLineNo">557</span><a name="line.557"></a> |
| <span class="sourceLineNo">558</span> // if the heap was left null, then the scanners had previously run out anyways, close and<a name="line.558"></a> |
| <span class="sourceLineNo">559</span> // return.<a name="line.559"></a> |
| <span class="sourceLineNo">560</span> if (this.heap == null) {<a name="line.560"></a> |
| <span class="sourceLineNo">561</span> // By this time partial close should happened because already heap is null<a name="line.561"></a> |
| <span class="sourceLineNo">562</span> close(false);// Do all cleanup except heap.close()<a name="line.562"></a> |
| <span class="sourceLineNo">563</span> return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();<a name="line.563"></a> |
| <span class="sourceLineNo">564</span> }<a name="line.564"></a> |
| <span class="sourceLineNo">565</span><a name="line.565"></a> |
| <span class="sourceLineNo">566</span> Cell cell = this.heap.peek();<a name="line.566"></a> |
| <span class="sourceLineNo">567</span> if (cell == null) {<a name="line.567"></a> |
| <span class="sourceLineNo">568</span> close(false);// Do all cleanup except heap.close()<a name="line.568"></a> |
| <span class="sourceLineNo">569</span> return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();<a name="line.569"></a> |
| <span class="sourceLineNo">570</span> }<a name="line.570"></a> |
| <span class="sourceLineNo">571</span><a name="line.571"></a> |
| <span class="sourceLineNo">572</span> // only call setRow if the row changes; avoids confusing the query matcher<a name="line.572"></a> |
| <span class="sourceLineNo">573</span> // if scanning intra-row<a name="line.573"></a> |
| <span class="sourceLineNo">574</span><a name="line.574"></a> |
| <span class="sourceLineNo">575</span> // If no limits exists in the scope LimitScope.Between_Cells then we are sure we are changing<a name="line.575"></a> |
| <span class="sourceLineNo">576</span> // rows. Else it is possible we are still traversing the same row so we must perform the row<a name="line.576"></a> |
| <span class="sourceLineNo">577</span> // comparison.<a name="line.577"></a> |
| <span class="sourceLineNo">578</span> if (!scannerContext.hasAnyLimit(LimitScope.BETWEEN_CELLS) || matcher.currentRow() == null) {<a name="line.578"></a> |
| <span class="sourceLineNo">579</span> this.countPerRow = 0;<a name="line.579"></a> |
| <span class="sourceLineNo">580</span> matcher.setToNewRow(cell);<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> // Clear progress away unless invoker has indicated it should be kept.<a name="line.583"></a> |
| <span class="sourceLineNo">584</span> if (!scannerContext.getKeepProgress()) {<a name="line.584"></a> |
| <span class="sourceLineNo">585</span> scannerContext.clearProgress();<a name="line.585"></a> |
| <span class="sourceLineNo">586</span> }<a name="line.586"></a> |
| <span class="sourceLineNo">587</span><a name="line.587"></a> |
| <span class="sourceLineNo">588</span> int count = 0;<a name="line.588"></a> |
| <span class="sourceLineNo">589</span> long totalBytesRead = 0;<a name="line.589"></a> |
| <span class="sourceLineNo">590</span> // track the cells for metrics only if it is a user read request.<a name="line.590"></a> |
| <span class="sourceLineNo">591</span> boolean onlyFromMemstore = matcher.isUserScan();<a name="line.591"></a> |
| <span class="sourceLineNo">592</span> try {<a name="line.592"></a> |
| <span class="sourceLineNo">593</span> LOOP: do {<a name="line.593"></a> |
| <span class="sourceLineNo">594</span> // Update and check the time limit based on the configured value of cellsPerTimeoutCheck<a name="line.594"></a> |
| <span class="sourceLineNo">595</span> // Or if the preadMaxBytes is reached and we may want to return so we can switch to stream<a name="line.595"></a> |
| <span class="sourceLineNo">596</span> // in<a name="line.596"></a> |
| <span class="sourceLineNo">597</span> // the shipped method below.<a name="line.597"></a> |
| <span class="sourceLineNo">598</span> if (kvsScanned % cellsPerHeartbeatCheck == 0<a name="line.598"></a> |
| <span class="sourceLineNo">599</span> || (scanUsePread && readType == Scan.ReadType.DEFAULT && bytesRead > preadMaxBytes)) {<a name="line.599"></a> |
| <span class="sourceLineNo">600</span> if (scannerContext.checkTimeLimit(LimitScope.BETWEEN_CELLS)) {<a name="line.600"></a> |
| <span class="sourceLineNo">601</span> return scannerContext.setScannerState(NextState.TIME_LIMIT_REACHED).hasMoreValues();<a name="line.601"></a> |
| <span class="sourceLineNo">602</span> }<a name="line.602"></a> |
| <span class="sourceLineNo">603</span> }<a name="line.603"></a> |
| <span class="sourceLineNo">604</span> // Do object compare - we set prevKV from the same heap.<a name="line.604"></a> |
| <span class="sourceLineNo">605</span> if (prevCell != cell) {<a name="line.605"></a> |
| <span class="sourceLineNo">606</span> ++kvsScanned;<a name="line.606"></a> |
| <span class="sourceLineNo">607</span> }<a name="line.607"></a> |
| <span class="sourceLineNo">608</span> checkScanOrder(prevCell, cell, comparator);<a name="line.608"></a> |
| <span class="sourceLineNo">609</span> int cellSize = PrivateCellUtil.estimatedSerializedSizeOf(cell);<a name="line.609"></a> |
| <span class="sourceLineNo">610</span> bytesRead += cellSize;<a name="line.610"></a> |
| <span class="sourceLineNo">611</span> if (scanUsePread && readType == Scan.ReadType.DEFAULT && bytesRead > preadMaxBytes) {<a name="line.611"></a> |
| <span class="sourceLineNo">612</span> // return immediately if we want to switch from pread to stream. We need this because we<a name="line.612"></a> |
| <span class="sourceLineNo">613</span> // can<a name="line.613"></a> |
| <span class="sourceLineNo">614</span> // only switch in the shipped method, if user use a filter to filter out everything and<a name="line.614"></a> |
| <span class="sourceLineNo">615</span> // rpc<a name="line.615"></a> |
| <span class="sourceLineNo">616</span> // timeout is very large then the shipped method will never be called until the whole scan<a name="line.616"></a> |
| <span class="sourceLineNo">617</span> // is finished, but at that time we have already scan all the data...<a name="line.617"></a> |
| <span class="sourceLineNo">618</span> // See HBASE-20457 for more details.<a name="line.618"></a> |
| <span class="sourceLineNo">619</span> // And there is still a scenario that can not be handled. If we have a very large row,<a name="line.619"></a> |
| <span class="sourceLineNo">620</span> // which<a name="line.620"></a> |
| <span class="sourceLineNo">621</span> // have millions of qualifiers, and filter.filterRow is used, then even if we set the flag<a name="line.621"></a> |
| <span class="sourceLineNo">622</span> // here, we still need to scan all the qualifiers before returning...<a name="line.622"></a> |
| <span class="sourceLineNo">623</span> scannerContext.returnImmediately();<a name="line.623"></a> |
| <span class="sourceLineNo">624</span> }<a name="line.624"></a> |
| <span class="sourceLineNo">625</span> prevCell = cell;<a name="line.625"></a> |
| <span class="sourceLineNo">626</span> scannerContext.setLastPeekedCell(cell);<a name="line.626"></a> |
| <span class="sourceLineNo">627</span> topChanged = false;<a name="line.627"></a> |
| <span class="sourceLineNo">628</span> ScanQueryMatcher.MatchCode qcode = matcher.match(cell);<a name="line.628"></a> |
| <span class="sourceLineNo">629</span> switch (qcode) {<a name="line.629"></a> |
| <span class="sourceLineNo">630</span> case INCLUDE:<a name="line.630"></a> |
| <span class="sourceLineNo">631</span> case INCLUDE_AND_SEEK_NEXT_ROW:<a name="line.631"></a> |
| <span class="sourceLineNo">632</span> case INCLUDE_AND_SEEK_NEXT_COL:<a name="line.632"></a> |
| <span class="sourceLineNo">633</span><a name="line.633"></a> |
| <span class="sourceLineNo">634</span> Filter f = matcher.getFilter();<a name="line.634"></a> |
| <span class="sourceLineNo">635</span> if (f != null) {<a name="line.635"></a> |
| <span class="sourceLineNo">636</span> cell = f.transformCell(cell);<a name="line.636"></a> |
| <span class="sourceLineNo">637</span> }<a name="line.637"></a> |
| <span class="sourceLineNo">638</span> this.countPerRow++;<a name="line.638"></a> |
| <span class="sourceLineNo">639</span> if (storeLimit > -1 && this.countPerRow > (storeLimit + storeOffset)) {<a name="line.639"></a> |
| <span class="sourceLineNo">640</span> // do what SEEK_NEXT_ROW does.<a name="line.640"></a> |
| <span class="sourceLineNo">641</span> if (!matcher.moreRowsMayExistAfter(cell)) {<a name="line.641"></a> |
| <span class="sourceLineNo">642</span> close(false);// Do all cleanup except heap.close()<a name="line.642"></a> |
| <span class="sourceLineNo">643</span> return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();<a name="line.643"></a> |
| <span class="sourceLineNo">644</span> }<a name="line.644"></a> |
| <span class="sourceLineNo">645</span> matcher.clearCurrentRow();<a name="line.645"></a> |
| <span class="sourceLineNo">646</span> seekToNextRow(cell);<a name="line.646"></a> |
| <span class="sourceLineNo">647</span> break LOOP;<a name="line.647"></a> |
| <span class="sourceLineNo">648</span> }<a name="line.648"></a> |
| <span class="sourceLineNo">649</span><a name="line.649"></a> |
| <span class="sourceLineNo">650</span> // add to results only if we have skipped #storeOffset kvs<a name="line.650"></a> |
| <span class="sourceLineNo">651</span> // also update metric accordingly<a name="line.651"></a> |
| <span class="sourceLineNo">652</span> if (this.countPerRow > storeOffset) {<a name="line.652"></a> |
| <span class="sourceLineNo">653</span> outResult.add(cell);<a name="line.653"></a> |
| <span class="sourceLineNo">654</span><a name="line.654"></a> |
| <span class="sourceLineNo">655</span> // Update local tracking information<a name="line.655"></a> |
| <span class="sourceLineNo">656</span> count++;<a name="line.656"></a> |
| <span class="sourceLineNo">657</span> totalBytesRead += cellSize;<a name="line.657"></a> |
| <span class="sourceLineNo">658</span><a name="line.658"></a> |
| <span class="sourceLineNo">659</span> /**<a name="line.659"></a> |
| <span class="sourceLineNo">660</span> * Increment the metric if all the cells are from memstore.<a name="line.660"></a> |
| <span class="sourceLineNo">661</span> * If not we will account it for mixed reads<a name="line.661"></a> |
| <span class="sourceLineNo">662</span> */<a name="line.662"></a> |
| <span class="sourceLineNo">663</span> onlyFromMemstore = onlyFromMemstore && heap.isLatestCellFromMemstore();<a name="line.663"></a> |
| <span class="sourceLineNo">664</span> // Update the progress of the scanner context<a name="line.664"></a> |
| <span class="sourceLineNo">665</span> scannerContext.incrementSizeProgress(cellSize, cell.heapSize());<a name="line.665"></a> |
| <span class="sourceLineNo">666</span> scannerContext.incrementBatchProgress(1);<a name="line.666"></a> |
| <span class="sourceLineNo">667</span><a name="line.667"></a> |
| <span class="sourceLineNo">668</span> if (matcher.isUserScan() && totalBytesRead > maxRowSize) {<a name="line.668"></a> |
| <span class="sourceLineNo">669</span> String message = "Max row size allowed: " + maxRowSize<a name="line.669"></a> |
| <span class="sourceLineNo">670</span> + ", but the row is bigger than that, the row info: "<a name="line.670"></a> |
| <span class="sourceLineNo">671</span> + CellUtil.toString(cell, false) + ", already have process row cells = "<a name="line.671"></a> |
| <span class="sourceLineNo">672</span> + outResult.size() + ", it belong to region = "<a name="line.672"></a> |
| <span class="sourceLineNo">673</span> + store.getHRegion().getRegionInfo().getRegionNameAsString();<a name="line.673"></a> |
| <span class="sourceLineNo">674</span> LOG.warn(message);<a name="line.674"></a> |
| <span class="sourceLineNo">675</span> throw new RowTooBigException(message);<a name="line.675"></a> |
| <span class="sourceLineNo">676</span> }<a name="line.676"></a> |
| <span class="sourceLineNo">677</span> }<a name="line.677"></a> |
| <span class="sourceLineNo">678</span><a name="line.678"></a> |
| <span class="sourceLineNo">679</span> if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_ROW) {<a name="line.679"></a> |
| <span class="sourceLineNo">680</span> if (!matcher.moreRowsMayExistAfter(cell)) {<a name="line.680"></a> |
| <span class="sourceLineNo">681</span> close(false);// Do all cleanup except heap.close()<a name="line.681"></a> |
| <span class="sourceLineNo">682</span> return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();<a name="line.682"></a> |
| <span class="sourceLineNo">683</span> }<a name="line.683"></a> |
| <span class="sourceLineNo">684</span> matcher.clearCurrentRow();<a name="line.684"></a> |
| <span class="sourceLineNo">685</span> seekOrSkipToNextRow(cell);<a name="line.685"></a> |
| <span class="sourceLineNo">686</span> } else if (qcode == ScanQueryMatcher.MatchCode.INCLUDE_AND_SEEK_NEXT_COL) {<a name="line.686"></a> |
| <span class="sourceLineNo">687</span> seekOrSkipToNextColumn(cell);<a name="line.687"></a> |
| <span class="sourceLineNo">688</span> } else {<a name="line.688"></a> |
| <span class="sourceLineNo">689</span> this.heap.next();<a name="line.689"></a> |
| <span class="sourceLineNo">690</span> }<a name="line.690"></a> |
| <span class="sourceLineNo">691</span><a name="line.691"></a> |
| <span class="sourceLineNo">692</span> if (scannerContext.checkBatchLimit(LimitScope.BETWEEN_CELLS)) {<a name="line.692"></a> |
| <span class="sourceLineNo">693</span> break LOOP;<a name="line.693"></a> |
| <span class="sourceLineNo">694</span> }<a name="line.694"></a> |
| <span class="sourceLineNo">695</span> if (scannerContext.checkSizeLimit(LimitScope.BETWEEN_CELLS)) {<a name="line.695"></a> |
| <span class="sourceLineNo">696</span> break LOOP;<a name="line.696"></a> |
| <span class="sourceLineNo">697</span> }<a name="line.697"></a> |
| <span class="sourceLineNo">698</span> continue;<a name="line.698"></a> |
| <span class="sourceLineNo">699</span><a name="line.699"></a> |
| <span class="sourceLineNo">700</span> case DONE:<a name="line.700"></a> |
| <span class="sourceLineNo">701</span> // Optimization for Gets! If DONE, no more to get on this row, early exit!<a name="line.701"></a> |
| <span class="sourceLineNo">702</span> if (get) {<a name="line.702"></a> |
| <span class="sourceLineNo">703</span> // Then no more to this row... exit.<a name="line.703"></a> |
| <span class="sourceLineNo">704</span> close(false);// Do all cleanup except heap.close()<a name="line.704"></a> |
| <span class="sourceLineNo">705</span> // update metric<a name="line.705"></a> |
| <span class="sourceLineNo">706</span> return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();<a name="line.706"></a> |
| <span class="sourceLineNo">707</span> }<a name="line.707"></a> |
| <span class="sourceLineNo">708</span> matcher.clearCurrentRow();<a name="line.708"></a> |
| <span class="sourceLineNo">709</span> return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();<a name="line.709"></a> |
| <span class="sourceLineNo">710</span><a name="line.710"></a> |
| <span class="sourceLineNo">711</span> case DONE_SCAN:<a name="line.711"></a> |
| <span class="sourceLineNo">712</span> close(false);// Do all cleanup except heap.close()<a name="line.712"></a> |
| <span class="sourceLineNo">713</span> return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();<a name="line.713"></a> |
| <span class="sourceLineNo">714</span><a name="line.714"></a> |
| <span class="sourceLineNo">715</span> case SEEK_NEXT_ROW:<a name="line.715"></a> |
| <span class="sourceLineNo">716</span> // This is just a relatively simple end of scan fix, to short-cut end<a name="line.716"></a> |
| <span class="sourceLineNo">717</span> // us if there is an endKey in the scan.<a name="line.717"></a> |
| <span class="sourceLineNo">718</span> if (!matcher.moreRowsMayExistAfter(cell)) {<a name="line.718"></a> |
| <span class="sourceLineNo">719</span> close(false);// Do all cleanup except heap.close()<a name="line.719"></a> |
| <span class="sourceLineNo">720</span> return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();<a name="line.720"></a> |
| <span class="sourceLineNo">721</span> }<a name="line.721"></a> |
| <span class="sourceLineNo">722</span> matcher.clearCurrentRow();<a name="line.722"></a> |
| <span class="sourceLineNo">723</span> seekOrSkipToNextRow(cell);<a name="line.723"></a> |
| <span class="sourceLineNo">724</span> NextState stateAfterSeekNextRow = needToReturn(outResult);<a name="line.724"></a> |
| <span class="sourceLineNo">725</span> if (stateAfterSeekNextRow != null) {<a name="line.725"></a> |
| <span class="sourceLineNo">726</span> return scannerContext.setScannerState(stateAfterSeekNextRow).hasMoreValues();<a name="line.726"></a> |
| <span class="sourceLineNo">727</span> }<a name="line.727"></a> |
| <span class="sourceLineNo">728</span> break;<a name="line.728"></a> |
| <span class="sourceLineNo">729</span><a name="line.729"></a> |
| <span class="sourceLineNo">730</span> case SEEK_NEXT_COL:<a name="line.730"></a> |
| <span class="sourceLineNo">731</span> seekOrSkipToNextColumn(cell);<a name="line.731"></a> |
| <span class="sourceLineNo">732</span> NextState stateAfterSeekNextColumn = needToReturn(outResult);<a name="line.732"></a> |
| <span class="sourceLineNo">733</span> if (stateAfterSeekNextColumn != null) {<a name="line.733"></a> |
| <span class="sourceLineNo">734</span> return scannerContext.setScannerState(stateAfterSeekNextColumn).hasMoreValues();<a name="line.734"></a> |
| <span class="sourceLineNo">735</span> }<a name="line.735"></a> |
| <span class="sourceLineNo">736</span> break;<a name="line.736"></a> |
| <span class="sourceLineNo">737</span><a name="line.737"></a> |
| <span class="sourceLineNo">738</span> case SKIP:<a name="line.738"></a> |
| <span class="sourceLineNo">739</span> this.heap.next();<a name="line.739"></a> |
| <span class="sourceLineNo">740</span> break;<a name="line.740"></a> |
| <span class="sourceLineNo">741</span><a name="line.741"></a> |
| <span class="sourceLineNo">742</span> case SEEK_NEXT_USING_HINT:<a name="line.742"></a> |
| <span class="sourceLineNo">743</span> Cell nextKV = matcher.getNextKeyHint(cell);<a name="line.743"></a> |
| <span class="sourceLineNo">744</span> if (nextKV != null && comparator.compare(nextKV, cell) > 0) {<a name="line.744"></a> |
| <span class="sourceLineNo">745</span> seekAsDirection(nextKV);<a name="line.745"></a> |
| <span class="sourceLineNo">746</span> NextState stateAfterSeekByHint = needToReturn(outResult);<a name="line.746"></a> |
| <span class="sourceLineNo">747</span> if (stateAfterSeekByHint != null) {<a name="line.747"></a> |
| <span class="sourceLineNo">748</span> return scannerContext.setScannerState(stateAfterSeekByHint).hasMoreValues();<a name="line.748"></a> |
| <span class="sourceLineNo">749</span> }<a name="line.749"></a> |
| <span class="sourceLineNo">750</span> } else {<a name="line.750"></a> |
| <span class="sourceLineNo">751</span> heap.next();<a name="line.751"></a> |
| <span class="sourceLineNo">752</span> }<a name="line.752"></a> |
| <span class="sourceLineNo">753</span> break;<a name="line.753"></a> |
| <span class="sourceLineNo">754</span><a name="line.754"></a> |
| <span class="sourceLineNo">755</span> default:<a name="line.755"></a> |
| <span class="sourceLineNo">756</span> throw new RuntimeException("UNEXPECTED");<a name="line.756"></a> |
| <span class="sourceLineNo">757</span> }<a name="line.757"></a> |
| <span class="sourceLineNo">758</span> } while ((cell = this.heap.peek()) != null);<a name="line.758"></a> |
| <span class="sourceLineNo">759</span><a name="line.759"></a> |
| <span class="sourceLineNo">760</span> if (count > 0) {<a name="line.760"></a> |
| <span class="sourceLineNo">761</span> return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();<a name="line.761"></a> |
| <span class="sourceLineNo">762</span> }<a name="line.762"></a> |
| <span class="sourceLineNo">763</span><a name="line.763"></a> |
| <span class="sourceLineNo">764</span> // No more keys<a name="line.764"></a> |
| <span class="sourceLineNo">765</span> close(false);// Do all cleanup except heap.close()<a name="line.765"></a> |
| <span class="sourceLineNo">766</span> return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();<a name="line.766"></a> |
| <span class="sourceLineNo">767</span> } finally {<a name="line.767"></a> |
| <span class="sourceLineNo">768</span> // increment only if we have some result<a name="line.768"></a> |
| <span class="sourceLineNo">769</span> if (count > 0 && matcher.isUserScan()) {<a name="line.769"></a> |
| <span class="sourceLineNo">770</span> // if true increment memstore metrics, if not the mixed one<a name="line.770"></a> |
| <span class="sourceLineNo">771</span> updateMetricsStore(onlyFromMemstore);<a name="line.771"></a> |
| <span class="sourceLineNo">772</span> }<a name="line.772"></a> |
| <span class="sourceLineNo">773</span> }<a name="line.773"></a> |
| <span class="sourceLineNo">774</span> }<a name="line.774"></a> |
| <span class="sourceLineNo">775</span><a name="line.775"></a> |
| <span class="sourceLineNo">776</span> private void updateMetricsStore(boolean memstoreRead) {<a name="line.776"></a> |
| <span class="sourceLineNo">777</span> if (store != null) {<a name="line.777"></a> |
| <span class="sourceLineNo">778</span> store.updateMetricsStore(memstoreRead);<a name="line.778"></a> |
| <span class="sourceLineNo">779</span> } else {<a name="line.779"></a> |
| <span class="sourceLineNo">780</span> // for testing.<a name="line.780"></a> |
| <span class="sourceLineNo">781</span> if (memstoreRead) {<a name="line.781"></a> |
| <span class="sourceLineNo">782</span> memstoreOnlyReads++;<a name="line.782"></a> |
| <span class="sourceLineNo">783</span> } else {<a name="line.783"></a> |
| <span class="sourceLineNo">784</span> mixedReads++;<a name="line.784"></a> |
| <span class="sourceLineNo">785</span> }<a name="line.785"></a> |
| <span class="sourceLineNo">786</span> }<a name="line.786"></a> |
| <span class="sourceLineNo">787</span> }<a name="line.787"></a> |
| <span class="sourceLineNo">788</span><a name="line.788"></a> |
| <span class="sourceLineNo">789</span> /**<a name="line.789"></a> |
| <span class="sourceLineNo">790</span> * If the top cell won't be flushed into disk, the new top cell may be<a name="line.790"></a> |
| <span class="sourceLineNo">791</span> * changed after #reopenAfterFlush. Because the older top cell only exist<a name="line.791"></a> |
| <span class="sourceLineNo">792</span> * in the memstore scanner but the memstore scanner is replaced by hfile<a name="line.792"></a> |
| <span class="sourceLineNo">793</span> * scanner after #reopenAfterFlush. If the row of top cell is changed,<a name="line.793"></a> |
| <span class="sourceLineNo">794</span> * we should return the current cells. Otherwise, we may return<a name="line.794"></a> |
| <span class="sourceLineNo">795</span> * the cells across different rows.<a name="line.795"></a> |
| <span class="sourceLineNo">796</span> * @param outResult the cells which are visible for user scan<a name="line.796"></a> |
| <span class="sourceLineNo">797</span> * @return null is the top cell doesn't change. Otherwise, the NextState<a name="line.797"></a> |
| <span class="sourceLineNo">798</span> * to return<a name="line.798"></a> |
| <span class="sourceLineNo">799</span> */<a name="line.799"></a> |
| <span class="sourceLineNo">800</span> private NextState needToReturn(List<Cell> outResult) {<a name="line.800"></a> |
| <span class="sourceLineNo">801</span> if (!outResult.isEmpty() && topChanged) {<a name="line.801"></a> |
| <span class="sourceLineNo">802</span> return heap.peek() == null ? NextState.NO_MORE_VALUES : NextState.MORE_VALUES;<a name="line.802"></a> |
| <span class="sourceLineNo">803</span> }<a name="line.803"></a> |
| <span class="sourceLineNo">804</span> return null;<a name="line.804"></a> |
| <span class="sourceLineNo">805</span> }<a name="line.805"></a> |
| <span class="sourceLineNo">806</span><a name="line.806"></a> |
| <span class="sourceLineNo">807</span> private void seekOrSkipToNextRow(Cell cell) throws IOException {<a name="line.807"></a> |
| <span class="sourceLineNo">808</span> // If it is a Get Scan, then we know that we are done with this row; there are no more<a name="line.808"></a> |
| <span class="sourceLineNo">809</span> // rows beyond the current one: don't try to optimize.<a name="line.809"></a> |
| <span class="sourceLineNo">810</span> if (!get) {<a name="line.810"></a> |
| <span class="sourceLineNo">811</span> if (trySkipToNextRow(cell)) {<a name="line.811"></a> |
| <span class="sourceLineNo">812</span> return;<a name="line.812"></a> |
| <span class="sourceLineNo">813</span> }<a name="line.813"></a> |
| <span class="sourceLineNo">814</span> }<a name="line.814"></a> |
| <span class="sourceLineNo">815</span> seekToNextRow(cell);<a name="line.815"></a> |
| <span class="sourceLineNo">816</span> }<a name="line.816"></a> |
| <span class="sourceLineNo">817</span><a name="line.817"></a> |
| <span class="sourceLineNo">818</span> private void seekOrSkipToNextColumn(Cell cell) throws IOException {<a name="line.818"></a> |
| <span class="sourceLineNo">819</span> if (!trySkipToNextColumn(cell)) {<a name="line.819"></a> |
| <span class="sourceLineNo">820</span> seekAsDirection(matcher.getKeyForNextColumn(cell));<a name="line.820"></a> |
| <span class="sourceLineNo">821</span> }<a name="line.821"></a> |
| <span class="sourceLineNo">822</span> }<a name="line.822"></a> |
| <span class="sourceLineNo">823</span><a name="line.823"></a> |
| <span class="sourceLineNo">824</span> /**<a name="line.824"></a> |
| <span class="sourceLineNo">825</span> * See if we should actually SEEK or rather just SKIP to the next Cell (see HBASE-13109).<a name="line.825"></a> |
| <span class="sourceLineNo">826</span> * ScanQueryMatcher may issue SEEK hints, such as seek to next column, next row,<a name="line.826"></a> |
| <span class="sourceLineNo">827</span> * or seek to an arbitrary seek key. This method decides whether a seek is the most efficient<a name="line.827"></a> |
| <span class="sourceLineNo">828</span> * _actual_ way to get us to the requested cell (SEEKs are more expensive than SKIP, SKIP,<a name="line.828"></a> |
| <span class="sourceLineNo">829</span> * SKIP inside the current, loaded block).<a name="line.829"></a> |
| <span class="sourceLineNo">830</span> * It does this by looking at the next indexed key of the current HFile. This key<a name="line.830"></a> |
| <span class="sourceLineNo">831</span> * is then compared with the _SEEK_ key, where a SEEK key is an artificial 'last possible key<a name="line.831"></a> |
| <span class="sourceLineNo">832</span> * on the row' (only in here, we avoid actually creating a SEEK key; in the compare we work with<a name="line.832"></a> |
| <span class="sourceLineNo">833</span> * the current Cell but compare as though it were a seek key; see down in<a name="line.833"></a> |
| <span class="sourceLineNo">834</span> * matcher.compareKeyForNextRow, etc). If the compare gets us onto the<a name="line.834"></a> |
| <span class="sourceLineNo">835</span> * next block we *_SEEK, otherwise we just SKIP to the next requested cell.<a name="line.835"></a> |
| <span class="sourceLineNo">836</span> *<a name="line.836"></a> |
| <span class="sourceLineNo">837</span> * <p>Other notes:<a name="line.837"></a> |
| <span class="sourceLineNo">838</span> * <ul><a name="line.838"></a> |
| <span class="sourceLineNo">839</span> * <li>Rows can straddle block boundaries</li><a name="line.839"></a> |
| <span class="sourceLineNo">840</span> * <li>Versions of columns can straddle block boundaries (i.e. column C1 at T1 might be in a<a name="line.840"></a> |
| <span class="sourceLineNo">841</span> * different block than column C1 at T2)</li><a name="line.841"></a> |
| <span class="sourceLineNo">842</span> * <li>We want to SKIP if the chance is high that we'll find the desired Cell after a<a name="line.842"></a> |
| <span class="sourceLineNo">843</span> * few SKIPs...</li><a name="line.843"></a> |
| <span class="sourceLineNo">844</span> * <li>We want to SEEK when the chance is high that we'll be able to seek<a name="line.844"></a> |
| <span class="sourceLineNo">845</span> * past many Cells, especially if we know we need to go to the next block.</li><a name="line.845"></a> |
| <span class="sourceLineNo">846</span> * </ul><a name="line.846"></a> |
| <span class="sourceLineNo">847</span> * <p>A good proxy (best effort) to determine whether SKIP is better than SEEK is whether<a name="line.847"></a> |
| <span class="sourceLineNo">848</span> * we'll likely end up seeking to the next block (or past the next block) to get our next column.<a name="line.848"></a> |
| <span class="sourceLineNo">849</span> * Example:<a name="line.849"></a> |
| <span class="sourceLineNo">850</span> * <pre><a name="line.850"></a> |
| <span class="sourceLineNo">851</span> * | BLOCK 1 | BLOCK 2 |<a name="line.851"></a> |
| <span class="sourceLineNo">852</span> * | r1/c1, r1/c2, r1/c3 | r1/c4, r1/c5, r2/c1 |<a name="line.852"></a> |
| <span class="sourceLineNo">853</span> * ^ ^<a name="line.853"></a> |
| <span class="sourceLineNo">854</span> * | |<a name="line.854"></a> |
| <span class="sourceLineNo">855</span> * Next Index Key SEEK_NEXT_ROW (before r2/c1)<a name="line.855"></a> |
| <span class="sourceLineNo">856</span> *<a name="line.856"></a> |
| <span class="sourceLineNo">857</span> *<a name="line.857"></a> |
| <span class="sourceLineNo">858</span> * | BLOCK 1 | BLOCK 2 |<a name="line.858"></a> |
| <span class="sourceLineNo">859</span> * | r1/c1/t5, r1/c1/t4, r1/c1/t3 | r1/c1/t2, r1/c1/T1, r1/c2/T3 |<a name="line.859"></a> |
| <span class="sourceLineNo">860</span> * ^ ^<a name="line.860"></a> |
| <span class="sourceLineNo">861</span> * | |<a name="line.861"></a> |
| <span class="sourceLineNo">862</span> * Next Index Key SEEK_NEXT_COL<a name="line.862"></a> |
| <span class="sourceLineNo">863</span> * </pre><a name="line.863"></a> |
| <span class="sourceLineNo">864</span> * Now imagine we want columns c1 and c3 (see first diagram above), the 'Next Index Key' of r1/c4<a name="line.864"></a> |
| <span class="sourceLineNo">865</span> * is > r1/c3 so we should seek to get to the c1 on the next row, r2. In second case, say we only<a name="line.865"></a> |
| <span class="sourceLineNo">866</span> * want one version of c1, after we have it, a SEEK_COL will be issued to get to c2. Looking at<a name="line.866"></a> |
| <span class="sourceLineNo">867</span> * the 'Next Index Key', it would land us in the next block, so we should SEEK. In other scenarios<a name="line.867"></a> |
| <span class="sourceLineNo">868</span> * where the SEEK will not land us in the next block, it is very likely better to issues a series<a name="line.868"></a> |
| <span class="sourceLineNo">869</span> * of SKIPs.<a name="line.869"></a> |
| <span class="sourceLineNo">870</span> * @param cell current cell<a name="line.870"></a> |
| <span class="sourceLineNo">871</span> * @return true means skip to next row, false means not<a name="line.871"></a> |
| <span class="sourceLineNo">872</span> */<a name="line.872"></a> |
| <span class="sourceLineNo">873</span> @VisibleForTesting<a name="line.873"></a> |
| <span class="sourceLineNo">874</span> protected boolean trySkipToNextRow(Cell cell) throws IOException {<a name="line.874"></a> |
| <span class="sourceLineNo">875</span> Cell nextCell = null;<a name="line.875"></a> |
| <span class="sourceLineNo">876</span> // used to guard against a changed next indexed key by doing a identity comparison<a name="line.876"></a> |
| <span class="sourceLineNo">877</span> // when the identity changes we need to compare the bytes again<a name="line.877"></a> |
| <span class="sourceLineNo">878</span> Cell previousIndexedKey = null;<a name="line.878"></a> |
| <span class="sourceLineNo">879</span> do {<a name="line.879"></a> |
| <span class="sourceLineNo">880</span> Cell nextIndexedKey = getNextIndexedKey();<a name="line.880"></a> |
| <span class="sourceLineNo">881</span> if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY &&<a name="line.881"></a> |
| <span class="sourceLineNo">882</span> (nextIndexedKey == previousIndexedKey ||<a name="line.882"></a> |
| <span class="sourceLineNo">883</span> matcher.compareKeyForNextRow(nextIndexedKey, cell) >= 0)) {<a name="line.883"></a> |
| <span class="sourceLineNo">884</span> this.heap.next();<a name="line.884"></a> |
| <span class="sourceLineNo">885</span> ++kvsScanned;<a name="line.885"></a> |
| <span class="sourceLineNo">886</span> previousIndexedKey = nextIndexedKey;<a name="line.886"></a> |
| <span class="sourceLineNo">887</span> } else {<a name="line.887"></a> |
| <span class="sourceLineNo">888</span> return false;<a name="line.888"></a> |
| <span class="sourceLineNo">889</span> }<a name="line.889"></a> |
| <span class="sourceLineNo">890</span> } while ((nextCell = this.heap.peek()) != null && CellUtil.matchingRows(cell, nextCell));<a name="line.890"></a> |
| <span class="sourceLineNo">891</span> return true;<a name="line.891"></a> |
| <span class="sourceLineNo">892</span> }<a name="line.892"></a> |
| <span class="sourceLineNo">893</span><a name="line.893"></a> |
| <span class="sourceLineNo">894</span> /**<a name="line.894"></a> |
| <span class="sourceLineNo">895</span> * See {@link org.apache.hadoop.hbase.regionserver.StoreScanner#trySkipToNextRow(Cell)}<a name="line.895"></a> |
| <span class="sourceLineNo">896</span> * @param cell current cell<a name="line.896"></a> |
| <span class="sourceLineNo">897</span> * @return true means skip to next column, false means not<a name="line.897"></a> |
| <span class="sourceLineNo">898</span> */<a name="line.898"></a> |
| <span class="sourceLineNo">899</span> @VisibleForTesting<a name="line.899"></a> |
| <span class="sourceLineNo">900</span> protected boolean trySkipToNextColumn(Cell cell) throws IOException {<a name="line.900"></a> |
| <span class="sourceLineNo">901</span> Cell nextCell = null;<a name="line.901"></a> |
| <span class="sourceLineNo">902</span> // used to guard against a changed next indexed key by doing a identity comparison<a name="line.902"></a> |
| <span class="sourceLineNo">903</span> // when the identity changes we need to compare the bytes again<a name="line.903"></a> |
| <span class="sourceLineNo">904</span> Cell previousIndexedKey = null;<a name="line.904"></a> |
| <span class="sourceLineNo">905</span> do {<a name="line.905"></a> |
| <span class="sourceLineNo">906</span> Cell nextIndexedKey = getNextIndexedKey();<a name="line.906"></a> |
| <span class="sourceLineNo">907</span> if (nextIndexedKey != null && nextIndexedKey != KeyValueScanner.NO_NEXT_INDEXED_KEY &&<a name="line.907"></a> |
| <span class="sourceLineNo">908</span> (nextIndexedKey == previousIndexedKey ||<a name="line.908"></a> |
| <span class="sourceLineNo">909</span> matcher.compareKeyForNextColumn(nextIndexedKey, cell) >= 0)) {<a name="line.909"></a> |
| <span class="sourceLineNo">910</span> this.heap.next();<a name="line.910"></a> |
| <span class="sourceLineNo">911</span> ++kvsScanned;<a name="line.911"></a> |
| <span class="sourceLineNo">912</span> previousIndexedKey = nextIndexedKey;<a name="line.912"></a> |
| <span class="sourceLineNo">913</span> } else {<a name="line.913"></a> |
| <span class="sourceLineNo">914</span> return false;<a name="line.914"></a> |
| <span class="sourceLineNo">915</span> }<a name="line.915"></a> |
| <span class="sourceLineNo">916</span> } while ((nextCell = this.heap.peek()) != null && CellUtil.matchingRowColumn(cell, nextCell));<a name="line.916"></a> |
| <span class="sourceLineNo">917</span> // We need this check because it may happen that the new scanner that we get<a name="line.917"></a> |
| <span class="sourceLineNo">918</span> // during heap.next() is requiring reseek due of fake KV previously generated for<a name="line.918"></a> |
| <span class="sourceLineNo">919</span> // ROWCOL bloom filter optimization. See HBASE-19863 for more details<a name="line.919"></a> |
| <span class="sourceLineNo">920</span> if (useRowColBloom && nextCell != null && matcher.compareKeyForNextColumn(nextCell, cell) < 0) {<a name="line.920"></a> |
| <span class="sourceLineNo">921</span> return false;<a name="line.921"></a> |
| <span class="sourceLineNo">922</span> }<a name="line.922"></a> |
| <span class="sourceLineNo">923</span> return true;<a name="line.923"></a> |
| <span class="sourceLineNo">924</span> }<a name="line.924"></a> |
| <span class="sourceLineNo">925</span><a name="line.925"></a> |
| <span class="sourceLineNo">926</span> @Override<a name="line.926"></a> |
| <span class="sourceLineNo">927</span> public long getReadPoint() {<a name="line.927"></a> |
| <span class="sourceLineNo">928</span> return this.readPt;<a name="line.928"></a> |
| <span class="sourceLineNo">929</span> }<a name="line.929"></a> |
| <span class="sourceLineNo">930</span><a name="line.930"></a> |
| <span class="sourceLineNo">931</span> private static void clearAndClose(List<KeyValueScanner> scanners) {<a name="line.931"></a> |
| <span class="sourceLineNo">932</span> if (scanners == null) {<a name="line.932"></a> |
| <span class="sourceLineNo">933</span> return;<a name="line.933"></a> |
| <span class="sourceLineNo">934</span> }<a name="line.934"></a> |
| <span class="sourceLineNo">935</span> for (KeyValueScanner s : scanners) {<a name="line.935"></a> |
| <span class="sourceLineNo">936</span> s.close();<a name="line.936"></a> |
| <span class="sourceLineNo">937</span> }<a name="line.937"></a> |
| <span class="sourceLineNo">938</span> scanners.clear();<a name="line.938"></a> |
| <span class="sourceLineNo">939</span> }<a name="line.939"></a> |
| <span class="sourceLineNo">940</span><a name="line.940"></a> |
| <span class="sourceLineNo">941</span> // Implementation of ChangedReadersObserver<a name="line.941"></a> |
| <span class="sourceLineNo">942</span> @Override<a name="line.942"></a> |
| <span class="sourceLineNo">943</span> public void updateReaders(List<HStoreFile> sfs, List<KeyValueScanner> memStoreScanners)<a name="line.943"></a> |
| <span class="sourceLineNo">944</span> throws IOException {<a name="line.944"></a> |
| <span class="sourceLineNo">945</span> if (CollectionUtils.isEmpty(sfs) && CollectionUtils.isEmpty(memStoreScanners)) {<a name="line.945"></a> |
| <span class="sourceLineNo">946</span> return;<a name="line.946"></a> |
| <span class="sourceLineNo">947</span> }<a name="line.947"></a> |
| <span class="sourceLineNo">948</span> boolean updateReaders = false;<a name="line.948"></a> |
| <span class="sourceLineNo">949</span> flushLock.lock();<a name="line.949"></a> |
| <span class="sourceLineNo">950</span> try {<a name="line.950"></a> |
| <span class="sourceLineNo">951</span> if (!closeLock.tryLock()) {<a name="line.951"></a> |
| <span class="sourceLineNo">952</span> // The reason for doing this is that when the current store scanner does not retrieve<a name="line.952"></a> |
| <span class="sourceLineNo">953</span> // any new cells, then the scanner is considered to be done. The heap of this scanner<a name="line.953"></a> |
| <span class="sourceLineNo">954</span> // is not closed till the shipped() call is completed. Hence in that case if at all<a name="line.954"></a> |
| <span class="sourceLineNo">955</span> // the partial close (close (false)) has been called before updateReaders(), there is no<a name="line.955"></a> |
| <span class="sourceLineNo">956</span> // need for the updateReaders() to happen.<a name="line.956"></a> |
| <span class="sourceLineNo">957</span> LOG.debug("StoreScanner already has the close lock. There is no need to updateReaders");<a name="line.957"></a> |
| <span class="sourceLineNo">958</span> // no lock acquired.<a name="line.958"></a> |
| <span class="sourceLineNo">959</span> clearAndClose(memStoreScanners);<a name="line.959"></a> |
| <span class="sourceLineNo">960</span> return;<a name="line.960"></a> |
| <span class="sourceLineNo">961</span> }<a name="line.961"></a> |
| <span class="sourceLineNo">962</span> // lock acquired<a name="line.962"></a> |
| <span class="sourceLineNo">963</span> updateReaders = true;<a name="line.963"></a> |
| <span class="sourceLineNo">964</span> if (this.closing) {<a name="line.964"></a> |
| <span class="sourceLineNo">965</span> LOG.debug("StoreScanner already closing. There is no need to updateReaders");<a name="line.965"></a> |
| <span class="sourceLineNo">966</span> clearAndClose(memStoreScanners);<a name="line.966"></a> |
| <span class="sourceLineNo">967</span> return;<a name="line.967"></a> |
| <span class="sourceLineNo">968</span> }<a name="line.968"></a> |
| <span class="sourceLineNo">969</span> flushed = true;<a name="line.969"></a> |
| <span class="sourceLineNo">970</span> final boolean isCompaction = false;<a name="line.970"></a> |
| <span class="sourceLineNo">971</span> boolean usePread = get || scanUsePread;<a name="line.971"></a> |
| <span class="sourceLineNo">972</span> // SEE HBASE-19468 where the flushed files are getting compacted even before a scanner<a name="line.972"></a> |
| <span class="sourceLineNo">973</span> // calls next(). So its better we create scanners here rather than next() call. Ensure<a name="line.973"></a> |
| <span class="sourceLineNo">974</span> // these scanners are properly closed() whether or not the scan is completed successfully<a name="line.974"></a> |
| <span class="sourceLineNo">975</span> // Eagerly creating scanners so that we have the ref counting ticking on the newly created<a name="line.975"></a> |
| <span class="sourceLineNo">976</span> // store files. In case of stream scanners this eager creation does not induce performance<a name="line.976"></a> |
| <span class="sourceLineNo">977</span> // penalty because in scans (that uses stream scanners) the next() call is bound to happen.<a name="line.977"></a> |
| <span class="sourceLineNo">978</span> List<KeyValueScanner> scanners = store.getScanners(sfs, cacheBlocks, get, usePread,<a name="line.978"></a> |
| <span class="sourceLineNo">979</span> isCompaction, matcher, scan.getStartRow(), scan.getStopRow(), this.readPt, false);<a name="line.979"></a> |
| <span class="sourceLineNo">980</span> flushedstoreFileScanners.addAll(scanners);<a name="line.980"></a> |
| <span class="sourceLineNo">981</span> if (!CollectionUtils.isEmpty(memStoreScanners)) {<a name="line.981"></a> |
| <span class="sourceLineNo">982</span> clearAndClose(memStoreScannersAfterFlush);<a name="line.982"></a> |
| <span class="sourceLineNo">983</span> memStoreScannersAfterFlush.addAll(memStoreScanners);<a name="line.983"></a> |
| <span class="sourceLineNo">984</span> }<a name="line.984"></a> |
| <span class="sourceLineNo">985</span> } finally {<a name="line.985"></a> |
| <span class="sourceLineNo">986</span> flushLock.unlock();<a name="line.986"></a> |
| <span class="sourceLineNo">987</span> if (updateReaders) {<a name="line.987"></a> |
| <span class="sourceLineNo">988</span> closeLock.unlock();<a name="line.988"></a> |
| <span class="sourceLineNo">989</span> }<a name="line.989"></a> |
| <span class="sourceLineNo">990</span> }<a name="line.990"></a> |
| <span class="sourceLineNo">991</span> // Let the next() call handle re-creating and seeking<a name="line.991"></a> |
| <span class="sourceLineNo">992</span> }<a name="line.992"></a> |
| <span class="sourceLineNo">993</span><a name="line.993"></a> |
| <span class="sourceLineNo">994</span> /**<a name="line.994"></a> |
| <span class="sourceLineNo">995</span> * @return if top of heap has changed (and KeyValueHeap has to try the next KV)<a name="line.995"></a> |
| <span class="sourceLineNo">996</span> */<a name="line.996"></a> |
| <span class="sourceLineNo">997</span> protected final boolean reopenAfterFlush() throws IOException {<a name="line.997"></a> |
| <span class="sourceLineNo">998</span> // here we can make sure that we have a Store instance so no null check on store.<a name="line.998"></a> |
| <span class="sourceLineNo">999</span> Cell lastTop = heap.peek();<a name="line.999"></a> |
| <span class="sourceLineNo">1000</span> // When we have the scan object, should we not pass it to getScanners() to get a limited set of<a name="line.1000"></a> |
| <span class="sourceLineNo">1001</span> // scanners? We did so in the constructor and we could have done it now by storing the scan<a name="line.1001"></a> |
| <span class="sourceLineNo">1002</span> // object from the constructor<a name="line.1002"></a> |
| <span class="sourceLineNo">1003</span> List<KeyValueScanner> scanners;<a name="line.1003"></a> |
| <span class="sourceLineNo">1004</span> flushLock.lock();<a name="line.1004"></a> |
| <span class="sourceLineNo">1005</span> try {<a name="line.1005"></a> |
| <span class="sourceLineNo">1006</span> List<KeyValueScanner> allScanners =<a name="line.1006"></a> |
| <span class="sourceLineNo">1007</span> new ArrayList<>(flushedstoreFileScanners.size() + memStoreScannersAfterFlush.size());<a name="line.1007"></a> |
| <span class="sourceLineNo">1008</span> allScanners.addAll(flushedstoreFileScanners);<a name="line.1008"></a> |
| <span class="sourceLineNo">1009</span> allScanners.addAll(memStoreScannersAfterFlush);<a name="line.1009"></a> |
| <span class="sourceLineNo">1010</span> scanners = selectScannersFrom(store, allScanners);<a name="line.1010"></a> |
| <span class="sourceLineNo">1011</span> // Clear the current set of flushed store files scanners so that they don't get added again<a name="line.1011"></a> |
| <span class="sourceLineNo">1012</span> flushedstoreFileScanners.clear();<a name="line.1012"></a> |
| <span class="sourceLineNo">1013</span> memStoreScannersAfterFlush.clear();<a name="line.1013"></a> |
| <span class="sourceLineNo">1014</span> } finally {<a name="line.1014"></a> |
| <span class="sourceLineNo">1015</span> flushLock.unlock();<a name="line.1015"></a> |
| <span class="sourceLineNo">1016</span> }<a name="line.1016"></a> |
| <span class="sourceLineNo">1017</span><a name="line.1017"></a> |
| <span class="sourceLineNo">1018</span> // Seek the new scanners to the last key<a name="line.1018"></a> |
| <span class="sourceLineNo">1019</span> seekScanners(scanners, lastTop, false, parallelSeekEnabled);<a name="line.1019"></a> |
| <span class="sourceLineNo">1020</span> // remove the older memstore scanner<a name="line.1020"></a> |
| <span class="sourceLineNo">1021</span> for (int i = currentScanners.size() - 1; i >=0; i--) {<a name="line.1021"></a> |
| <span class="sourceLineNo">1022</span> if (!currentScanners.get(i).isFileScanner()) {<a name="line.1022"></a> |
| <span class="sourceLineNo">1023</span> scannersForDelayedClose.add(currentScanners.remove(i));<a name="line.1023"></a> |
| <span class="sourceLineNo">1024</span> } else {<a name="line.1024"></a> |
| <span class="sourceLineNo">1025</span> // we add the memstore scanner to the end of currentScanners<a name="line.1025"></a> |
| <span class="sourceLineNo">1026</span> break;<a name="line.1026"></a> |
| <span class="sourceLineNo">1027</span> }<a name="line.1027"></a> |
| <span class="sourceLineNo">1028</span> }<a name="line.1028"></a> |
| <span class="sourceLineNo">1029</span> // add the newly created scanners on the flushed files and the current active memstore scanner<a name="line.1029"></a> |
| <span class="sourceLineNo">1030</span> addCurrentScanners(scanners);<a name="line.1030"></a> |
| <span class="sourceLineNo">1031</span> // Combine all seeked scanners with a heap<a name="line.1031"></a> |
| <span class="sourceLineNo">1032</span> resetKVHeap(this.currentScanners, store.getComparator());<a name="line.1032"></a> |
| <span class="sourceLineNo">1033</span> resetQueryMatcher(lastTop);<a name="line.1033"></a> |
| <span class="sourceLineNo">1034</span> if (heap.peek() == null || store.getComparator().compareRows(lastTop, this.heap.peek()) != 0) {<a name="line.1034"></a> |
| <span class="sourceLineNo">1035</span> LOG.info("Storescanner.peek() is changed where before = " + lastTop.toString() +<a name="line.1035"></a> |
| <span class="sourceLineNo">1036</span> ",and after = " + heap.peek());<a name="line.1036"></a> |
| <span class="sourceLineNo">1037</span> topChanged = true;<a name="line.1037"></a> |
| <span class="sourceLineNo">1038</span> } else {<a name="line.1038"></a> |
| <span class="sourceLineNo">1039</span> topChanged = false;<a name="line.1039"></a> |
| <span class="sourceLineNo">1040</span> }<a name="line.1040"></a> |
| <span class="sourceLineNo">1041</span> return topChanged;<a name="line.1041"></a> |
| <span class="sourceLineNo">1042</span> }<a name="line.1042"></a> |
| <span class="sourceLineNo">1043</span><a name="line.1043"></a> |
| <span class="sourceLineNo">1044</span> private void resetQueryMatcher(Cell lastTopKey) {<a name="line.1044"></a> |
| <span class="sourceLineNo">1045</span> // Reset the state of the Query Matcher and set to top row.<a name="line.1045"></a> |
| <span class="sourceLineNo">1046</span> // Only reset and call setRow if the row changes; avoids confusing the<a name="line.1046"></a> |
| <span class="sourceLineNo">1047</span> // query matcher if scanning intra-row.<a name="line.1047"></a> |
| <span class="sourceLineNo">1048</span> Cell cell = heap.peek();<a name="line.1048"></a> |
| <span class="sourceLineNo">1049</span> if (cell == null) {<a name="line.1049"></a> |
| <span class="sourceLineNo">1050</span> cell = lastTopKey;<a name="line.1050"></a> |
| <span class="sourceLineNo">1051</span> }<a name="line.1051"></a> |
| <span class="sourceLineNo">1052</span> if ((matcher.currentRow() == null) || !CellUtil.matchingRows(cell, matcher.currentRow())) {<a name="line.1052"></a> |
| <span class="sourceLineNo">1053</span> this.countPerRow = 0;<a name="line.1053"></a> |
| <span class="sourceLineNo">1054</span> // The setToNewRow will call reset internally<a name="line.1054"></a> |
| <span class="sourceLineNo">1055</span> matcher.setToNewRow(cell);<a name="line.1055"></a> |
| <span class="sourceLineNo">1056</span> }<a name="line.1056"></a> |
| <span class="sourceLineNo">1057</span> }<a name="line.1057"></a> |
| <span class="sourceLineNo">1058</span><a name="line.1058"></a> |
| <span class="sourceLineNo">1059</span> /**<a name="line.1059"></a> |
| <span class="sourceLineNo">1060</span> * Check whether scan as expected order<a name="line.1060"></a> |
| <span class="sourceLineNo">1061</span> * @param prevKV<a name="line.1061"></a> |
| <span class="sourceLineNo">1062</span> * @param kv<a name="line.1062"></a> |
| <span class="sourceLineNo">1063</span> * @param comparator<a name="line.1063"></a> |
| <span class="sourceLineNo">1064</span> * @throws IOException<a name="line.1064"></a> |
| <span class="sourceLineNo">1065</span> */<a name="line.1065"></a> |
| <span class="sourceLineNo">1066</span> protected void checkScanOrder(Cell prevKV, Cell kv,<a name="line.1066"></a> |
| <span class="sourceLineNo">1067</span> CellComparator comparator) throws IOException {<a name="line.1067"></a> |
| <span class="sourceLineNo">1068</span> // Check that the heap gives us KVs in an increasing order.<a name="line.1068"></a> |
| <span class="sourceLineNo">1069</span> assert prevKV == null || comparator == null || comparator.compare(prevKV, kv) <= 0 : "Key "<a name="line.1069"></a> |
| <span class="sourceLineNo">1070</span> + prevKV + " followed by a smaller key " + kv + " in cf " + store;<a name="line.1070"></a> |
| <span class="sourceLineNo">1071</span> }<a name="line.1071"></a> |
| <span class="sourceLineNo">1072</span><a name="line.1072"></a> |
| <span class="sourceLineNo">1073</span> protected boolean seekToNextRow(Cell c) throws IOException {<a name="line.1073"></a> |
| <span class="sourceLineNo">1074</span> return reseek(PrivateCellUtil.createLastOnRow(c));<a name="line.1074"></a> |
| <span class="sourceLineNo">1075</span> }<a name="line.1075"></a> |
| <span class="sourceLineNo">1076</span><a name="line.1076"></a> |
| <span class="sourceLineNo">1077</span> /**<a name="line.1077"></a> |
| <span class="sourceLineNo">1078</span> * Do a reseek in a normal StoreScanner(scan forward)<a name="line.1078"></a> |
| <span class="sourceLineNo">1079</span> * @param kv<a name="line.1079"></a> |
| <span class="sourceLineNo">1080</span> * @return true if scanner has values left, false if end of scanner<a name="line.1080"></a> |
| <span class="sourceLineNo">1081</span> * @throws IOException<a name="line.1081"></a> |
| <span class="sourceLineNo">1082</span> */<a name="line.1082"></a> |
| <span class="sourceLineNo">1083</span> protected boolean seekAsDirection(Cell kv)<a name="line.1083"></a> |
| <span class="sourceLineNo">1084</span> throws IOException {<a name="line.1084"></a> |
| <span class="sourceLineNo">1085</span> return reseek(kv);<a name="line.1085"></a> |
| <span class="sourceLineNo">1086</span> }<a name="line.1086"></a> |
| <span class="sourceLineNo">1087</span><a name="line.1087"></a> |
| <span class="sourceLineNo">1088</span> @Override<a name="line.1088"></a> |
| <span class="sourceLineNo">1089</span> public boolean reseek(Cell kv) throws IOException {<a name="line.1089"></a> |
| <span class="sourceLineNo">1090</span> if (checkFlushed()) {<a name="line.1090"></a> |
| <span class="sourceLineNo">1091</span> reopenAfterFlush();<a name="line.1091"></a> |
| <span class="sourceLineNo">1092</span> }<a name="line.1092"></a> |
| <span class="sourceLineNo">1093</span> if (explicitColumnQuery && lazySeekEnabledGlobally) {<a name="line.1093"></a> |
| <span class="sourceLineNo">1094</span> return heap.requestSeek(kv, true, useRowColBloom);<a name="line.1094"></a> |
| <span class="sourceLineNo">1095</span> }<a name="line.1095"></a> |
| <span class="sourceLineNo">1096</span> return heap.reseek(kv);<a name="line.1096"></a> |
| <span class="sourceLineNo">1097</span> }<a name="line.1097"></a> |
| <span class="sourceLineNo">1098</span><a name="line.1098"></a> |
| <span class="sourceLineNo">1099</span> @VisibleForTesting<a name="line.1099"></a> |
| <span class="sourceLineNo">1100</span> void trySwitchToStreamRead() {<a name="line.1100"></a> |
| <span class="sourceLineNo">1101</span> if (readType != Scan.ReadType.DEFAULT || !scanUsePread || closing ||<a name="line.1101"></a> |
| <span class="sourceLineNo">1102</span> heap.peek() == null || bytesRead < preadMaxBytes) {<a name="line.1102"></a> |
| <span class="sourceLineNo">1103</span> return;<a name="line.1103"></a> |
| <span class="sourceLineNo">1104</span> }<a name="line.1104"></a> |
| <span class="sourceLineNo">1105</span> LOG.debug("Switch to stream read (scanned={} bytes) of {}", bytesRead,<a name="line.1105"></a> |
| <span class="sourceLineNo">1106</span> this.store.getColumnFamilyName());<a name="line.1106"></a> |
| <span class="sourceLineNo">1107</span> scanUsePread = false;<a name="line.1107"></a> |
| <span class="sourceLineNo">1108</span> Cell lastTop = heap.peek();<a name="line.1108"></a> |
| <span class="sourceLineNo">1109</span> List<KeyValueScanner> memstoreScanners = new ArrayList<>();<a name="line.1109"></a> |
| <span class="sourceLineNo">1110</span> List<KeyValueScanner> scannersToClose = new ArrayList<>();<a name="line.1110"></a> |
| <span class="sourceLineNo">1111</span> for (KeyValueScanner kvs : currentScanners) {<a name="line.1111"></a> |
| <span class="sourceLineNo">1112</span> if (!kvs.isFileScanner()) {<a name="line.1112"></a> |
| <span class="sourceLineNo">1113</span> // collect memstorescanners here<a name="line.1113"></a> |
| <span class="sourceLineNo">1114</span> memstoreScanners.add(kvs);<a name="line.1114"></a> |
| <span class="sourceLineNo">1115</span> } else {<a name="line.1115"></a> |
| <span class="sourceLineNo">1116</span> scannersToClose.add(kvs);<a name="line.1116"></a> |
| <span class="sourceLineNo">1117</span> }<a name="line.1117"></a> |
| <span class="sourceLineNo">1118</span> }<a name="line.1118"></a> |
| <span class="sourceLineNo">1119</span> List<KeyValueScanner> fileScanners = null;<a name="line.1119"></a> |
| <span class="sourceLineNo">1120</span> List<KeyValueScanner> newCurrentScanners;<a name="line.1120"></a> |
| <span class="sourceLineNo">1121</span> KeyValueHeap newHeap;<a name="line.1121"></a> |
| <span class="sourceLineNo">1122</span> try {<a name="line.1122"></a> |
| <span class="sourceLineNo">1123</span> // We must have a store instance here so no null check<a name="line.1123"></a> |
| <span class="sourceLineNo">1124</span> // recreate the scanners on the current file scanners<a name="line.1124"></a> |
| <span class="sourceLineNo">1125</span> fileScanners = store.recreateScanners(scannersToClose, cacheBlocks, false, false,<a name="line.1125"></a> |
| <span class="sourceLineNo">1126</span> matcher, scan.getStartRow(), scan.includeStartRow(), scan.getStopRow(),<a name="line.1126"></a> |
| <span class="sourceLineNo">1127</span> scan.includeStopRow(), readPt, false);<a name="line.1127"></a> |
| <span class="sourceLineNo">1128</span> if (fileScanners == null) {<a name="line.1128"></a> |
| <span class="sourceLineNo">1129</span> return;<a name="line.1129"></a> |
| <span class="sourceLineNo">1130</span> }<a name="line.1130"></a> |
| <span class="sourceLineNo">1131</span> seekScanners(fileScanners, lastTop, false, parallelSeekEnabled);<a name="line.1131"></a> |
| <span class="sourceLineNo">1132</span> newCurrentScanners = new ArrayList<>(fileScanners.size() + memstoreScanners.size());<a name="line.1132"></a> |
| <span class="sourceLineNo">1133</span> newCurrentScanners.addAll(fileScanners);<a name="line.1133"></a> |
| <span class="sourceLineNo">1134</span> newCurrentScanners.addAll(memstoreScanners);<a name="line.1134"></a> |
| <span class="sourceLineNo">1135</span> newHeap = newKVHeap(newCurrentScanners, comparator);<a name="line.1135"></a> |
| <span class="sourceLineNo">1136</span> } catch (Exception e) {<a name="line.1136"></a> |
| <span class="sourceLineNo">1137</span> LOG.warn("failed to switch to stream read", e);<a name="line.1137"></a> |
| <span class="sourceLineNo">1138</span> if (fileScanners != null) {<a name="line.1138"></a> |
| <span class="sourceLineNo">1139</span> fileScanners.forEach(KeyValueScanner::close);<a name="line.1139"></a> |
| <span class="sourceLineNo">1140</span> }<a name="line.1140"></a> |
| <span class="sourceLineNo">1141</span> return;<a name="line.1141"></a> |
| <span class="sourceLineNo">1142</span> }<a name="line.1142"></a> |
| <span class="sourceLineNo">1143</span> currentScanners.clear();<a name="line.1143"></a> |
| <span class="sourceLineNo">1144</span> addCurrentScanners(newCurrentScanners);<a name="line.1144"></a> |
| <span class="sourceLineNo">1145</span> this.heap = newHeap;<a name="line.1145"></a> |
| <span class="sourceLineNo">1146</span> resetQueryMatcher(lastTop);<a name="line.1146"></a> |
| <span class="sourceLineNo">1147</span> scannersToClose.forEach(KeyValueScanner::close);<a name="line.1147"></a> |
| <span class="sourceLineNo">1148</span> }<a name="line.1148"></a> |
| <span class="sourceLineNo">1149</span><a name="line.1149"></a> |
| <span class="sourceLineNo">1150</span> protected final boolean checkFlushed() {<a name="line.1150"></a> |
| <span class="sourceLineNo">1151</span> // check the var without any lock. Suppose even if we see the old<a name="line.1151"></a> |
| <span class="sourceLineNo">1152</span> // value here still it is ok to continue because we will not be resetting<a name="line.1152"></a> |
| <span class="sourceLineNo">1153</span> // the heap but will continue with the referenced memstore's snapshot. For compactions<a name="line.1153"></a> |
| <span class="sourceLineNo">1154</span> // any way we don't need the updateReaders at all to happen as we still continue with<a name="line.1154"></a> |
| <span class="sourceLineNo">1155</span> // the older files<a name="line.1155"></a> |
| <span class="sourceLineNo">1156</span> if (flushed) {<a name="line.1156"></a> |
| <span class="sourceLineNo">1157</span> // If there is a flush and the current scan is notified on the flush ensure that the<a name="line.1157"></a> |
| <span class="sourceLineNo">1158</span> // scan's heap gets reset and we do a seek on the newly flushed file.<a name="line.1158"></a> |
| <span class="sourceLineNo">1159</span> if (this.closing) {<a name="line.1159"></a> |
| <span class="sourceLineNo">1160</span> return false;<a name="line.1160"></a> |
| <span class="sourceLineNo">1161</span> }<a name="line.1161"></a> |
| <span class="sourceLineNo">1162</span> // reset the flag<a name="line.1162"></a> |
| <span class="sourceLineNo">1163</span> flushed = false;<a name="line.1163"></a> |
| <span class="sourceLineNo">1164</span> return true;<a name="line.1164"></a> |
| <span class="sourceLineNo">1165</span> }<a name="line.1165"></a> |
| <span class="sourceLineNo">1166</span> return false;<a name="line.1166"></a> |
| <span class="sourceLineNo">1167</span> }<a name="line.1167"></a> |
| <span class="sourceLineNo">1168</span><a name="line.1168"></a> |
| <span class="sourceLineNo">1169</span><a name="line.1169"></a> |
| <span class="sourceLineNo">1170</span> /**<a name="line.1170"></a> |
| <span class="sourceLineNo">1171</span> * Seek storefiles in parallel to optimize IO latency as much as possible<a name="line.1171"></a> |
| <span class="sourceLineNo">1172</span> * @param scanners the list {@link KeyValueScanner}s to be read from<a name="line.1172"></a> |
| <span class="sourceLineNo">1173</span> * @param kv the KeyValue on which the operation is being requested<a name="line.1173"></a> |
| <span class="sourceLineNo">1174</span> * @throws IOException<a name="line.1174"></a> |
| <span class="sourceLineNo">1175</span> */<a name="line.1175"></a> |
| <span class="sourceLineNo">1176</span> private void parallelSeek(final List<? extends KeyValueScanner><a name="line.1176"></a> |
| <span class="sourceLineNo">1177</span> scanners, final Cell kv) throws IOException {<a name="line.1177"></a> |
| <span class="sourceLineNo">1178</span> if (scanners.isEmpty()) return;<a name="line.1178"></a> |
| <span class="sourceLineNo">1179</span> int storeFileScannerCount = scanners.size();<a name="line.1179"></a> |
| <span class="sourceLineNo">1180</span> CountDownLatch latch = new CountDownLatch(storeFileScannerCount);<a name="line.1180"></a> |
| <span class="sourceLineNo">1181</span> List<ParallelSeekHandler> handlers = new ArrayList<>(storeFileScannerCount);<a name="line.1181"></a> |
| <span class="sourceLineNo">1182</span> for (KeyValueScanner scanner : scanners) {<a name="line.1182"></a> |
| <span class="sourceLineNo">1183</span> if (scanner instanceof StoreFileScanner) {<a name="line.1183"></a> |
| <span class="sourceLineNo">1184</span> ParallelSeekHandler seekHandler = new ParallelSeekHandler(scanner, kv,<a name="line.1184"></a> |
| <span class="sourceLineNo">1185</span> this.readPt, latch);<a name="line.1185"></a> |
| <span class="sourceLineNo">1186</span> executor.submit(seekHandler);<a name="line.1186"></a> |
| <span class="sourceLineNo">1187</span> handlers.add(seekHandler);<a name="line.1187"></a> |
| <span class="sourceLineNo">1188</span> } else {<a name="line.1188"></a> |
| <span class="sourceLineNo">1189</span> scanner.seek(kv);<a name="line.1189"></a> |
| <span class="sourceLineNo">1190</span> latch.countDown();<a name="line.1190"></a> |
| <span class="sourceLineNo">1191</span> }<a name="line.1191"></a> |
| <span class="sourceLineNo">1192</span> }<a name="line.1192"></a> |
| <span class="sourceLineNo">1193</span><a name="line.1193"></a> |
| <span class="sourceLineNo">1194</span> try {<a name="line.1194"></a> |
| <span class="sourceLineNo">1195</span> latch.await();<a name="line.1195"></a> |
| <span class="sourceLineNo">1196</span> } catch (InterruptedException ie) {<a name="line.1196"></a> |
| <span class="sourceLineNo">1197</span> throw (InterruptedIOException)new InterruptedIOException().initCause(ie);<a name="line.1197"></a> |
| <span class="sourceLineNo">1198</span> }<a name="line.1198"></a> |
| <span class="sourceLineNo">1199</span><a name="line.1199"></a> |
| <span class="sourceLineNo">1200</span> for (ParallelSeekHandler handler : handlers) {<a name="line.1200"></a> |
| <span class="sourceLineNo">1201</span> if (handler.getErr() != null) {<a name="line.1201"></a> |
| <span class="sourceLineNo">1202</span> throw new IOException(handler.getErr());<a name="line.1202"></a> |
| <span class="sourceLineNo">1203</span> }<a name="line.1203"></a> |
| <span class="sourceLineNo">1204</span> }<a name="line.1204"></a> |
| <span class="sourceLineNo">1205</span> }<a name="line.1205"></a> |
| <span class="sourceLineNo">1206</span><a name="line.1206"></a> |
| <span class="sourceLineNo">1207</span> /**<a name="line.1207"></a> |
| <span class="sourceLineNo">1208</span> * Used in testing.<a name="line.1208"></a> |
| <span class="sourceLineNo">1209</span> * @return all scanners in no particular order<a name="line.1209"></a> |
| <span class="sourceLineNo">1210</span> */<a name="line.1210"></a> |
| <span class="sourceLineNo">1211</span> @VisibleForTesting<a name="line.1211"></a> |
| <span class="sourceLineNo">1212</span> List<KeyValueScanner> getAllScannersForTesting() {<a name="line.1212"></a> |
| <span class="sourceLineNo">1213</span> List<KeyValueScanner> allScanners = new ArrayList<>();<a name="line.1213"></a> |
| <span class="sourceLineNo">1214</span> KeyValueScanner current = heap.getCurrentForTesting();<a name="line.1214"></a> |
| <span class="sourceLineNo">1215</span> if (current != null)<a name="line.1215"></a> |
| <span class="sourceLineNo">1216</span> allScanners.add(current);<a name="line.1216"></a> |
| <span class="sourceLineNo">1217</span> for (KeyValueScanner scanner : heap.getHeap())<a name="line.1217"></a> |
| <span class="sourceLineNo">1218</span> allScanners.add(scanner);<a name="line.1218"></a> |
| <span class="sourceLineNo">1219</span> return allScanners;<a name="line.1219"></a> |
| <span class="sourceLineNo">1220</span> }<a name="line.1220"></a> |
| <span class="sourceLineNo">1221</span><a name="line.1221"></a> |
| <span class="sourceLineNo">1222</span> static void enableLazySeekGlobally(boolean enable) {<a name="line.1222"></a> |
| <span class="sourceLineNo">1223</span> lazySeekEnabledGlobally = enable;<a name="line.1223"></a> |
| <span class="sourceLineNo">1224</span> }<a name="line.1224"></a> |
| <span class="sourceLineNo">1225</span><a name="line.1225"></a> |
| <span class="sourceLineNo">1226</span> /**<a name="line.1226"></a> |
| <span class="sourceLineNo">1227</span> * @return The estimated number of KVs seen by this scanner (includes some skipped KVs).<a name="line.1227"></a> |
| <span class="sourceLineNo">1228</span> */<a name="line.1228"></a> |
| <span class="sourceLineNo">1229</span> public long getEstimatedNumberOfKvsScanned() {<a name="line.1229"></a> |
| <span class="sourceLineNo">1230</span> return this.kvsScanned;<a name="line.1230"></a> |
| <span class="sourceLineNo">1231</span> }<a name="line.1231"></a> |
| <span class="sourceLineNo">1232</span><a name="line.1232"></a> |
| <span class="sourceLineNo">1233</span> @Override<a name="line.1233"></a> |
| <span class="sourceLineNo">1234</span> public Cell getNextIndexedKey() {<a name="line.1234"></a> |
| <span class="sourceLineNo">1235</span> return this.heap.getNextIndexedKey();<a name="line.1235"></a> |
| <span class="sourceLineNo">1236</span> }<a name="line.1236"></a> |
| <span class="sourceLineNo">1237</span><a name="line.1237"></a> |
| <span class="sourceLineNo">1238</span> @Override<a name="line.1238"></a> |
| <span class="sourceLineNo">1239</span> public void shipped() throws IOException {<a name="line.1239"></a> |
| <span class="sourceLineNo">1240</span> if (prevCell != null) {<a name="line.1240"></a> |
| <span class="sourceLineNo">1241</span> // Do the copy here so that in case the prevCell ref is pointing to the previous<a name="line.1241"></a> |
| <span class="sourceLineNo">1242</span> // blocks we can safely release those blocks.<a name="line.1242"></a> |
| <span class="sourceLineNo">1243</span> // This applies to blocks that are got from Bucket cache, L1 cache and the blocks<a name="line.1243"></a> |
| <span class="sourceLineNo">1244</span> // fetched from HDFS. Copying this would ensure that we let go the references to these<a name="line.1244"></a> |
| <span class="sourceLineNo">1245</span> // blocks so that they can be GCed safely(in case of bucket cache)<a name="line.1245"></a> |
| <span class="sourceLineNo">1246</span> prevCell = KeyValueUtil.toNewKeyCell(this.prevCell);<a name="line.1246"></a> |
| <span class="sourceLineNo">1247</span> }<a name="line.1247"></a> |
| <span class="sourceLineNo">1248</span> matcher.beforeShipped();<a name="line.1248"></a> |
| <span class="sourceLineNo">1249</span> // There wont be further fetch of Cells from these scanners. Just close.<a name="line.1249"></a> |
| <span class="sourceLineNo">1250</span> clearAndClose(scannersForDelayedClose);<a name="line.1250"></a> |
| <span class="sourceLineNo">1251</span> if (this.heap != null) {<a name="line.1251"></a> |
| <span class="sourceLineNo">1252</span> this.heap.shipped();<a name="line.1252"></a> |
| <span class="sourceLineNo">1253</span> // When switching from pread to stream, we will open a new scanner for each store file, but<a name="line.1253"></a> |
| <span class="sourceLineNo">1254</span> // the old scanner may still track the HFileBlocks we have scanned but not sent back to client<a name="line.1254"></a> |
| <span class="sourceLineNo">1255</span> // yet. If we close the scanner immediately then the HFileBlocks may be messed up by others<a name="line.1255"></a> |
| <span class="sourceLineNo">1256</span> // before we serialize and send it back to client. The HFileBlocks will be released in shipped<a name="line.1256"></a> |
| <span class="sourceLineNo">1257</span> // method, so we here will also open new scanners and close old scanners in shipped method.<a name="line.1257"></a> |
| <span class="sourceLineNo">1258</span> // See HBASE-18055 for more details.<a name="line.1258"></a> |
| <span class="sourceLineNo">1259</span> trySwitchToStreamRead();<a name="line.1259"></a> |
| <span class="sourceLineNo">1260</span> }<a name="line.1260"></a> |
| <span class="sourceLineNo">1261</span> }<a name="line.1261"></a> |
| <span class="sourceLineNo">1262</span>}<a name="line.1262"></a> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| </pre> |
| </div> |
| </body> |
| </html> |