Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -436,7 +436,6 @@ byte[][] generateBoundaryKeys(ArrayList<Path> regionDirList) throws IOException
HFile.Reader reader = HFile.createReader(fs, hfile, conf);
final byte[] first, last;
try {
reader.loadFileInfo();
first = reader.getFirstRowKey().get();
last = reader.getLastRowKey().get();
LOG.debug("Trying to figure out region boundaries hfile=" + hfile + " first="
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,7 +91,6 @@ public void initialize(InputSplit split, TaskAttemptContext context)

// The file info must be loaded before the scanner can be used.
// This seems like a bug in HBase, but it's easily worked around.
this.in.loadFileInfo();
this.scanner = in.getScanner(false, false);

}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -401,7 +401,7 @@ public void test_TIMERANGE() throws Exception {
// open as HFile Reader and pull out TIMERANGE FileInfo.
HFile.Reader rd =
HFile.createReader(fs, file[0].getPath(), new CacheConfig(conf), true, conf);
Map<byte[],byte[]> finfo = rd.loadFileInfo();
Map<byte[],byte[]> finfo = rd.getHFileInfo();
byte[] range = finfo.get(Bytes.toBytes("TIMERANGE"));
assertNotNull(range);

Expand Down Expand Up @@ -1188,7 +1188,7 @@ public void testColumnFamilySettings() throws Exception {
// compression
Path dataFilePath = fs.listStatus(f.getPath())[0].getPath();
Reader reader = HFile.createReader(fs, dataFilePath, new CacheConfig(conf), true, conf);
Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
Map<byte[], byte[]> fileInfo = reader.getHFileInfo();

byte[] bloomFilter = fileInfo.get(BLOOM_FILTER_TYPE_KEY);
if (bloomFilter == null) bloomFilter = Bytes.toBytes("NONE");
Expand Down Expand Up @@ -1601,7 +1601,8 @@ public void TestConfigureCompression() throws Exception {
LocatedFileStatus keyFileStatus = iterator.next();
HFile.Reader reader =
HFile.createReader(fs, keyFileStatus.getPath(), new CacheConfig(conf), true, conf);
assertEquals(reader.getCompressionAlgorithm().getName(), hfileoutputformatCompression);
assertEquals(reader.getTrailer().getCompressionCodec().getName(),
hfileoutputformatCompression);
}
} finally {
if (writer != null && context != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -484,7 +484,6 @@ private static void validateTable(Configuration conf, TableName tableName, Strin
private static int getKVCountFromHfile(FileSystem fs, Path p) throws IOException {
Configuration conf = util.getConfiguration();
HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), true, conf);
reader.loadFileInfo();
HFileScanner scanner = reader.getScanner(false, false);
scanner.seekTo();
int count = 0;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -561,7 +561,6 @@ private static void validateHFiles(FileSystem fs, String outputPath, String fami
private static int getKVCountFromHfile(FileSystem fs, Path p) throws IOException {
Configuration conf = util.getConfiguration();
HFile.Reader reader = HFile.createReader(fs, p, new CacheConfig(conf), true, conf);
reader.loadFileInfo();
HFileScanner scanner = reader.getScanner(false, false);
scanner.seekTo();
int count = 0;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,21 +23,21 @@
import java.util.Optional;
import java.util.concurrent.atomic.AtomicInteger;

import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.PrivateCellUtil;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFileInfo;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.io.hfile.ReaderContext;
import org.apache.hadoop.hbase.regionserver.StoreFileReader;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* A facade for a {@link org.apache.hadoop.hbase.io.hfile.HFile.Reader} that serves up
Expand Down Expand Up @@ -66,45 +66,19 @@ public class HalfStoreFileReader extends StoreFileReader {

private boolean firstKeySeeked = false;

/**
* Creates a half file reader for a normal hfile.
* @param fs fileystem to read from
* @param p path to hfile
* @param cacheConf
* @param r original reference file (contains top or bottom)
* @param conf Configuration
* @throws IOException
*/
public HalfStoreFileReader(FileSystem fs, Path p, CacheConfig cacheConf, Reference r,
boolean isPrimaryReplicaStoreFile, AtomicInteger refCount, boolean shared, Configuration conf)
throws IOException {
super(fs, p, cacheConf, isPrimaryReplicaStoreFile, refCount, shared, conf);
// This is not actual midkey for this half-file; its just border
// around which we split top and bottom. Have to look in files to find
// actual last and first keys for bottom and top halves. Half-files don't
// have an actual midkey themselves. No midkey is how we indicate file is
// not splittable.
this.splitkey = r.getSplitKey();
this.splitCell = new KeyValue.KeyOnlyKeyValue(this.splitkey, 0, this.splitkey.length);
// Is it top or bottom half?
this.top = Reference.isTopFileRegion(r.getFileRegion());
}

/**
* Creates a half file reader for a hfile referred to by an hfilelink.
* @param fs fileystem to read from
* @param p path to hfile
* @param in {@link FSDataInputStreamWrapper}
* @param size Full size of the hfile file
* @param cacheConf
* @param context Reader context info
* @param fileInfo HFile info
* @param cacheConf CacheConfig
* @param r original reference file (contains top or bottom)
* @param refCount reference count
* @param conf Configuration
* @throws IOException
*/
public HalfStoreFileReader(final FileSystem fs, final Path p, final FSDataInputStreamWrapper in,
long size, final CacheConfig cacheConf, final Reference r, boolean isPrimaryReplicaStoreFile,
AtomicInteger refCount, boolean shared, final Configuration conf) throws IOException {
super(fs, p, in, size, cacheConf, isPrimaryReplicaStoreFile, refCount, shared, conf);
public HalfStoreFileReader(final ReaderContext context, final HFileInfo fileInfo,
final CacheConfig cacheConf, final Reference r,
AtomicInteger refCount, final Configuration conf) throws IOException {
super(context, fileInfo, cacheConf, refCount, conf);
// This is not actual midkey for this half-file; its just border
// around which we split top and bottom. Have to look in files to find
// actual last and first keys for bottom and top halves. Half-files don't
Expand Down
Loading