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.hfile; 019 020import org.apache.hadoop.fs.Path; 021import org.apache.hadoop.hbase.HConstants; 022import org.apache.hadoop.hbase.fs.HFileSystem; 023import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; 024import org.apache.hadoop.hbase.keymeta.ManagedKeyDataCache; 025import org.apache.hadoop.hbase.keymeta.SystemKeyCache; 026import org.apache.yetus.audience.InterfaceAudience; 027 028/** 029 * Carries the information on some of the meta data about the HFile Reader 030 */ 031@InterfaceAudience.Private 032public class ReaderContext { 033 @InterfaceAudience.Private 034 public enum ReaderType { 035 PREAD, 036 STREAM 037 } 038 039 private final Path filePath; 040 private final FSDataInputStreamWrapper fsdis; 041 private final long fileSize; 042 private final HFileSystem hfs; 043 private final boolean primaryReplicaReader; 044 private final ReaderType type; 045 private final boolean preadAllBytes; 046 private final SystemKeyCache systemKeyCache; 047 private final ManagedKeyDataCache managedKeyDataCache; 048 049 public ReaderContext(Path filePath, FSDataInputStreamWrapper fsdis, long fileSize, 050 HFileSystem hfs, boolean primaryReplicaReader, ReaderType type, SystemKeyCache systemKeyCache, 051 ManagedKeyDataCache managedKeyDataCache) { 052 this.filePath = filePath; 053 this.fsdis = fsdis; 054 this.fileSize = fileSize; 055 this.hfs = hfs; 056 this.primaryReplicaReader = primaryReplicaReader; 057 this.type = type; 058 this.preadAllBytes = hfs.getConf().getBoolean(HConstants.HFILE_PREAD_ALL_BYTES_ENABLED_KEY, 059 HConstants.HFILE_PREAD_ALL_BYTES_ENABLED_DEFAULT); 060 this.systemKeyCache = systemKeyCache; 061 this.managedKeyDataCache = managedKeyDataCache; 062 } 063 064 public Path getFilePath() { 065 return this.filePath; 066 } 067 068 public FSDataInputStreamWrapper getInputStreamWrapper() { 069 return this.fsdis; 070 } 071 072 public long getFileSize() { 073 return this.fileSize; 074 } 075 076 public HFileSystem getFileSystem() { 077 return this.hfs; 078 } 079 080 public boolean isPrimaryReplicaReader() { 081 return this.primaryReplicaReader; 082 } 083 084 public ReaderType getReaderType() { 085 return this.type; 086 } 087 088 public boolean isPreadAllBytes() { 089 return preadAllBytes; 090 } 091 092 public SystemKeyCache getSystemKeyCache() { 093 return this.systemKeyCache; 094 } 095 096 public ManagedKeyDataCache getManagedKeyDataCache() { 097 return this.managedKeyDataCache; 098 } 099}