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