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.yetus.audience.InterfaceAudience;
027import org.slf4j.Logger;
028import org.slf4j.LoggerFactory;
029import org.apache.hadoop.conf.Configuration;
030import org.apache.hadoop.fs.FileSystem;
031import org.apache.hadoop.fs.Path;
032import org.apache.hadoop.hbase.Cell;
033import org.apache.hadoop.hbase.HConstants;
034import org.apache.hadoop.hbase.PrivateCellUtil;
035import org.apache.hadoop.hbase.KeyValue;
036import org.apache.hadoop.hbase.client.Scan;
037import org.apache.hadoop.hbase.io.hfile.CacheConfig;
038import org.apache.hadoop.hbase.io.hfile.HFileScanner;
039import org.apache.hadoop.hbase.regionserver.StoreFileReader;
040import org.apache.hadoop.hbase.util.Bytes;
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 normal hfile.
071   * @param fs fileystem to read from
072   * @param p path to hfile
073   * @param cacheConf
074   * @param r original reference file (contains top or bottom)
075   * @param conf Configuration
076   * @throws IOException
077   */
078  public HalfStoreFileReader(FileSystem fs, Path p, CacheConfig cacheConf, Reference r,
079      boolean isPrimaryReplicaStoreFile, AtomicInteger refCount, boolean shared, Configuration conf)
080      throws IOException {
081    super(fs, p, cacheConf, isPrimaryReplicaStoreFile, refCount, shared, 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  /**
094   * Creates a half file reader for a hfile referred to by an hfilelink.
095   * @param fs fileystem to read from
096   * @param p path to hfile
097   * @param in {@link FSDataInputStreamWrapper}
098   * @param size Full size of the hfile file
099   * @param cacheConf
100   * @param r original reference file (contains top or bottom)
101   * @param conf Configuration
102   * @throws IOException
103   */
104  public HalfStoreFileReader(final FileSystem fs, final Path p, final FSDataInputStreamWrapper in,
105      long size, final CacheConfig cacheConf, final Reference r, boolean isPrimaryReplicaStoreFile,
106      AtomicInteger refCount, boolean shared, final Configuration conf) throws IOException {
107    super(fs, p, in, size, cacheConf, isPrimaryReplicaStoreFile, refCount, shared, conf);
108    // This is not actual midkey for this half-file; its just border
109    // around which we split top and bottom.  Have to look in files to find
110    // actual last and first keys for bottom and top halves.  Half-files don't
111    // have an actual midkey themselves. No midkey is how we indicate file is
112    // not splittable.
113    this.splitkey = r.getSplitKey();
114    this.splitCell = new KeyValue.KeyOnlyKeyValue(this.splitkey, 0, this.splitkey.length);
115    // Is it top or bottom half?
116    this.top = Reference.isTopFileRegion(r.getFileRegion());
117  }
118
119  protected boolean isTop() {
120    return this.top;
121  }
122
123  @Override
124  public HFileScanner getScanner(final boolean cacheBlocks,
125      final boolean pread, final boolean isCompaction) {
126    final HFileScanner s = super.getScanner(cacheBlocks, pread, isCompaction);
127    return new HFileScanner() {
128      final HFileScanner delegate = s;
129      public boolean atEnd = false;
130
131      @Override
132      public Cell getKey() {
133        if (atEnd) return null;
134        return delegate.getKey();
135      }
136
137      @Override
138      public String getKeyString() {
139        if (atEnd) return null;
140
141        return delegate.getKeyString();
142      }
143
144      @Override
145      public ByteBuffer getValue() {
146        if (atEnd) return null;
147
148        return delegate.getValue();
149      }
150
151      @Override
152      public String getValueString() {
153        if (atEnd) return null;
154
155        return delegate.getValueString();
156      }
157
158      @Override
159      public Cell getCell() {
160        if (atEnd) return null;
161
162        return delegate.getCell();
163      }
164
165      @Override
166      public boolean next() throws IOException {
167        if (atEnd) return false;
168
169        boolean b = delegate.next();
170        if (!b) {
171          return b;
172        }
173        // constrain the bottom.
174        if (!top) {
175          if (getComparator().compare(splitCell, getKey()) <= 0) {
176            atEnd = true;
177            return false;
178          }
179        }
180        return true;
181      }
182
183      @Override
184      public boolean seekTo() throws IOException {
185        if (top) {
186          int r = this.delegate.seekTo(splitCell);
187          if (r == HConstants.INDEX_KEY_MAGIC) {
188            return true;
189          }
190          if (r < 0) {
191            // midkey is < first key in file
192            return this.delegate.seekTo();
193          }
194          if (r > 0) {
195            return this.delegate.next();
196          }
197          return true;
198        }
199
200        boolean b = delegate.seekTo();
201        if (!b) {
202          return b;
203        }
204        // Check key.
205        return (this.delegate.getReader().getComparator().compare(splitCell, getKey())) > 0;
206      }
207
208      @Override
209      public org.apache.hadoop.hbase.io.hfile.HFile.Reader getReader() {
210        return this.delegate.getReader();
211      }
212
213      @Override
214      public boolean isSeeked() {
215        return this.delegate.isSeeked();
216      }
217
218      @Override
219      public int seekTo(Cell key) throws IOException {
220        if (top) {
221          if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) < 0) {
222            return -1;
223          }
224        } else {
225          if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) >= 0) {
226            // we would place the scanner in the second half.
227            // it might be an error to return false here ever...
228            boolean res = delegate.seekBefore(splitCell);
229            if (!res) {
230              throw new IOException(
231                  "Seeking for a key in bottom of file, but key exists in top of file, " +
232                  "failed on seekBefore(midkey)");
233            }
234            return 1;
235          }
236        }
237        return delegate.seekTo(key);
238      }
239
240      @Override
241      public int reseekTo(Cell key) throws IOException {
242        // This function is identical to the corresponding seekTo function
243        // except
244        // that we call reseekTo (and not seekTo) on the delegate.
245        if (top) {
246          if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) < 0) {
247            return -1;
248          }
249        } else {
250          if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) >= 0) {
251            // we would place the scanner in the second half.
252            // it might be an error to return false here ever...
253            boolean res = delegate.seekBefore(splitCell);
254            if (!res) {
255              throw new IOException("Seeking for a key in bottom of file, but"
256                  + " key exists in top of file, failed on seekBefore(midkey)");
257            }
258            return 1;
259          }
260        }
261        if (atEnd) {
262          // skip the 'reseek' and just return 1.
263          return 1;
264        }
265        return delegate.reseekTo(key);
266      }
267
268      @Override
269      public boolean seekBefore(Cell key) throws IOException {
270        if (top) {
271          Optional<Cell> fk = getFirstKey();
272          if (fk.isPresent() &&
273                  PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, fk.get()) <= 0) {
274            return false;
275          }
276        } else {
277          // The equals sign isn't strictly necessary just here to be consistent
278          // with seekTo
279          if (PrivateCellUtil.compareKeyIgnoresMvcc(getComparator(), key, splitCell) >= 0) {
280            boolean ret = this.delegate.seekBefore(splitCell);
281            if (ret) {
282              atEnd = false;
283            }
284            return ret;
285          }
286        }
287        boolean ret = this.delegate.seekBefore(key);
288        if (ret) {
289          atEnd = false;
290        }
291        return ret;
292      }
293
294      @Override
295      public Cell getNextIndexedKey() {
296        return null;
297      }
298
299      @Override
300      public void close() {
301        this.delegate.close();
302      }
303
304      @Override
305      public void shipped() throws IOException {
306        this.delegate.shipped();
307      }
308    };
309  }
310  
311  @Override
312  public boolean passesKeyRangeFilter(Scan scan) {
313    return true;
314  }
315  
316  @Override
317  public Optional<Cell> getLastKey() {
318    if (top) {
319      return super.getLastKey();
320    }
321    // Get a scanner that caches the block and that uses pread.
322    HFileScanner scanner = getScanner(true, true);
323    try {
324      if (scanner.seekBefore(this.splitCell)) {
325        return Optional.ofNullable(scanner.getKey());
326      }
327    } catch (IOException e) {
328      LOG.warn("Failed seekBefore " + Bytes.toStringBinary(this.splitkey), e);
329    } finally {
330      if (scanner != null) {
331        scanner.close();
332      }
333    }
334    return Optional.empty();
335  }
336
337  @Override
338  public Optional<Cell> midKey() throws IOException {
339    // Returns null to indicate file is not splitable.
340    return Optional.empty();
341  }
342
343  @Override
344  public Optional<Cell> getFirstKey() {
345    if (!firstKeySeeked) {
346      HFileScanner scanner = getScanner(true, true, false);
347      try {
348        if (scanner.seekTo()) {
349          this.firstKey = Optional.ofNullable(scanner.getKey());
350        }
351        firstKeySeeked = true;
352      } catch (IOException e) {
353        LOG.warn("Failed seekTo first KV in the file", e);
354      } finally {
355        if(scanner != null) {
356          scanner.close();
357        }
358      }
359    }
360    return this.firstKey;
361  }
362
363  @Override
364  public long getEntries() {
365    // Estimate the number of entries as half the original file; this may be wildly inaccurate.
366    return super.getEntries() / 2;
367  }
368
369  @Override
370  public long getFilterEntries() {
371    // Estimate the number of entries as half the original file; this may be wildly inaccurate.
372    return super.getFilterEntries() / 2;
373  }
374}