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.regionserver;
019
020import java.io.FileNotFoundException;
021import java.io.IOException;
022import java.util.concurrent.atomic.AtomicInteger;
023import java.util.regex.Matcher;
024import java.util.regex.Pattern;
025import org.apache.hadoop.conf.Configurable;
026import org.apache.hadoop.conf.Configuration;
027import org.apache.hadoop.fs.FileStatus;
028import org.apache.hadoop.fs.FileSystem;
029import org.apache.hadoop.fs.Path;
030import org.apache.hadoop.hbase.HDFSBlocksDistribution;
031import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
032import org.apache.hadoop.hbase.io.HFileLink;
033import org.apache.hadoop.hbase.io.HalfStoreFileReader;
034import org.apache.hadoop.hbase.io.Reference;
035import org.apache.hadoop.hbase.io.hfile.CacheConfig;
036import org.apache.hadoop.hbase.io.hfile.HFileInfo;
037import org.apache.hadoop.hbase.io.hfile.ReaderContext;
038import org.apache.hadoop.hbase.io.hfile.ReaderContext.ReaderType;
039import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder;
040import org.apache.hadoop.hbase.mob.MobUtils;
041import org.apache.hadoop.hbase.util.FSUtils;
042import org.apache.yetus.audience.InterfaceAudience;
043import org.slf4j.Logger;
044import org.slf4j.LoggerFactory;
045
046/**
047 * Describe a StoreFile (hfile, reference, link)
048 */
049@InterfaceAudience.Private
050public class StoreFileInfo implements Configurable {
051  private static final Logger LOG = LoggerFactory.getLogger(StoreFileInfo.class);
052
053  /**
054   * A non-capture group, for hfiles, so that this can be embedded. HFiles are uuid ([0-9a-z]+).
055   * Bulk loaded hfiles has (_SeqId_[0-9]+_) has suffix. The mob del file has (_del) as suffix.
056   */
057  public static final String HFILE_NAME_REGEX = "[0-9a-f]+(?:(?:_SeqId_[0-9]+_)|(?:_del))?";
058
059  /** Regex that will work for hfiles */
060  private static final Pattern HFILE_NAME_PATTERN = Pattern.compile("^(" + HFILE_NAME_REGEX + ")");
061
062  /**
063   * Regex that will work for straight reference names ({@code <hfile>.<parentEncRegion>}) and
064   * hfilelink reference names ({@code
065   *
066  <table>
067   * =<region>-<hfile>.<parentEncRegion>}) If reference, then the regex has more than just one
068   * group. Group 1, hfile/hfilelink pattern, is this file's id. Group 2 '(.+)' is the reference's
069   * parent region name.
070   */
071  private static final Pattern REF_NAME_PATTERN =
072    Pattern.compile(String.format("^(%s|%s)\\.(.+)$", HFILE_NAME_REGEX, HFileLink.LINK_NAME_REGEX));
073
074  public static final String STORE_FILE_READER_NO_READAHEAD = "hbase.store.reader.no-readahead";
075  public static final boolean DEFAULT_STORE_FILE_READER_NO_READAHEAD = false;
076
077  // Configuration
078  private Configuration conf;
079
080  // FileSystem handle
081  private final FileSystem fs;
082
083  // HDFS blocks distribution information
084  private HDFSBlocksDistribution hdfsBlocksDistribution = null;
085
086  private HFileInfo hfileInfo;
087
088  // If this storefile references another, this is the reference instance.
089  private final Reference reference;
090
091  // If this storefile is a link to another, this is the link instance.
092  private final HFileLink link;
093
094  private final Path initialPath;
095
096  private RegionCoprocessorHost coprocessorHost;
097
098  // timestamp on when the file was created, is 0 and ignored for reference or link files
099  private long createdTimestamp;
100
101  private long size;
102
103  private final boolean primaryReplica;
104
105  private final boolean noReadahead;
106
107  // Counter that is incremented every time a scanner is created on the
108  // store file. It is decremented when the scan on the store file is
109  // done.
110  final AtomicInteger refCount = new AtomicInteger(0);
111
112  /**
113   * Create a Store File Info
114   * @param conf           the {@link Configuration} to use
115   * @param fs             The current file system to use.
116   * @param initialPath    The {@link Path} of the file
117   * @param primaryReplica true if this is a store file for primary replica, otherwise false.
118   */
119  public StoreFileInfo(final Configuration conf, final FileSystem fs, final Path initialPath,
120    final boolean primaryReplica) throws IOException {
121    this(conf, fs, null, initialPath, primaryReplica);
122  }
123
124  private StoreFileInfo(final Configuration conf, final FileSystem fs, final FileStatus fileStatus,
125    final Path initialPath, final boolean primaryReplica) throws IOException {
126    assert fs != null;
127    assert initialPath != null;
128    assert conf != null;
129
130    this.fs = fs;
131    this.conf = conf;
132    this.initialPath = fs.makeQualified(initialPath);
133    this.primaryReplica = primaryReplica;
134    this.noReadahead =
135      this.conf.getBoolean(STORE_FILE_READER_NO_READAHEAD, DEFAULT_STORE_FILE_READER_NO_READAHEAD);
136    Path p = initialPath;
137    if (HFileLink.isHFileLink(p)) {
138      // HFileLink
139      this.reference = null;
140      this.link = HFileLink.buildFromHFileLinkPattern(conf, p);
141      LOG.trace("{} is a link", p);
142    } else if (isReference(p)) {
143      this.reference = Reference.read(fs, p);
144      Path referencePath = getReferredToFile(p);
145      if (HFileLink.isHFileLink(referencePath)) {
146        // HFileLink Reference
147        this.link = HFileLink.buildFromHFileLinkPattern(conf, referencePath);
148      } else {
149        // Reference
150        this.link = null;
151      }
152      LOG.trace("{} is a {} reference to {}", p, reference.getFileRegion(), referencePath);
153    } else if (isHFile(p) || isMobFile(p) || isMobRefFile(p)) {
154      // HFile
155      if (fileStatus != null) {
156        this.createdTimestamp = fileStatus.getModificationTime();
157        this.size = fileStatus.getLen();
158      } else {
159        FileStatus fStatus = fs.getFileStatus(initialPath);
160        this.createdTimestamp = fStatus.getModificationTime();
161        this.size = fStatus.getLen();
162      }
163      this.reference = null;
164      this.link = null;
165    } else {
166      throw new IOException("path=" + p + " doesn't look like a valid StoreFile");
167    }
168  }
169
170  /**
171   * Create a Store File Info
172   * @param conf       the {@link Configuration} to use
173   * @param fs         The current file system to use.
174   * @param fileStatus The {@link FileStatus} of the file
175   */
176  public StoreFileInfo(final Configuration conf, final FileSystem fs, final FileStatus fileStatus)
177    throws IOException {
178    this(conf, fs, fileStatus, fileStatus.getPath(), true);
179  }
180
181  /**
182   * Create a Store File Info from an HFileLink
183   * @param conf       The {@link Configuration} to use
184   * @param fs         The current file system to use
185   * @param fileStatus The {@link FileStatus} of the file
186   */
187  public StoreFileInfo(final Configuration conf, final FileSystem fs, final FileStatus fileStatus,
188    final HFileLink link) {
189    this(conf, fs, fileStatus, null, link);
190  }
191
192  /**
193   * Create a Store File Info from an HFileLink
194   * @param conf       The {@link Configuration} to use
195   * @param fs         The current file system to use
196   * @param fileStatus The {@link FileStatus} of the file
197   * @param reference  The reference instance
198   */
199  public StoreFileInfo(final Configuration conf, final FileSystem fs, final FileStatus fileStatus,
200    final Reference reference) {
201    this(conf, fs, fileStatus, reference, null);
202  }
203
204  /**
205   * Create a Store File Info from an HFileLink and a Reference
206   * @param conf       The {@link Configuration} to use
207   * @param fs         The current file system to use
208   * @param fileStatus The {@link FileStatus} of the file
209   * @param reference  The reference instance
210   * @param link       The link instance
211   */
212  public StoreFileInfo(final Configuration conf, final FileSystem fs, final FileStatus fileStatus,
213    final Reference reference, final HFileLink link) {
214    this.fs = fs;
215    this.conf = conf;
216    this.primaryReplica = false;
217    this.initialPath = (fileStatus == null) ? null : fileStatus.getPath();
218    this.createdTimestamp = (fileStatus == null) ? 0 : fileStatus.getModificationTime();
219    this.reference = reference;
220    this.link = link;
221    this.noReadahead =
222      this.conf.getBoolean(STORE_FILE_READER_NO_READAHEAD, DEFAULT_STORE_FILE_READER_NO_READAHEAD);
223  }
224
225  @Override
226  public Configuration getConf() {
227    return conf;
228  }
229
230  @Override
231  public void setConf(Configuration conf) {
232    this.conf = conf;
233  }
234
235  /**
236   * Size of the Hfile n
237   */
238  public long getSize() {
239    return size;
240  }
241
242  /**
243   * Sets the region coprocessor env. n
244   */
245  public void setRegionCoprocessorHost(RegionCoprocessorHost coprocessorHost) {
246    this.coprocessorHost = coprocessorHost;
247  }
248
249  /*
250   * @return the Reference object associated to this StoreFileInfo. null if the StoreFile is not a
251   * reference.
252   */
253  public Reference getReference() {
254    return this.reference;
255  }
256
257  /** Returns True if the store file is a Reference */
258  public boolean isReference() {
259    return this.reference != null;
260  }
261
262  /** Returns True if the store file is a top Reference */
263  public boolean isTopReference() {
264    return this.reference != null && Reference.isTopFileRegion(this.reference.getFileRegion());
265  }
266
267  /** Returns True if the store file is a link */
268  public boolean isLink() {
269    return this.link != null && this.reference == null;
270  }
271
272  /** Returns the HDFS block distribution */
273  public HDFSBlocksDistribution getHDFSBlockDistribution() {
274    return this.hdfsBlocksDistribution;
275  }
276
277  StoreFileReader createReader(ReaderContext context, CacheConfig cacheConf) throws IOException {
278    StoreFileReader reader = null;
279    if (this.reference != null) {
280      reader = new HalfStoreFileReader(context, hfileInfo, cacheConf, reference, refCount, conf);
281    } else {
282      reader = new StoreFileReader(context, hfileInfo, cacheConf, refCount, conf);
283    }
284    return reader;
285  }
286
287  ReaderContext createReaderContext(boolean doDropBehind, long readahead, ReaderType type)
288    throws IOException {
289    FSDataInputStreamWrapper in;
290    FileStatus status;
291    if (this.link != null) {
292      // HFileLink
293      in = new FSDataInputStreamWrapper(fs, this.link, doDropBehind, readahead);
294      status = this.link.getFileStatus(fs);
295    } else if (this.reference != null) {
296      // HFile Reference
297      Path referencePath = getReferredToFile(this.getPath());
298      try {
299        in = new FSDataInputStreamWrapper(fs, referencePath, doDropBehind, readahead);
300      } catch (FileNotFoundException fnfe) {
301        // Intercept the exception so can insert more info about the Reference; otherwise
302        // exception just complains about some random file -- operator doesn't realize it
303        // other end of a Reference
304        FileNotFoundException newFnfe = new FileNotFoundException(toString());
305        newFnfe.initCause(fnfe);
306        throw newFnfe;
307      }
308      status = fs.getFileStatus(referencePath);
309    } else {
310      in = new FSDataInputStreamWrapper(fs, this.getPath(), doDropBehind, readahead);
311      status = fs.getFileStatus(initialPath);
312    }
313    long length = status.getLen();
314    ReaderContextBuilder contextBuilder =
315      new ReaderContextBuilder().withInputStreamWrapper(in).withFileSize(length)
316        .withPrimaryReplicaReader(this.primaryReplica).withReaderType(type).withFileSystem(fs);
317    if (this.reference != null) {
318      contextBuilder.withFilePath(this.getPath());
319    } else {
320      contextBuilder.withFilePath(status.getPath());
321    }
322    return contextBuilder.build();
323  }
324
325  /**
326   * Compute the HDFS Block Distribution for this StoreFile
327   */
328  public HDFSBlocksDistribution computeHDFSBlocksDistribution(final FileSystem fs)
329    throws IOException {
330    // guard against the case where we get the FileStatus from link, but by the time we
331    // call compute the file is moved again
332    if (this.link != null) {
333      FileNotFoundException exToThrow = null;
334      for (int i = 0; i < this.link.getLocations().length; i++) {
335        try {
336          return computeHDFSBlocksDistributionInternal(fs);
337        } catch (FileNotFoundException ex) {
338          // try the other location
339          exToThrow = ex;
340        }
341      }
342      throw exToThrow;
343    } else {
344      return computeHDFSBlocksDistributionInternal(fs);
345    }
346  }
347
348  private HDFSBlocksDistribution computeHDFSBlocksDistributionInternal(final FileSystem fs)
349    throws IOException {
350    FileStatus status = getReferencedFileStatus(fs);
351    if (this.reference != null) {
352      return computeRefFileHDFSBlockDistribution(fs, reference, status);
353    } else {
354      return FSUtils.computeHDFSBlocksDistribution(fs, status, 0, status.getLen());
355    }
356  }
357
358  /**
359   * Get the {@link FileStatus} of the file referenced by this StoreFileInfo
360   * @param fs The current file system to use.
361   * @return The {@link FileStatus} of the file referenced by this StoreFileInfo
362   */
363  public FileStatus getReferencedFileStatus(final FileSystem fs) throws IOException {
364    FileStatus status;
365    if (this.reference != null) {
366      if (this.link != null) {
367        FileNotFoundException exToThrow = null;
368        for (int i = 0; i < this.link.getLocations().length; i++) {
369          // HFileLink Reference
370          try {
371            return link.getFileStatus(fs);
372          } catch (FileNotFoundException ex) {
373            // try the other location
374            exToThrow = ex;
375          }
376        }
377        throw exToThrow;
378      } else {
379        // HFile Reference
380        Path referencePath = getReferredToFile(this.getPath());
381        status = fs.getFileStatus(referencePath);
382      }
383    } else {
384      if (this.link != null) {
385        FileNotFoundException exToThrow = null;
386        for (int i = 0; i < this.link.getLocations().length; i++) {
387          // HFileLink
388          try {
389            return link.getFileStatus(fs);
390          } catch (FileNotFoundException ex) {
391            // try the other location
392            exToThrow = ex;
393          }
394        }
395        throw exToThrow;
396      } else {
397        status = fs.getFileStatus(initialPath);
398      }
399    }
400    return status;
401  }
402
403  /** Returns The {@link Path} of the file */
404  public Path getPath() {
405    return initialPath;
406  }
407
408  /** Returns The {@link FileStatus} of the file */
409  public FileStatus getFileStatus() throws IOException {
410    return getReferencedFileStatus(fs);
411  }
412
413  /** Returns Get the modification time of the file. */
414  public long getModificationTime() throws IOException {
415    return getFileStatus().getModificationTime();
416  }
417
418  @Override
419  public String toString() {
420    return this.getPath()
421      + (isReference() ? "->" + getReferredToFile(this.getPath()) + "-" + reference : "");
422  }
423
424  /**
425   * @param path Path to check.
426   * @return True if the path has format of a HFile.
427   */
428  public static boolean isHFile(final Path path) {
429    return isHFile(path.getName());
430  }
431
432  public static boolean isHFile(final String fileName) {
433    Matcher m = HFILE_NAME_PATTERN.matcher(fileName);
434    return m.matches() && m.groupCount() > 0;
435  }
436
437  /**
438   * Checks if the file is a MOB file
439   * @param path path to a file
440   * @return true, if - yes, false otherwise
441   */
442  public static boolean isMobFile(final Path path) {
443    String fileName = path.getName();
444    String[] parts = fileName.split(MobUtils.SEP);
445    if (parts.length != 2) {
446      return false;
447    }
448    Matcher m = HFILE_NAME_PATTERN.matcher(parts[0]);
449    Matcher mm = HFILE_NAME_PATTERN.matcher(parts[1]);
450    return m.matches() && mm.matches();
451  }
452
453  /**
454   * Checks if the file is a MOB reference file, created by snapshot
455   * @param path path to a file
456   * @return true, if - yes, false otherwise
457   */
458  public static boolean isMobRefFile(final Path path) {
459    String fileName = path.getName();
460    int lastIndex = fileName.lastIndexOf(MobUtils.SEP);
461    if (lastIndex < 0) {
462      return false;
463    }
464    String[] parts = new String[2];
465    parts[0] = fileName.substring(0, lastIndex);
466    parts[1] = fileName.substring(lastIndex + 1);
467    String name = parts[0] + "." + parts[1];
468    Matcher m = REF_NAME_PATTERN.matcher(name);
469    return m.matches() && m.groupCount() > 1;
470  }
471
472  /**
473   * @param path Path to check.
474   * @return True if the path has format of a HStoreFile reference.
475   */
476  public static boolean isReference(final Path path) {
477    return isReference(path.getName());
478  }
479
480  /**
481   * @param name file name to check.
482   * @return True if the path has format of a HStoreFile reference.
483   */
484  public static boolean isReference(final String name) {
485    Matcher m = REF_NAME_PATTERN.matcher(name);
486    return m.matches() && m.groupCount() > 1;
487  }
488
489  /** Returns timestamp when this file was created (as returned by filesystem) */
490  public long getCreatedTimestamp() {
491    return createdTimestamp;
492  }
493
494  /*
495   * Return path to the file referred to by a Reference. Presumes a directory hierarchy of
496   * <code>${hbase.rootdir}/data/${namespace}/tablename/regionname/familyname</code>.
497   * @param p Path to a Reference file.
498   * @return Calculated path to parent region file.
499   * @throws IllegalArgumentException when path regex fails to match.
500   */
501  public static Path getReferredToFile(final Path p) {
502    Matcher m = REF_NAME_PATTERN.matcher(p.getName());
503    if (m == null || !m.matches()) {
504      LOG.warn("Failed match of store file name {}", p.toString());
505      throw new IllegalArgumentException("Failed match of store file name " + p.toString());
506    }
507
508    // Other region name is suffix on the passed Reference file name
509    String otherRegion = m.group(2);
510    // Tabledir is up two directories from where Reference was written.
511    Path tableDir = p.getParent().getParent().getParent();
512    String nameStrippedOfSuffix = m.group(1);
513    LOG.trace("reference {} to region={} hfile={}", p, otherRegion, nameStrippedOfSuffix);
514
515    // Build up new path with the referenced region in place of our current
516    // region in the reference path. Also strip regionname suffix from name.
517    return new Path(new Path(new Path(tableDir, otherRegion), p.getParent().getName()),
518      nameStrippedOfSuffix);
519  }
520
521  /**
522   * Validate the store file name.
523   * @param fileName name of the file to validate
524   * @return <tt>true</tt> if the file could be a valid store file, <tt>false</tt> otherwise
525   */
526  public static boolean validateStoreFileName(final String fileName) {
527    if (HFileLink.isHFileLink(fileName) || isReference(fileName)) return (true);
528    return !fileName.contains("-");
529  }
530
531  /**
532   * Return if the specified file is a valid store file or not.
533   * @param fileStatus The {@link FileStatus} of the file
534   * @return <tt>true</tt> if the file is valid
535   */
536  public static boolean isValid(final FileStatus fileStatus) throws IOException {
537    final Path p = fileStatus.getPath();
538
539    if (fileStatus.isDirectory()) return false;
540
541    // Check for empty hfile. Should never be the case but can happen
542    // after data loss in hdfs for whatever reason (upgrade, etc.): HBASE-646
543    // NOTE: that the HFileLink is just a name, so it's an empty file.
544    if (!HFileLink.isHFileLink(p) && fileStatus.getLen() <= 0) {
545      LOG.warn("Skipping {} because it is empty. HBASE-646 DATA LOSS?", p);
546      return false;
547    }
548
549    return validateStoreFileName(p.getName());
550  }
551
552  /**
553   * helper function to compute HDFS blocks distribution of a given reference file.For reference
554   * file, we don't compute the exact value. We use some estimate instead given it might be good
555   * enough. we assume bottom part takes the first half of reference file, top part takes the second
556   * half of the reference file. This is just estimate, given midkey ofregion != midkey of HFile,
557   * also the number and size of keys vary. If this estimate isn't good enough, we can improve it
558   * later.
559   * @param fs        The FileSystem
560   * @param reference The reference
561   * @param status    The reference FileStatus
562   * @return HDFS blocks distribution
563   */
564  private static HDFSBlocksDistribution computeRefFileHDFSBlockDistribution(final FileSystem fs,
565    final Reference reference, final FileStatus status) throws IOException {
566    if (status == null) {
567      return null;
568    }
569
570    long start = 0;
571    long length = 0;
572
573    if (Reference.isTopFileRegion(reference.getFileRegion())) {
574      start = status.getLen() / 2;
575      length = status.getLen() - status.getLen() / 2;
576    } else {
577      start = 0;
578      length = status.getLen() / 2;
579    }
580    return FSUtils.computeHDFSBlocksDistribution(fs, status, start, length);
581  }
582
583  @Override
584  public boolean equals(Object that) {
585    if (this == that) {
586      return true;
587    }
588    if (that == null) {
589      return false;
590    }
591
592    if (!(that instanceof StoreFileInfo)) {
593      return false;
594    }
595
596    StoreFileInfo o = (StoreFileInfo) that;
597    if (initialPath != null && o.initialPath == null) return false;
598    if (initialPath == null && o.initialPath != null) return false;
599    if (initialPath != o.initialPath && initialPath != null && !initialPath.equals(o.initialPath))
600      return false;
601
602    if (reference != null && o.reference == null) return false;
603    if (reference == null && o.reference != null) return false;
604    if (reference != o.reference && reference != null && !reference.equals(o.reference))
605      return false;
606
607    if (link != null && o.link == null) {
608      return false;
609    }
610    if (link == null && o.link != null) {
611      return false;
612    }
613    if (link != o.link && link != null && !link.equals(o.link)) {
614      return false;
615    }
616
617    return true;
618  };
619
620  @Override
621  public int hashCode() {
622    int hash = 17;
623    hash = hash * 31 + ((reference == null) ? 0 : reference.hashCode());
624    hash = hash * 31 + ((initialPath == null) ? 0 : initialPath.hashCode());
625    hash = hash * 31 + ((link == null) ? 0 : link.hashCode());
626    return hash;
627  }
628
629  /**
630   * Return the active file name that contains the real data.
631   * <p>
632   * For referenced hfile, we will return the name of the reference file as it will be used to
633   * construct the StoreFileReader. And for linked hfile, we will return the name of the file being
634   * linked.
635   */
636  public String getActiveFileName() {
637    if (reference != null || link == null) {
638      return initialPath.getName();
639    } else {
640      return HFileLink.getReferencedHFileName(initialPath.getName());
641    }
642  }
643
644  FileSystem getFileSystem() {
645    return this.fs;
646  }
647
648  boolean isNoReadahead() {
649    return this.noReadahead;
650  }
651
652  HFileInfo getHFileInfo() {
653    return hfileInfo;
654  }
655
656  void initHDFSBlocksDistribution() throws IOException {
657    hdfsBlocksDistribution = computeHDFSBlocksDistribution(fs);
658  }
659
660  StoreFileReader preStoreFileReaderOpen(ReaderContext context, CacheConfig cacheConf)
661    throws IOException {
662    StoreFileReader reader = null;
663    if (this.coprocessorHost != null) {
664      reader = this.coprocessorHost.preStoreFileReaderOpen(fs, this.getPath(),
665        context.getInputStreamWrapper(), context.getFileSize(), cacheConf, reference);
666    }
667    return reader;
668  }
669
670  StoreFileReader postStoreFileReaderOpen(ReaderContext context, CacheConfig cacheConf,
671    StoreFileReader reader) throws IOException {
672    StoreFileReader res = reader;
673    if (this.coprocessorHost != null) {
674      res = this.coprocessorHost.postStoreFileReaderOpen(fs, this.getPath(),
675        context.getInputStreamWrapper(), context.getFileSize(), cacheConf, reference, reader);
676    }
677    return res;
678  }
679
680  public void initHFileInfo(ReaderContext context) throws IOException {
681    this.hfileInfo = new HFileInfo(context, conf);
682  }
683
684}