001/** 002 * 003 * Licensed to the Apache Software Foundation (ASF) under one 004 * or more contributor license agreements. See the NOTICE file 005 * distributed with this work for additional information 006 * regarding copyright ownership. The ASF licenses this file 007 * to you under the Apache License, Version 2.0 (the 008 * "License"); you may not use this file except in compliance 009 * with the License. You may obtain a copy of the License at 010 * 011 * http://www.apache.org/licenses/LICENSE-2.0 012 * 013 * Unless required by applicable law or agreed to in writing, software 014 * distributed under the License is distributed on an "AS IS" BASIS, 015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 016 * See the License for the specific language governing permissions and 017 * limitations under the License. 018 */ 019package org.apache.hadoop.hbase.io; 020 021import java.io.IOException; 022import java.nio.ByteBuffer; 023import java.util.Optional; 024import java.util.concurrent.atomic.AtomicInteger; 025 026import org.apache.hadoop.conf.Configuration; 027import org.apache.hadoop.hbase.Cell; 028import org.apache.hadoop.hbase.HConstants; 029import org.apache.hadoop.hbase.KeyValue; 030import org.apache.hadoop.hbase.PrivateCellUtil; 031import org.apache.hadoop.hbase.client.Scan; 032import org.apache.hadoop.hbase.io.hfile.CacheConfig; 033import org.apache.hadoop.hbase.io.hfile.HFileInfo; 034import org.apache.hadoop.hbase.io.hfile.HFileScanner; 035import org.apache.hadoop.hbase.io.hfile.ReaderContext; 036import org.apache.hadoop.hbase.regionserver.StoreFileReader; 037import org.apache.hadoop.hbase.util.Bytes; 038import org.apache.yetus.audience.InterfaceAudience; 039import org.slf4j.Logger; 040import org.slf4j.LoggerFactory; 041 042/** 043 * A facade for a {@link org.apache.hadoop.hbase.io.hfile.HFile.Reader} that serves up 044 * either the top or bottom half of a HFile where 'bottom' is the first half 045 * of the file containing the keys that sort lowest and 'top' is the second half 046 * of the file with keys that sort greater than those of the bottom half. 047 * The top includes the split files midkey, of the key that follows if it does 048 * not exist in the file. 049 * 050 * <p>This type works in tandem with the {@link Reference} type. This class 051 * is used reading while Reference is used writing. 052 * 053 * <p>This file is not splitable. Calls to {@link #midKey()} return null. 054 */ 055@InterfaceAudience.Private 056public class HalfStoreFileReader extends StoreFileReader { 057 private static final Logger LOG = LoggerFactory.getLogger(HalfStoreFileReader.class); 058 final boolean top; 059 // This is the key we split around. Its the first possible entry on a row: 060 // i.e. empty column and a timestamp of LATEST_TIMESTAMP. 061 protected final byte [] splitkey; 062 063 private final Cell splitCell; 064 065 private Optional<Cell> firstKey = Optional.empty(); 066 067 private boolean firstKeySeeked = false; 068 069 /** 070 * Creates a half file reader for a hfile referred to by an hfilelink. 071 * @param context Reader context info 072 * @param fileInfo HFile info 073 * @param cacheConf CacheConfig 074 * @param r original reference file (contains top or bottom) 075 * @param refCount reference count 076 * @param conf Configuration 077 */ 078 public HalfStoreFileReader(final ReaderContext context, final HFileInfo fileInfo, 079 final CacheConfig cacheConf, final Reference r, 080 AtomicInteger refCount, final Configuration conf) throws IOException { 081 super(context, fileInfo, cacheConf, refCount, conf); 082 // This is not actual midkey for this half-file; its just border 083 // around which we split top and bottom. Have to look in files to find 084 // actual last and first keys for bottom and top halves. Half-files don't 085 // have an actual midkey themselves. No midkey is how we indicate file is 086 // not splittable. 087 this.splitkey = r.getSplitKey(); 088 this.splitCell = new KeyValue.KeyOnlyKeyValue(this.splitkey, 0, this.splitkey.length); 089 // Is it top or bottom half? 090 this.top = Reference.isTopFileRegion(r.getFileRegion()); 091 } 092 093 protected boolean isTop() { 094 return this.top; 095 } 096 097 @Override 098 public HFileScanner getScanner(final boolean cacheBlocks, 099 final boolean pread, final boolean isCompaction) { 100 final HFileScanner s = super.getScanner(cacheBlocks, pread, isCompaction); 101 return new HFileScanner() { 102 final HFileScanner delegate = s; 103 public boolean atEnd = false; 104 105 @Override 106 public Cell getKey() { 107 if (atEnd) return null; 108 return delegate.getKey(); 109 } 110 111 @Override 112 public String getKeyString() { 113 if (atEnd) return null; 114 115 return delegate.getKeyString(); 116 } 117 118 @Override 119 public ByteBuffer getValue() { 120 if (atEnd) return null; 121 122 return delegate.getValue(); 123 } 124 125 @Override 126 public String getValueString() { 127 if (atEnd) return null; 128 129 return delegate.getValueString(); 130 } 131 132 @Override 133 public Cell getCell() { 134 if (atEnd) return null; 135 136 return delegate.getCell(); 137 } 138 139 @Override 140 public boolean next() throws IOException { 141 if (atEnd) return false; 142 143 boolean b = delegate.next(); 144 if (!b) { 145 return b; 146 } 147 // constrain the bottom. 148 if (!top) { 149 if (getComparator().compare(splitCell, getKey()) <= 0) { 150 atEnd = true; 151 return false; 152 } 153 } 154 return true; 155 } 156 157 @Override 158 public boolean seekTo() throws IOException { 159 if (top) { 160 int r = this.delegate.seekTo(splitCell); 161 if (r == HConstants.INDEX_KEY_MAGIC) { 162 return true; 163 } 164 if (r < 0) { 165 // midkey is < first key in file 166 return this.delegate.seekTo(); 167 } 168 if (r > 0) { 169 return this.delegate.next(); 170 } 171 return true; 172 } 173 174 boolean b = delegate.seekTo(); 175 if (!b) { 176 return b; 177 } 178 // Check key. 179 return (this.delegate.getReader().getComparator().compare(splitCell, getKey())) > 0; 180 } 181 182 @Override 183 public org.apache.hadoop.hbase.io.hfile.HFile.Reader getReader() { 184 return this.delegate.getReader(); 185 } 186 187 @Override 188 public boolean isSeeked() { 189 return this.delegate.isSeeked(); 190 } 191 192 @Override 193 public int seekTo(Cell key) throws IOException { 194 if (top) { 195 if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) < 0) { 196 return -1; 197 } 198 } else { 199 if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) >= 0) { 200 // we would place the scanner in the second half. 201 // it might be an error to return false here ever... 202 boolean res = delegate.seekBefore(splitCell); 203 if (!res) { 204 throw new IOException( 205 "Seeking for a key in bottom of file, but key exists in top of file, " + 206 "failed on seekBefore(midkey)"); 207 } 208 return 1; 209 } 210 } 211 return delegate.seekTo(key); 212 } 213 214 @Override 215 public int reseekTo(Cell key) throws IOException { 216 // This function is identical to the corresponding seekTo function 217 // except 218 // that we call reseekTo (and not seekTo) on the delegate. 219 if (top) { 220 if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) < 0) { 221 return -1; 222 } 223 } else { 224 if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) >= 0) { 225 // we would place the scanner in the second half. 226 // it might be an error to return false here ever... 227 boolean res = delegate.seekBefore(splitCell); 228 if (!res) { 229 throw new IOException("Seeking for a key in bottom of file, but" 230 + " key exists in top of file, failed on seekBefore(midkey)"); 231 } 232 return 1; 233 } 234 } 235 if (atEnd) { 236 // skip the 'reseek' and just return 1. 237 return 1; 238 } 239 return delegate.reseekTo(key); 240 } 241 242 @Override 243 public boolean seekBefore(Cell key) throws IOException { 244 if (top) { 245 Optional<Cell> fk = getFirstKey(); 246 if (fk.isPresent() && 247 PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, fk.get()) <= 0) { 248 return false; 249 } 250 } else { 251 // The equals sign isn't strictly necessary just here to be consistent 252 // with seekTo 253 if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) >= 0) { 254 boolean ret = this.delegate.seekBefore(splitCell); 255 if (ret) { 256 atEnd = false; 257 } 258 return ret; 259 } 260 } 261 boolean ret = this.delegate.seekBefore(key); 262 if (ret) { 263 atEnd = false; 264 } 265 return ret; 266 } 267 268 @Override 269 public Cell getNextIndexedKey() { 270 return null; 271 } 272 273 @Override 274 public void close() { 275 this.delegate.close(); 276 } 277 278 @Override 279 public void shipped() throws IOException { 280 this.delegate.shipped(); 281 } 282 }; 283 } 284 285 @Override 286 public boolean passesKeyRangeFilter(Scan scan) { 287 return true; 288 } 289 290 @Override 291 public Optional<Cell> getLastKey() { 292 if (top) { 293 return super.getLastKey(); 294 } 295 // Get a scanner that caches the block and that uses pread. 296 HFileScanner scanner = getScanner(true, true); 297 try { 298 if (scanner.seekBefore(this.splitCell)) { 299 return Optional.ofNullable(scanner.getKey()); 300 } 301 } catch (IOException e) { 302 LOG.warn("Failed seekBefore " + Bytes.toStringBinary(this.splitkey), e); 303 } finally { 304 if (scanner != null) { 305 scanner.close(); 306 } 307 } 308 return Optional.empty(); 309 } 310 311 @Override 312 public Optional<Cell> midKey() throws IOException { 313 // Returns null to indicate file is not splitable. 314 return Optional.empty(); 315 } 316 317 @Override 318 public Optional<Cell> getFirstKey() { 319 if (!firstKeySeeked) { 320 HFileScanner scanner = getScanner(true, true, false); 321 try { 322 if (scanner.seekTo()) { 323 this.firstKey = Optional.ofNullable(scanner.getKey()); 324 } 325 firstKeySeeked = true; 326 } catch (IOException e) { 327 LOG.warn("Failed seekTo first KV in the file", e); 328 } finally { 329 if(scanner != null) { 330 scanner.close(); 331 } 332 } 333 } 334 return this.firstKey; 335 } 336 337 @Override 338 public long getEntries() { 339 // Estimate the number of entries as half the original file; this may be wildly inaccurate. 340 return super.getEntries() / 2; 341 } 342 343 @Override 344 public long getFilterEntries() { 345 // Estimate the number of entries as half the original file; this may be wildly inaccurate. 346 return super.getFilterEntries() / 2; 347 } 348}