HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface

This commit is contained in:
zhangduo 2017-09-24 19:22:16 +08:00
parent 1540483135
commit a5f84430a3
142 changed files with 2049 additions and 2276 deletions

View File

@ -440,8 +440,8 @@ public class RestoreTool {
final byte[] first, last;
try {
reader.loadFileInfo();
first = reader.getFirstRowKey();
last = reader.getLastRowKey();
first = reader.getFirstRowKey().get();
last = reader.getLastRowKey().get();
LOG.debug("Trying to figure out region boundaries hfile=" + hfile + " first="
+ Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last));

View File

@ -40,7 +40,6 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreScanner;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.zookeeper.KeeperException;
@ -130,32 +129,32 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
@Override
public void process(WatchedEvent event) {
switch(event.getType()) {
case NodeDataChanged:
case NodeCreated:
try {
// get data and re-watch
data = zk.getData(node, this, null);
LOG.debug("Read asynchronously: "+(data == null ? "null" : Bytes.toLong(data)));
} catch (InterruptedException ix) {
} catch (KeeperException kx) {
needSetup = true;
}
break;
switch (event.getType()) {
case NodeDataChanged:
case NodeCreated:
try {
// get data and re-watch
data = zk.getData(node, this, null);
LOG.debug("Read asynchronously: " + (data == null ? "null" : Bytes.toLong(data)));
} catch (InterruptedException ix) {
} catch (KeeperException kx) {
needSetup = true;
}
break;
case NodeDeleted:
try {
// just re-watch
zk.exists(node, this);
data = null;
} catch (InterruptedException ix) {
} catch (KeeperException kx) {
needSetup = true;
}
break;
case NodeDeleted:
try {
// just re-watch
zk.exists(node, this);
data = null;
} catch (InterruptedException ix) {
} catch (KeeperException kx) {
needSetup = true;
}
break;
default:
// ignore
default:
// ignore
}
}
}
@ -166,15 +165,13 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
if (!re.getSharedData().containsKey(zkkey)) {
// there is a short race here
// in the worst case we create a watcher that will be notified once
re.getSharedData().putIfAbsent(
zkkey,
new ZKWatcher(re.getRegionServerServices().getZooKeeper()
.getRecoverableZooKeeper().getZooKeeper()));
re.getSharedData().putIfAbsent(zkkey, new ZKWatcher(
re.getRegionServerServices().getZooKeeper().getRecoverableZooKeeper().getZooKeeper()));
}
}
protected ScanInfo getScanInfo(Store store, RegionCoprocessorEnvironment e) {
byte[] data = ((ZKWatcher)e.getSharedData().get(zkkey)).getData();
byte[] data = ((ZKWatcher) e.getSharedData().get(zkkey)).getData();
if (data == null) {
return null;
}
@ -182,8 +179,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
if (oldSI.getTtl() == Long.MAX_VALUE) {
return null;
}
long ttl = Math.max(EnvironmentEdgeManager.currentTime() -
Bytes.toLong(data), oldSI.getTtl());
long ttl = Math.max(EnvironmentEdgeManager.currentTime() - Bytes.toLong(data), oldSI.getTtl());
return new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(), ttl,
oldSI.getTimeToPurgeDeletes(), oldSI.getComparator());
}
@ -197,7 +193,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
// take default action
return null;
}
return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners,
return new StoreScanner((HStore) store, scanInfo, OptionalInt.empty(), scanners,
ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP);
}
@ -210,7 +206,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
// take default action
return null;
}
return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners, scanType,
return new StoreScanner((HStore) store, scanInfo, OptionalInt.empty(), scanners, scanType,
store.getSmallestReadPoint(), earliestPutTs);
}
@ -223,7 +219,7 @@ public class ZooKeeperScanPolicyObserver implements RegionObserver {
// take default action
return null;
}
return new StoreScanner(store, scanInfo, scan, targetCols,
((HStore)store).getHRegion().getReadPoint(IsolationLevel.READ_COMMITTED));
return new StoreScanner((HStore) store, scanInfo, scan, targetCols,
((HStore) store).getHRegion().getReadPoint(IsolationLevel.READ_COMMITTED));
}
}

View File

@ -17,6 +17,11 @@
*/
package org.apache.hadoop.hbase.mapreduce;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TASK_KEY;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.net.InetSocketAddress;
@ -43,8 +48,12 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
@ -52,10 +61,8 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
@ -65,13 +72,9 @@ import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.io.hfile.HFileWriterImpl;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
@ -85,6 +88,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.apache.hadoop.mapreduce.lib.partition.TotalOrderPartitioner;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
@ -413,13 +417,13 @@ public class HFileOutputFormat2
private void close(final StoreFileWriter w) throws IOException {
if (w != null) {
w.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY,
w.appendFileInfo(BULKLOAD_TIME_KEY,
Bytes.toBytes(System.currentTimeMillis()));
w.appendFileInfo(StoreFile.BULKLOAD_TASK_KEY,
w.appendFileInfo(BULKLOAD_TASK_KEY,
Bytes.toBytes(context.getTaskAttemptID().toString()));
w.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY,
w.appendFileInfo(MAJOR_COMPACTION_KEY,
Bytes.toBytes(true));
w.appendFileInfo(StoreFile.EXCLUDE_FROM_MINOR_COMPACTION_KEY,
w.appendFileInfo(EXCLUDE_FROM_MINOR_COMPACTION_KEY,
Bytes.toBytes(compactionExclude));
w.appendTrackedTimestampsToMetadata();
w.close();

View File

@ -38,7 +38,6 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.mapreduce.JobUtil;
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
@ -62,6 +61,7 @@ import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
import org.apache.hadoop.util.LineReader;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.yetus.audience.InterfaceAudience;
/*
* The CompactionTool allows to execute a compaction specifying a:
@ -165,11 +165,11 @@ public class CompactionTool extends Configured implements Tool {
if (!compaction.isPresent()) {
break;
}
List<StoreFile> storeFiles =
List<HStoreFile> storeFiles =
store.compact(compaction.get(), NoLimitThroughputController.INSTANCE);
if (storeFiles != null && !storeFiles.isEmpty()) {
if (keepCompactedFiles && deleteCompacted) {
for (StoreFile storeFile: storeFiles) {
for (HStoreFile storeFile: storeFiles) {
fs.delete(storeFile.getPath(), false);
}
}

View File

@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase.mapreduce;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
@ -85,7 +86,6 @@ import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.TestHRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.TimeRangeTracker;
import org.apache.hadoop.hbase.testclassification.LargeTests;
@ -1162,7 +1162,7 @@ public class TestHFileOutputFormat2 {
Reader reader = HFile.createReader(fs, dataFilePath, new CacheConfig(conf), true, conf);
Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
byte[] bloomFilter = fileInfo.get(StoreFile.BLOOM_FILTER_TYPE_KEY);
byte[] bloomFilter = fileInfo.get(BLOOM_FILTER_TYPE_KEY);
if (bloomFilter == null) bloomFilter = Bytes.toBytes("NONE");
assertEquals("Incorrect bloom filter used for column family " + familyStr +
"(reader: " + reader + ")",

View File

@ -33,14 +33,14 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.io.MultipleIOException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Function;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Collections2;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
/**
* Utility class to handle the removal of HFiles (or the respective {@link StoreFile StoreFiles})
* Utility class to handle the removal of HFiles (or the respective {@link HStoreFile StoreFiles})
* for a HRegion from the {@link FileSystem}. The hfiles will be archived or deleted, depending on
* the state of the system.
*/
@ -226,7 +226,7 @@ public class HFileArchiver {
* @throws IOException if the files could not be correctly disposed.
*/
public static void archiveStoreFiles(Configuration conf, FileSystem fs, HRegionInfo regionInfo,
Path tableDir, byte[] family, Collection<StoreFile> compactedFiles)
Path tableDir, byte[] family, Collection<HStoreFile> compactedFiles)
throws IOException, FailedArchiveException {
// sometimes in testing, we don't have rss, so we need to check for that
@ -479,13 +479,13 @@ public class HFileArchiver {
* @throws IOException if a file cannot be deleted. All files will be attempted to deleted before
* throwing the exception, rather than failing at the first file.
*/
private static void deleteStoreFilesWithoutArchiving(Collection<StoreFile> compactedFiles)
private static void deleteStoreFilesWithoutArchiving(Collection<HStoreFile> compactedFiles)
throws IOException {
LOG.debug("Deleting store files without archiving.");
List<IOException> errors = new ArrayList<>(0);
for (StoreFile hsf : compactedFiles) {
for (HStoreFile hsf : compactedFiles) {
try {
hsf.deleteReader();
hsf.deleteStoreFile();
} catch (IOException e) {
LOG.error("Failed to delete store file:" + hsf.getPath());
errors.add(e);
@ -524,16 +524,16 @@ public class HFileArchiver {
}
/**
* Convert the {@link StoreFile} into something we can manage in the archive
* Convert the {@link HStoreFile} into something we can manage in the archive
* methods
*/
private static class StoreToFile extends FileConverter<StoreFile> {
private static class StoreToFile extends FileConverter<HStoreFile> {
public StoreToFile(FileSystem fs) {
super(fs);
}
@Override
public File apply(StoreFile input) {
public File apply(HStoreFile input) {
return new FileableStoreFile(fs, input);
}
}
@ -656,20 +656,20 @@ public class HFileArchiver {
}
/**
* {@link File} adapter for a {@link StoreFile} living on a {@link FileSystem}
* {@link File} adapter for a {@link HStoreFile} living on a {@link FileSystem}
* .
*/
private static class FileableStoreFile extends File {
StoreFile file;
HStoreFile file;
public FileableStoreFile(FileSystem fs, StoreFile store) {
public FileableStoreFile(FileSystem fs, HStoreFile store) {
super(fs);
this.file = store;
}
@Override
public void delete() throws IOException {
file.deleteReader();
file.deleteStoreFile();
}
@Override
@ -690,7 +690,7 @@ public class HFileArchiver {
@Override
public void close() throws IOException {
file.closeReader(true);
file.closeStoreFile(true);
}
@Override

View File

@ -189,7 +189,7 @@ public interface RegionObserver extends Coprocessor {
* @param tracker tracker used to track the life cycle of a compaction
*/
default void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
List<StoreFile> candidates, CompactionLifeCycleTracker tracker) throws IOException {}
List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker) throws IOException {}
/**
* Called after the {@link StoreFile}s to compact have been selected from the available
@ -200,7 +200,7 @@ public interface RegionObserver extends Coprocessor {
* @param tracker tracker used to track the life cycle of a compaction
*/
default void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
ImmutableList<StoreFile> selected, CompactionLifeCycleTracker tracker) {}
ImmutableList<? extends StoreFile> selected, CompactionLifeCycleTracker tracker) {}
/**
* Called prior to writing the {@link StoreFile}s selected for compaction into a new

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.io;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
@ -48,7 +49,7 @@ import org.apache.hadoop.hbase.util.Bytes;
* <p>This type works in tandem with the {@link Reference} type. This class
* is used reading while Reference is used writing.
*
* <p>This file is not splitable. Calls to {@link #midkey()} return null.
* <p>This file is not splitable. Calls to {@link #midKey()} return null.
*/
@InterfaceAudience.Private
public class HalfStoreFileReader extends StoreFileReader {
@ -60,7 +61,7 @@ public class HalfStoreFileReader extends StoreFileReader {
protected final Cell splitCell;
private Cell firstKey = null;
private Optional<Cell> firstKey = null;
private boolean firstKeySeeked = false;
@ -258,8 +259,8 @@ public class HalfStoreFileReader extends StoreFileReader {
@Override
public boolean seekBefore(Cell key) throws IOException {
if (top) {
Cell fk = getFirstKey();
if (getComparator().compareKeyIgnoresMvcc(key, fk) <= 0) {
Optional<Cell> fk = getFirstKey();
if (getComparator().compareKeyIgnoresMvcc(key, fk.get()) <= 0) {
return false;
}
} else {
@ -303,7 +304,7 @@ public class HalfStoreFileReader extends StoreFileReader {
}
@Override
public Cell getLastKey() {
public Optional<Cell> getLastKey() {
if (top) {
return super.getLastKey();
}
@ -311,7 +312,7 @@ public class HalfStoreFileReader extends StoreFileReader {
HFileScanner scanner = getScanner(true, true);
try {
if (scanner.seekBefore(this.splitCell)) {
return scanner.getKey();
return Optional.ofNullable(scanner.getKey());
}
} catch (IOException e) {
LOG.warn("Failed seekBefore " + Bytes.toStringBinary(this.splitkey), e);
@ -320,22 +321,22 @@ public class HalfStoreFileReader extends StoreFileReader {
scanner.close();
}
}
return null;
return Optional.empty();
}
@Override
public Cell midkey() throws IOException {
public Optional<Cell> midKey() throws IOException {
// Returns null to indicate file is not splitable.
return null;
return Optional.empty();
}
@Override
public Cell getFirstKey() {
public Optional<Cell> getFirstKey() {
if (!firstKeySeeked) {
HFileScanner scanner = getScanner(true, true, false);
try {
if (scanner.seekTo()) {
this.firstKey = scanner.getKey();
this.firstKey = Optional.ofNullable(scanner.getKey());
}
firstKeySeeked = true;
} catch (IOException e) {

View File

@ -31,6 +31,7 @@ import java.util.Collection;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
@ -434,21 +435,21 @@ public class HFile {
Map<byte[], byte[]> loadFileInfo() throws IOException;
Cell getLastKey();
Optional<Cell> getLastKey();
Cell midkey() throws IOException;
Optional<Cell> midKey() throws IOException;
long length();
long getEntries();
Cell getFirstKey();
Optional<Cell> getFirstKey();
long indexSize();
byte[] getFirstRowKey();
Optional<byte[]> getFirstRowKey();
byte[] getLastRowKey();
Optional<byte[]> getLastRowKey();
FixedFileTrailer getTrailer();

View File

@ -544,7 +544,7 @@ public class HFilePrettyPrinter extends Configured implements Tool {
}
try {
out.println("Mid-key: " + (CellUtil.getCellKeyAsString(reader.midkey())));
out.println("Mid-key: " + reader.midKey().map(CellUtil::getCellKeyAsString));
} catch (Exception e) {
out.println ("Unable to retrieve the midkey");
}

View File

@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
import java.security.Key;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
@ -338,14 +339,12 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
}
}
private String toStringFirstKey() {
if(getFirstKey() == null)
return null;
return CellUtil.getCellKeyAsString(getFirstKey());
private Optional<String> toStringFirstKey() {
return getFirstKey().map(CellUtil::getCellKeyAsString);
}
private String toStringLastKey() {
return CellUtil.toString(getLastKey(), false);
private Optional<String> toStringLastKey() {
return getLastKey().map(CellUtil::getCellKeyAsString);
}
@Override
@ -382,12 +381,12 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
* first KeyValue.
*/
@Override
public Cell getFirstKey() {
public Optional<Cell> getFirstKey() {
if (dataBlockIndexReader == null) {
throw new BlockIndexNotLoadedException();
}
return dataBlockIndexReader.isEmpty() ? null
: dataBlockIndexReader.getRootBlockKey(0);
return dataBlockIndexReader.isEmpty() ? Optional.empty()
: Optional.of(dataBlockIndexReader.getRootBlockKey(0));
}
/**
@ -397,10 +396,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
* @return the first row key, or null if the file is empty.
*/
@Override
public byte[] getFirstRowKey() {
Cell firstKey = getFirstKey();
public Optional<byte[]> getFirstRowKey() {
// We have to copy the row part to form the row key alone
return firstKey == null? null: CellUtil.cloneRow(firstKey);
return getFirstKey().map(CellUtil::cloneRow);
}
/**
@ -410,9 +408,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
* @return the last row key, or null if the file is empty.
*/
@Override
public byte[] getLastRowKey() {
Cell lastKey = getLastKey();
return lastKey == null? null: CellUtil.cloneRow(lastKey);
public Optional<byte[]> getLastRowKey() {
// We have to copy the row part to form the row key alone
return getLastKey().map(CellUtil::cloneRow);
}
/** @return number of KV entries in this HFile */
@ -1550,8 +1548,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
* key
*/
@Override
public Cell getLastKey() {
return dataBlockIndexReader.isEmpty() ? null : lastKeyCell;
public Optional<Cell> getLastKey() {
return dataBlockIndexReader.isEmpty() ? Optional.empty() : Optional.of(lastKeyCell);
}
/**
@ -1560,8 +1558,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
* @throws IOException
*/
@Override
public Cell midkey() throws IOException {
return dataBlockIndexReader.midkey();
public Optional<Cell> midKey() throws IOException {
return Optional.ofNullable(dataBlockIndexReader.midkey());
}
@Override

View File

@ -18,12 +18,14 @@
package org.apache.hadoop.hbase.master.assignment;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.*;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
@ -41,7 +43,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
@ -58,19 +60,21 @@ import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
import org.apache.hadoop.hbase.quotas.QuotaExceededException;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.SplitTableRegionState;
/**
* The procedure to split a region in a table.
@ -636,8 +640,8 @@ public class SplitTableRegionProcedure
}
}
private Pair<Path, Path> splitStoreFile(final HRegionFileSystem regionFs,
final byte[] family, final StoreFile sf) throws IOException {
private Pair<Path, Path> splitStoreFile(HRegionFileSystem regionFs, byte[] family, HStoreFile sf)
throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("pid=" + getProcId() + " splitting started for store file: " +
sf.getPath() + " for region: " + getParentRegion().getShortNameToLog());
@ -663,7 +667,7 @@ public class SplitTableRegionProcedure
private class StoreFileSplitter implements Callable<Pair<Path,Path>> {
private final HRegionFileSystem regionFs;
private final byte[] family;
private final StoreFile sf;
private final HStoreFile sf;
/**
* Constructor that takes what it needs to split
@ -671,8 +675,7 @@ public class SplitTableRegionProcedure
* @param family Family that contains the store file
* @param sf which file
*/
public StoreFileSplitter(final HRegionFileSystem regionFs, final byte[] family,
final StoreFile sf) {
public StoreFileSplitter(HRegionFileSystem regionFs, byte[] family, HStoreFile sf) {
this.regionFs = regionFs;
this.sf = sf;
this.family = family;

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.master.balancer;
import com.google.common.annotations.VisibleForTesting;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Arrays;
@ -42,7 +43,6 @@ import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerLoad;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.Action;
@ -53,6 +53,7 @@ import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.MoveRegi
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.SwapRegionsAction;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Optional;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
@ -1241,7 +1242,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
/**
* Compute a cost of a potential cluster configuration based upon where
* {@link org.apache.hadoop.hbase.regionserver.StoreFile}s are located.
* {@link org.apache.hadoop.hbase.regionserver.HStoreFile}s are located.
*/
static abstract class LocalityBasedCostFunction extends CostFunction {

View File

@ -24,11 +24,10 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Cached mob file.
@ -39,7 +38,7 @@ public class CachedMobFile extends MobFile implements Comparable<CachedMobFile>
private long accessCount;
private AtomicLong referenceCount = new AtomicLong(0);
public CachedMobFile(StoreFile sf) {
public CachedMobFile(HStoreFile sf) {
super(sf);
}
@ -47,7 +46,7 @@ public class CachedMobFile extends MobFile implements Comparable<CachedMobFile>
CacheConfig cacheConf) throws IOException {
// XXX: primaryReplica is only used for constructing the key of block cache so it is not a
// critical problem if we pass the wrong value, so here we always pass true. Need to fix later.
StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
HStoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
return new CachedMobFile(sf);
}

View File

@ -91,7 +91,7 @@ public class DefaultMobStoreCompactor extends DefaultCompactor {
}
};
public DefaultMobStoreCompactor(Configuration conf, Store store) {
public DefaultMobStoreCompactor(Configuration conf, HStore store) {
super(conf, store);
// The mob cells reside in the mob-enabled column family which is held by HMobStore.
// During the compaction, the compactor reads the cells from the mob files and

View File

@ -32,19 +32,19 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.regionserver.DefaultStoreFlusher;
import org.apache.hadoop.hbase.regionserver.HMobStore;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot;
import org.apache.hadoop.hbase.regionserver.ScannerContext;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* An implementation of the StoreFlusher. It extends the DefaultStoreFlusher.
@ -70,8 +70,11 @@ public class DefaultMobStoreFlusher extends DefaultStoreFlusher {
private Path targetPath;
private HMobStore mobStore;
public DefaultMobStoreFlusher(Configuration conf, Store store) throws IOException {
public DefaultMobStoreFlusher(Configuration conf, HStore store) throws IOException {
super(conf, store);
if (!(store instanceof HMobStore)) {
throw new IllegalArgumentException("The store " + store + " is not a HMobStore");
}
mobCellValueSizeThreshold = store.getColumnFamilyDescriptor().getMobThreshold();
this.targetPath = MobUtils.getMobFamilyPath(conf, store.getTableName(),
store.getColumnFamilyName());

View File

@ -26,12 +26,11 @@ 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.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
import org.apache.yetus.audience.InterfaceAudience;
/**
* The mob file.
@ -39,13 +38,13 @@ import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
@InterfaceAudience.Private
public class MobFile {
private StoreFile sf;
private HStoreFile sf;
// internal use only for sub classes
protected MobFile() {
}
protected MobFile(StoreFile sf) {
protected MobFile(HStoreFile sf) {
this.sf = sf;
}
@ -56,7 +55,7 @@ public class MobFile {
* @throws IOException
*/
public StoreFileScanner getScanner() throws IOException {
List<StoreFile> sfs = new ArrayList<>();
List<HStoreFile> sfs = new ArrayList<>();
sfs.add(sf);
List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true,
false, false, sf.getMaxMemstoreTS());
@ -86,7 +85,7 @@ public class MobFile {
public Cell readCell(Cell search, boolean cacheMobBlocks, long readPt) throws IOException {
Cell result = null;
StoreFileScanner scanner = null;
List<StoreFile> sfs = new ArrayList<>();
List<HStoreFile> sfs = new ArrayList<>();
sfs.add(sf);
try {
List<StoreFileScanner> sfScanners = StoreFileScanner.getScannersForStoreFiles(sfs,
@ -129,7 +128,7 @@ public class MobFile {
*/
public void close() throws IOException {
if (sf != null) {
sf.closeReader(false);
sf.closeStoreFile(false);
sf = null;
}
}
@ -147,7 +146,7 @@ public class MobFile {
throws IOException {
// XXX: primaryReplica is only used for constructing the key of block cache so it is not a
// critical problem if we pass the wrong value, so here we always pass true. Need to fix later.
StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
HStoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
return new MobFile(sf);
}
}

View File

@ -20,9 +20,9 @@ package org.apache.hadoop.hbase.mob;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.yetus.audience.InterfaceAudience;
/**
* MobStoreEngine creates the mob specific compactor, and store flusher.
@ -31,7 +31,7 @@ import org.apache.hadoop.hbase.regionserver.Store;
public class MobStoreEngine extends DefaultStoreEngine {
@Override
protected void createStoreFlusher(Configuration conf, Store store) throws IOException {
protected void createStoreFlusher(Configuration conf, HStore store) throws IOException {
// When using MOB, we use DefaultMobStoreFlusher always
// Just use the compactor and compaction policy as that in DefaultStoreEngine. We can have MOB
// specific compactor and policy when that is implemented.
@ -42,7 +42,7 @@ public class MobStoreEngine extends DefaultStoreEngine {
* Creates the DefaultMobCompactor.
*/
@Override
protected void createCompactor(Configuration conf, Store store) throws IOException {
protected void createCompactor(Configuration conf, HStore store) throws IOException {
compactor = new DefaultMobStoreCompactor(conf, store);
}
}

View File

@ -51,7 +51,6 @@ import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.TagType;
import org.apache.hadoop.hbase.TagUtil;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
import org.apache.hadoop.hbase.client.Scan;
@ -70,7 +69,6 @@ import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactor;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ChecksumType;
@ -78,6 +76,7 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
/**
* The mob utilities
@ -315,7 +314,7 @@ public final class MobUtils {
// no file found
return;
}
List<StoreFile> filesToClean = new ArrayList<>();
List<HStoreFile> filesToClean = new ArrayList<>();
int deletedFileCount = 0;
for (FileStatus file : stats) {
String fileName = file.getPath().getName();
@ -467,7 +466,7 @@ public final class MobUtils {
* @throws IOException
*/
public static void removeMobFiles(Configuration conf, FileSystem fs, TableName tableName,
Path tableDir, byte[] family, Collection<StoreFile> storeFiles) throws IOException {
Path tableDir, byte[] family, Collection<HStoreFile> storeFiles) throws IOException {
HFileArchiver.archiveStoreFiles(conf, fs, getMobRegionInfo(tableName), tableDir, family,
storeFiles);
}
@ -721,7 +720,7 @@ public final class MobUtils {
*/
private static void validateMobFile(Configuration conf, FileSystem fs, Path path,
CacheConfig cacheConfig, boolean primaryReplica) throws IOException {
StoreFile storeFile = null;
HStoreFile storeFile = null;
try {
storeFile = new HStoreFile(fs, path, conf, cacheConfig, BloomType.NONE, primaryReplica);
storeFile.initReader();
@ -730,7 +729,7 @@ public final class MobUtils {
throw e;
} finally {
if (storeFile != null) {
storeFile.closeReader(false);
storeFile.closeStoreFile(false);
}
}
}

View File

@ -25,10 +25,10 @@ import java.util.List;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
/**
* An implementation of {@link MobCompactionRequest} that is used in
@ -106,8 +106,7 @@ public class PartitionedMobCompactionRequest extends MobCompactionRequest {
* Set start key of this partition, only if the input startKey is less than
* the current start key.
*/
public void setStartKey(final byte[] startKey)
{
public void setStartKey(final byte[] startKey) {
if ((this.startKey == null) || (Bytes.compareTo(startKey, this.startKey) < 0)) {
this.startKey = startKey;
}
@ -227,7 +226,7 @@ public class PartitionedMobCompactionRequest extends MobCompactionRequest {
*/
protected static class CompactionDelPartition {
private List<Path> delFiles = new ArrayList<Path>();
private List<StoreFile> storeFiles = new ArrayList<>();
private List<HStoreFile> storeFiles = new ArrayList<>();
private CompactionDelPartitionId id;
public CompactionDelPartition(CompactionDelPartitionId id) {
@ -241,11 +240,11 @@ public class PartitionedMobCompactionRequest extends MobCompactionRequest {
void addDelFile(FileStatus file) {
delFiles.add(file.getPath());
}
public void addStoreFile(final StoreFile file) {
public void addStoreFile(HStoreFile file) {
storeFiles.add(file);
}
public List<StoreFile> getStoreFiles() {
public List<HStoreFile> getStoreFiles() {
return storeFiles;
}

View File

@ -18,6 +18,10 @@
*/
package org.apache.hadoop.hbase.mob.compactions;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.BULKLOAD_TIME_KEY;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.SKIP_RESET_SEQ_ID;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
@ -50,7 +54,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.TagType;
import org.apache.hadoop.hbase.TagUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
@ -75,7 +78,6 @@ import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.ScanInfo;
import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.ScannerContext;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
@ -85,6 +87,7 @@ import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
@ -225,8 +228,8 @@ public class PartitionedMobCompactor extends MobCompactor {
// Get delId from the file
try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) {
delId.setStartKey(reader.getFirstRowKey());
delId.setEndKey(reader.getLastRowKey());
delId.setStartKey(reader.getFirstRowKey().get());
delId.setEndKey(reader.getLastRowKey().get());
}
CompactionDelPartition delPartition = delFilesToCompact.get(delId);
if (delPartition == null) {
@ -266,8 +269,8 @@ public class PartitionedMobCompactor extends MobCompactor {
// get startKey and endKey from the file and update partition
// TODO: is it possible to skip read of most hfiles?
try (Reader reader = HFile.createReader(fs, linkedFile.getPath(), conf)) {
compactionPartition.setStartKey(reader.getFirstRowKey());
compactionPartition.setEndKey(reader.getLastRowKey());
compactionPartition.setStartKey(reader.getFirstRowKey().get());
compactionPartition.setEndKey(reader.getLastRowKey().get());
}
}
@ -335,7 +338,7 @@ public class PartitionedMobCompactor extends MobCompactor {
try {
for (CompactionDelPartition delPartition : request.getDelPartitions()) {
for (Path newDelPath : delPartition.listDelFiles()) {
StoreFile sf =
HStoreFile sf =
new HStoreFile(fs, newDelPath, conf, compactionCacheConfig, BloomType.NONE, true);
// pre-create reader of a del file to avoid race condition when opening the reader in each
// partition.
@ -361,7 +364,8 @@ public class PartitionedMobCompactor extends MobCompactor {
for (CompactionDelPartition delPartition : request.getDelPartitions()) {
LOG.info(delPartition.listDelFiles());
try {
MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(), delPartition.getStoreFiles());
MobUtils.removeMobFiles(conf, fs, tableName, mobTableDir, column.getName(),
delPartition.getStoreFiles());
} catch (IOException e) {
LOG.error("Failed to archive the del files " + delPartition.getStoreFiles(), e);
}
@ -397,11 +401,11 @@ public class PartitionedMobCompactor extends MobCompactor {
}
@VisibleForTesting
List<StoreFile> getListOfDelFilesForPartition(final CompactionPartition partition,
List<HStoreFile> getListOfDelFilesForPartition(final CompactionPartition partition,
final List<CompactionDelPartition> delPartitions) {
// Binary search for startKey and endKey
List<StoreFile> result = new ArrayList<>();
List<HStoreFile> result = new ArrayList<>();
DelPartitionComparator comparator = new DelPartitionComparator(false);
CompactionDelPartitionId id = new CompactionDelPartitionId(null, partition.getStartKey());
@ -473,7 +477,7 @@ public class PartitionedMobCompactor extends MobCompactor {
// Search the delPartitions and collect all the delFiles for the partition
// One optimization can do is that if there is no del file, we do not need to
// come up with startKey/endKey.
List<StoreFile> delFiles = getListOfDelFilesForPartition(partition,
List<HStoreFile> delFiles = getListOfDelFilesForPartition(partition,
request.getDelPartitions());
results.put(partition.getPartitionId(), pool.submit(new Callable<List<Path>>() {
@ -521,7 +525,7 @@ public class PartitionedMobCompactor extends MobCompactor {
*/
private List<Path> compactMobFilePartition(PartitionedMobCompactionRequest request,
CompactionPartition partition,
List<StoreFile> delFiles,
List<HStoreFile> delFiles,
Connection connection,
Table table) throws IOException {
if (MobUtils.isMobFileExpired(column, EnvironmentEdgeManager.currentTime(),
@ -550,9 +554,9 @@ public class PartitionedMobCompactor extends MobCompactor {
// clean the bulkload directory to avoid loading old files.
fs.delete(bulkloadPathOfPartition, true);
// add the selected mob files and del files into filesToCompact
List<StoreFile> filesToCompact = new ArrayList<>();
List<HStoreFile> filesToCompact = new ArrayList<>();
for (int i = offset; i < batch + offset; i++) {
StoreFile sf = new HStoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig,
HStoreFile sf = new HStoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig,
BloomType.NONE, true);
filesToCompact.add(sf);
}
@ -572,10 +576,10 @@ public class PartitionedMobCompactor extends MobCompactor {
* Closes the readers of store files.
* @param storeFiles The store files to be closed.
*/
private void closeStoreFileReaders(List<StoreFile> storeFiles) {
for (StoreFile storeFile : storeFiles) {
private void closeStoreFileReaders(List<HStoreFile> storeFiles) {
for (HStoreFile storeFile : storeFiles) {
try {
storeFile.closeReader(true);
storeFile.closeStoreFile(true);
} catch (IOException e) {
LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e);
}
@ -600,14 +604,14 @@ public class PartitionedMobCompactor extends MobCompactor {
private void compactMobFilesInBatch(PartitionedMobCompactionRequest request,
CompactionPartition partition,
Connection connection, Table table,
List<StoreFile> filesToCompact, int batch,
List<HStoreFile> filesToCompact, int batch,
Path bulkloadPathOfPartition, Path bulkloadColumnPath,
List<Path> newFiles)
throws IOException {
// open scanner to the selected mob files and del files.
StoreScanner scanner = createScanner(filesToCompact, ScanType.COMPACT_DROP_DELETES);
// the mob files to be compacted, not include the del files.
List<StoreFile> mobFilesToCompact = filesToCompact.subList(0, batch);
List<HStoreFile> mobFilesToCompact = filesToCompact.subList(0, batch);
// Pair(maxSeqId, cellsCount)
Pair<Long, Long> fileInfo = getFileInfo(mobFilesToCompact);
// open writers for the mob files and new ref store files.
@ -726,7 +730,7 @@ public class PartitionedMobCompactor extends MobCompactor {
if (delFilePaths.size() - offset < compactionBatchSize) {
batch = delFilePaths.size() - offset;
}
List<StoreFile> batchedDelFiles = new ArrayList<>();
List<HStoreFile> batchedDelFiles = new ArrayList<>();
if (batch == 1) {
// only one file left, do not compact it, directly add it to the new files.
paths.add(delFilePaths.get(offset));
@ -753,7 +757,7 @@ public class PartitionedMobCompactor extends MobCompactor {
* @throws IOException if IO failure is encountered
*/
private Path compactDelFilesInBatch(PartitionedMobCompactionRequest request,
List<StoreFile> delFiles) throws IOException {
List<HStoreFile> delFiles) throws IOException {
// create a scanner for the del files.
StoreScanner scanner = createScanner(delFiles, ScanType.COMPACT_RETAIN_DELETES);
StoreFileWriter writer = null;
@ -803,7 +807,7 @@ public class PartitionedMobCompactor extends MobCompactor {
* @return The store scanner.
* @throws IOException if IO failure is encountered
*/
private StoreScanner createScanner(List<StoreFile> filesToCompact, ScanType scanType)
private StoreScanner createScanner(List<HStoreFile> filesToCompact, ScanType scanType)
throws IOException {
List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(filesToCompact,
false, true, false, false, HConstants.LATEST_TIMESTAMP);
@ -864,8 +868,8 @@ public class PartitionedMobCompactor extends MobCompactor {
throws IOException {
if (writer != null) {
writer.appendMetadata(maxSeqId, false);
writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(bulkloadTime));
writer.appendFileInfo(StoreFile.SKIP_RESET_SEQ_ID, Bytes.toBytes(true));
writer.appendFileInfo(BULKLOAD_TIME_KEY, Bytes.toBytes(bulkloadTime));
writer.appendFileInfo(SKIP_RESET_SEQ_ID, Bytes.toBytes(true));
try {
writer.close();
} catch (IOException e) {
@ -880,14 +884,14 @@ public class PartitionedMobCompactor extends MobCompactor {
* @return The pair of the max seqId and number of cells of the store files.
* @throws IOException if IO failure is encountered
*/
private Pair<Long, Long> getFileInfo(List<StoreFile> storeFiles) throws IOException {
private Pair<Long, Long> getFileInfo(List<HStoreFile> storeFiles) throws IOException {
long maxSeqId = 0;
long maxKeyCount = 0;
for (StoreFile sf : storeFiles) {
for (HStoreFile sf : storeFiles) {
// the readers will be closed later after the merge.
maxSeqId = Math.max(maxSeqId, sf.getMaxSequenceId());
sf.initReader();
byte[] count = sf.getReader().loadFileInfo().get(StoreFile.MOB_CELLS_COUNT);
byte[] count = sf.getReader().loadFileInfo().get(MOB_CELLS_COUNT);
if (count != null) {
maxKeyCount += Bytes.toLong(count);
}

View File

@ -16,11 +16,11 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.List;
import org.apache.yetus.audience.InterfaceAudience;
/**
@ -40,5 +40,5 @@ public interface ChangedReadersObserver {
* @param memStoreScanners scanner of current memstore
* @throws IOException e
*/
void updateReaders(List<StoreFile> sfs, List<KeyValueScanner> memStoreScanners) throws IOException;
void updateReaders(List<HStoreFile> sfs, List<KeyValueScanner> memStoreScanners) throws IOException;
}

View File

@ -69,7 +69,7 @@ public class CompactingMemStore extends AbstractMemStore {
private static final double IN_MEMORY_FLUSH_THRESHOLD_FACTOR_DEFAULT = 0.25;
private static final Log LOG = LogFactory.getLog(CompactingMemStore.class);
private Store store;
private HStore store;
private RegionServicesForStores regionServices;
private CompactionPipeline pipeline;
private MemStoreCompactor compactor;
@ -341,7 +341,7 @@ public class CompactingMemStore extends AbstractMemStore {
return store.getSmallestReadPoint();
}
public Store getStore() {
public HStore getStore() {
return store;
}

View File

@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.security.User;
public class DateTieredStoreEngine extends StoreEngine<DefaultStoreFlusher,
DateTieredCompactionPolicy, DateTieredCompactor, DefaultStoreFileManager> {
@Override
public boolean needsCompaction(List<StoreFile> filesCompacting) {
public boolean needsCompaction(List<HStoreFile> filesCompacting) {
return compactionPolicy.needsCompaction(storeFileManager.getStorefiles(),
filesCompacting);
}
@ -54,7 +54,7 @@ public class DateTieredStoreEngine extends StoreEngine<DefaultStoreFlusher,
}
@Override
protected void createComponents(Configuration conf, Store store, CellComparator kvComparator)
protected void createComponents(Configuration conf, HStore store, CellComparator kvComparator)
throws IOException {
this.compactionPolicy = new DateTieredCompactionPolicy(conf, store);
this.storeFileManager =
@ -67,13 +67,13 @@ public class DateTieredStoreEngine extends StoreEngine<DefaultStoreFlusher,
private final class DateTieredCompactionContext extends CompactionContext {
@Override
public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) {
return compactionPolicy.preSelectCompactionForCoprocessor(storeFileManager.getStorefiles(),
filesCompacting);
}
@Override
public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
public boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction,
boolean mayUseOffPeak, boolean forceMajor) throws IOException {
request = compactionPolicy.selectCompaction(storeFileManager.getStorefiles(), filesCompacting,
isUserCompaction, mayUseOffPeak, forceMajor);

View File

@ -25,7 +25,6 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
@ -33,6 +32,7 @@ import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPoli
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Default StoreEngine creates the default compactor, policy, and store file manager, or
@ -57,14 +57,14 @@ public class DefaultStoreEngine extends StoreEngine<
DEFAULT_COMPACTION_POLICY_CLASS = ExploringCompactionPolicy.class;
@Override
public boolean needsCompaction(List<StoreFile> filesCompacting) {
public boolean needsCompaction(List<HStoreFile> filesCompacting) {
return compactionPolicy.needsCompaction(
this.storeFileManager.getStorefiles(), filesCompacting);
}
@Override
protected void createComponents(
Configuration conf, Store store, CellComparator kvComparator) throws IOException {
Configuration conf, HStore store, CellComparator kvComparator) throws IOException {
createCompactor(conf, store);
createCompactionPolicy(conf, store);
createStoreFlusher(conf, store);
@ -73,17 +73,17 @@ public class DefaultStoreEngine extends StoreEngine<
compactionPolicy.getConf());
}
protected void createCompactor(Configuration conf, Store store) throws IOException {
protected void createCompactor(Configuration conf, HStore store) throws IOException {
String className = conf.get(DEFAULT_COMPACTOR_CLASS_KEY, DEFAULT_COMPACTOR_CLASS.getName());
try {
compactor = ReflectionUtils.instantiateWithCustomCtor(className,
new Class[] { Configuration.class, Store.class }, new Object[] { conf, store });
new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store });
} catch (Exception e) {
throw new IOException("Unable to load configured compactor '" + className + "'", e);
}
}
protected void createCompactionPolicy(Configuration conf, Store store) throws IOException {
protected void createCompactionPolicy(Configuration conf, HStore store) throws IOException {
String className = conf.get(
DEFAULT_COMPACTION_POLICY_CLASS_KEY, DEFAULT_COMPACTION_POLICY_CLASS.getName());
try {
@ -95,12 +95,12 @@ public class DefaultStoreEngine extends StoreEngine<
}
}
protected void createStoreFlusher(Configuration conf, Store store) throws IOException {
protected void createStoreFlusher(Configuration conf, HStore store) throws IOException {
String className = conf.get(
DEFAULT_STORE_FLUSHER_CLASS_KEY, DEFAULT_STORE_FLUSHER_CLASS.getName());
try {
storeFlusher = ReflectionUtils.instantiateWithCustomCtor(className,
new Class[] { Configuration.class, Store.class }, new Object[] { conf, store });
new Class[] { Configuration.class, HStore.class }, new Object[] { conf, store });
} catch (Exception e) {
throw new IOException("Unable to load configured store flusher '" + className + "'", e);
}
@ -113,7 +113,7 @@ public class DefaultStoreEngine extends StoreEngine<
private class DefaultCompactionContext extends CompactionContext {
@Override
public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
public boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction,
boolean mayUseOffPeak, boolean forceMajor) throws IOException {
request = compactionPolicy.selectCompaction(storeFileManager.getStorefiles(),
filesCompacting, isUserCompaction, mayUseOffPeak, forceMajor);
@ -127,7 +127,7 @@ public class DefaultStoreEngine extends StoreEngine<
}
@Override
public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) {
return compactionPolicy.preSelectCompactionForCoprocessor(
storeFileManager.getStorefiles(), filesCompacting);
}

View File

@ -19,13 +19,12 @@
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -33,12 +32,12 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionConfiguration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableCollection;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterables;
/**
* Default implementation of StoreFileManager. Not thread-safe.
@ -47,27 +46,27 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
class DefaultStoreFileManager implements StoreFileManager {
private static final Log LOG = LogFactory.getLog(DefaultStoreFileManager.class);
private final CellComparator kvComparator;
private final CellComparator cellComparator;
private final CompactionConfiguration comConf;
private final int blockingFileCount;
private final Comparator<StoreFile> storeFileComparator;
private final Comparator<HStoreFile> storeFileComparator;
/**
* List of store files inside this store. This is an immutable list that
* is atomically replaced when its contents change.
*/
private volatile ImmutableList<StoreFile> storefiles = null;
private volatile ImmutableList<HStoreFile> storefiles = ImmutableList.of();
/**
* List of compacted files inside this store that needs to be excluded in reads
* because further new reads will be using only the newly created files out of compaction.
* These compacted files will be deleted/cleared once all the existing readers on these
* compacted files are done.
*/
private volatile List<StoreFile> compactedfiles = null;
private volatile ImmutableList<HStoreFile> compactedfiles = ImmutableList.of();
public DefaultStoreFileManager(CellComparator kvComparator,
Comparator<StoreFile> storeFileComparator, Configuration conf,
public DefaultStoreFileManager(CellComparator cellComparator,
Comparator<HStoreFile> storeFileComparator, Configuration conf,
CompactionConfiguration comConf) {
this.kvComparator = kvComparator;
this.cellComparator = cellComparator;
this.storeFileComparator = storeFileComparator;
this.comConf = comConf;
this.blockingFileCount =
@ -75,39 +74,37 @@ class DefaultStoreFileManager implements StoreFileManager {
}
@Override
public void loadFiles(List<StoreFile> storeFiles) {
sortAndSetStoreFiles(storeFiles);
public void loadFiles(List<HStoreFile> storeFiles) {
this.storefiles = ImmutableList.sortedCopyOf(storeFileComparator, storeFiles);
}
@Override
public final Collection<StoreFile> getStorefiles() {
// TODO: I can return a null list of StoreFiles? That'll mess up clients. St.Ack 20151111
public final Collection<HStoreFile> getStorefiles() {
return storefiles;
}
@Override
public Collection<StoreFile> getCompactedfiles() {
public Collection<HStoreFile> getCompactedfiles() {
return compactedfiles;
}
@Override
public void insertNewFiles(Collection<StoreFile> sfs) throws IOException {
ArrayList<StoreFile> newFiles = new ArrayList<>(storefiles);
newFiles.addAll(sfs);
sortAndSetStoreFiles(newFiles);
public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
this.storefiles =
ImmutableList.sortedCopyOf(storeFileComparator, Iterables.concat(this.storefiles, sfs));
}
@Override
public ImmutableCollection<StoreFile> clearFiles() {
ImmutableList<StoreFile> result = storefiles;
public ImmutableCollection<HStoreFile> clearFiles() {
ImmutableList<HStoreFile> result = storefiles;
storefiles = ImmutableList.of();
return result;
}
@Override
public Collection<StoreFile> clearCompactedFiles() {
List<StoreFile> result = compactedfiles;
compactedfiles = new ArrayList<>();
public Collection<HStoreFile> clearCompactedFiles() {
List<HStoreFile> result = compactedfiles;
compactedfiles = ImmutableList.of();
return result;
}
@ -118,60 +115,39 @@ class DefaultStoreFileManager implements StoreFileManager {
@Override
public final int getCompactedFilesCount() {
if (compactedfiles == null) {
return 0;
}
return compactedfiles.size();
}
@Override
public void addCompactionResults(
Collection<StoreFile> newCompactedfiles, Collection<StoreFile> results) {
ArrayList<StoreFile> newStoreFiles = Lists.newArrayList(storefiles);
newStoreFiles.removeAll(newCompactedfiles);
if (!results.isEmpty()) {
newStoreFiles.addAll(results);
}
sortAndSetStoreFiles(newStoreFiles);
ArrayList<StoreFile> updatedCompactedfiles = null;
if (this.compactedfiles != null) {
updatedCompactedfiles = new ArrayList<>(this.compactedfiles);
updatedCompactedfiles.addAll(newCompactedfiles);
} else {
updatedCompactedfiles = new ArrayList<>(newCompactedfiles);
}
markCompactedAway(newCompactedfiles);
this.compactedfiles = sortCompactedfiles(updatedCompactedfiles);
}
// Mark the files as compactedAway once the storefiles and compactedfiles list is finalized
// Let a background thread close the actual reader on these compacted files and also
// ensure to evict the blocks from block cache so that they are no longer in
// cache
private void markCompactedAway(Collection<StoreFile> compactedFiles) {
for (StoreFile file : compactedFiles) {
file.markCompactedAway();
}
public void addCompactionResults(Collection<HStoreFile> newCompactedfiles,
Collection<HStoreFile> results) {
this.storefiles = ImmutableList.sortedCopyOf(storeFileComparator, Iterables
.concat(Iterables.filter(storefiles, sf -> !newCompactedfiles.contains(sf)), results));
// Mark the files as compactedAway once the storefiles and compactedfiles list is finalized
// Let a background thread close the actual reader on these compacted files and also
// ensure to evict the blocks from block cache so that they are no longer in
// cache
newCompactedfiles.forEach(HStoreFile::markCompactedAway);
this.compactedfiles = ImmutableList.sortedCopyOf(storeFileComparator,
Iterables.concat(this.compactedfiles, newCompactedfiles));
}
@Override
public void removeCompactedFiles(Collection<StoreFile> removedCompactedfiles) throws IOException {
ArrayList<StoreFile> updatedCompactedfiles = null;
if (this.compactedfiles != null) {
updatedCompactedfiles = new ArrayList<>(this.compactedfiles);
updatedCompactedfiles.removeAll(removedCompactedfiles);
this.compactedfiles = sortCompactedfiles(updatedCompactedfiles);
}
public void removeCompactedFiles(Collection<HStoreFile> removedCompactedfiles)
throws IOException {
this.compactedfiles =
this.compactedfiles.stream().filter(sf -> !removedCompactedfiles.contains(sf))
.sorted(storeFileComparator).collect(ImmutableList.toImmutableList());
}
@Override
public final Iterator<StoreFile> getCandidateFilesForRowKeyBefore(final KeyValue targetKey) {
return new ArrayList<>(Lists.reverse(this.storefiles)).iterator();
public final Iterator<HStoreFile> getCandidateFilesForRowKeyBefore(KeyValue targetKey) {
return this.storefiles.reverse().iterator();
}
@Override
public Iterator<StoreFile> updateCandidateFilesForRowKeyBefore(
Iterator<StoreFile> candidateFiles, final KeyValue targetKey, final Cell candidate) {
public Iterator<HStoreFile> updateCandidateFilesForRowKeyBefore(
Iterator<HStoreFile> candidateFiles, KeyValue targetKey, Cell candidate) {
// Default store has nothing useful to do here.
// TODO: move this comment when implementing Level:
// Level store can trim the list by range, removing all the files which cannot have
@ -180,18 +156,12 @@ class DefaultStoreFileManager implements StoreFileManager {
}
@Override
public final byte[] getSplitPoint() throws IOException {
List<StoreFile> storefiles = this.storefiles;
if (storefiles.isEmpty()) {
return null;
}
Optional<StoreFile> largestFile = StoreUtils.getLargestFile(storefiles);
return largestFile.isPresent()
? StoreUtils.getFileSplitPoint(largestFile.get(), kvComparator).orElse(null) : null;
public final Optional<byte[]> getSplitPoint() throws IOException {
return StoreUtils.getSplitPoint(storefiles, cellComparator);
}
@Override
public final Collection<StoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow,
public final Collection<HStoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow,
byte[] stopRow, boolean includeStopRow) {
// We cannot provide any useful input and already have the files sorted by seqNum.
return getStorefiles();
@ -204,35 +174,20 @@ class DefaultStoreFileManager implements StoreFileManager {
}
@Override
public Collection<StoreFile> getUnneededFiles(long maxTs, List<StoreFile> filesCompacting) {
Collection<StoreFile> expiredStoreFiles = null;
ImmutableList<StoreFile> files = storefiles;
public Collection<HStoreFile> getUnneededFiles(long maxTs, List<HStoreFile> filesCompacting) {
ImmutableList<HStoreFile> files = storefiles;
// 1) We can never get rid of the last file which has the maximum seqid.
// 2) Files that are not the latest can't become one due to (1), so the rest are fair game.
for (int i = 0; i < files.size() - 1; ++i) {
StoreFile sf = files.get(i);
return files.stream().limit(Math.max(0, files.size() - 1)).filter(sf -> {
long fileTs = sf.getReader().getMaxTimestamp();
if (fileTs < maxTs && !filesCompacting.contains(sf)) {
LOG.info("Found an expired store file: " + sf.getPath()
+ " whose maxTimeStamp is " + fileTs + ", which is below " + maxTs);
if (expiredStoreFiles == null) {
expiredStoreFiles = new ArrayList<>();
}
expiredStoreFiles.add(sf);
LOG.info("Found an expired store file: " + sf.getPath() + " whose maxTimeStamp is " +
fileTs + ", which is below " + maxTs);
return true;
} else {
return false;
}
}
return expiredStoreFiles;
}
private void sortAndSetStoreFiles(List<StoreFile> storeFiles) {
Collections.sort(storeFiles, storeFileComparator);
storefiles = ImmutableList.copyOf(storeFiles);
}
private List<StoreFile> sortCompactedfiles(List<StoreFile> storefiles) {
// Sorting may not be really needed here for the compacted files?
Collections.sort(storefiles, storeFileComparator);
return new ArrayList<>(storefiles);
}).collect(Collectors.toList());
}
@Override
@ -246,7 +201,7 @@ class DefaultStoreFileManager implements StoreFileManager {
}
@Override
public Comparator<StoreFile> getStoreFileComparator() {
public Comparator<HStoreFile> getStoreFileComparator() {
return storeFileComparator;
}
}

View File

@ -39,7 +39,7 @@ public class DefaultStoreFlusher extends StoreFlusher {
private static final Log LOG = LogFactory.getLog(DefaultStoreFlusher.class);
private final Object flushLock = new Object();
public DefaultStoreFlusher(Configuration conf, Store store) {
public DefaultStoreFlusher(Configuration conf, HStore store) {
super(conf, store);
}

View File

@ -43,7 +43,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.TagType;
import org.apache.hadoop.hbase.TagUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.filter.Filter;
@ -59,6 +58,7 @@ import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.IdLock;
import org.apache.yetus.audience.InterfaceAudience;
/**
* The store implementation to save MOBs (medium objects), it extends the HStore.
@ -166,7 +166,7 @@ public class HMobStore extends HStore {
* Creates the mob store engine.
*/
@Override
protected StoreEngine<?, ?, ?, ?> createStoreEngine(Store store, Configuration conf,
protected StoreEngine<?, ?, ?, ?> createStoreEngine(HStore store, Configuration conf,
CellComparator cellComparator) throws IOException {
MobStoreEngine engine = new MobStoreEngine();
engine.createComponents(conf, store, cellComparator);
@ -291,7 +291,7 @@ public class HMobStore extends HStore {
* @param path the path to the mob file
*/
private void validateMobFile(Path path) throws IOException {
StoreFile storeFile = null;
HStoreFile storeFile = null;
try {
storeFile = new HStoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig,
BloomType.NONE, isPrimaryReplicaStore());
@ -301,7 +301,7 @@ public class HMobStore extends HStore {
throw e;
} finally {
if (storeFile != null) {
storeFile.closeReader(false);
storeFile.closeStoreFile(false);
}
}
}

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.regionserver;
import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;
import static org.apache.hadoop.hbase.util.CollectionUtils.computeIfAbsent;
import java.io.EOFException;
@ -100,7 +101,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.TagUtil;
import org.apache.hadoop.hbase.UnknownScannerException;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.Delete;
@ -147,9 +147,34 @@ import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTrack
import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.CollectionUtils;
import org.apache.hadoop.hbase.util.CompressionTest;
import org.apache.hadoop.hbase.util.EncryptionTest;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HashedBytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay;
import org.apache.hadoop.io.MultipleIOException;
import org.apache.hadoop.util.StringUtils;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
@ -172,29 +197,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescript
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.CollectionUtils;
import org.apache.hadoop.hbase.util.CompressionTest;
import org.apache.hadoop.hbase.util.EncryptionTest;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HashedBytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay;
import org.apache.hadoop.io.MultipleIOException;
import org.apache.hadoop.util.StringUtils;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
@SuppressWarnings("deprecation")
@InterfaceAudience.Private
@ -1066,12 +1068,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
private NavigableMap<byte[], List<Path>> getStoreFiles() {
NavigableMap<byte[], List<Path>> allStoreFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
for (HStore store : stores.values()) {
Collection<StoreFile> storeFiles = store.getStorefiles();
Collection<HStoreFile> storeFiles = store.getStorefiles();
if (storeFiles == null) {
continue;
}
List<Path> storeFileNames = new ArrayList<>();
for (StoreFile storeFile : storeFiles) {
for (HStoreFile storeFile : storeFiles) {
storeFileNames.add(storeFile.getPath());
}
allStoreFiles.put(store.getColumnFamilyDescriptor().getName(), storeFileNames);
@ -1124,7 +1126,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
public HDFSBlocksDistribution getHDFSBlocksDistribution() {
HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
stores.values().stream().filter(s -> s.getStorefiles() != null)
.flatMap(s -> s.getStorefiles().stream()).map(StoreFile::getHDFSBlockDistribution)
.flatMap(s -> s.getStorefiles().stream()).map(HStoreFile::getHDFSBlockDistribution)
.forEachOrdered(hdfsBlocksDistribution::add);
return hdfsBlocksDistribution;
}
@ -1384,7 +1386,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
LOG.info("DEBUG LIST ALL FILES");
for (HStore store : this.stores.values()) {
LOG.info("store " + store.getColumnFamilyName());
for (StoreFile sf : store.getStorefiles()) {
for (HStoreFile sf : store.getStorefiles()) {
LOG.info(sf.toStringDetailed());
}
}
@ -1458,7 +1460,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* because a Snapshot was not properly persisted. The region is put in closing mode, and the
* caller MUST abort after this.
*/
public Map<byte[], List<StoreFile>> close() throws IOException {
public Map<byte[], List<HStoreFile>> close() throws IOException {
return close(false);
}
@ -1499,7 +1501,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* because a Snapshot was not properly persisted. The region is put in closing mode, and the
* caller MUST abort after this.
*/
public Map<byte[], List<StoreFile>> close(final boolean abort) throws IOException {
public Map<byte[], List<HStoreFile>> close(boolean abort) throws IOException {
// Only allow one thread to close at a time. Serialize them so dual
// threads attempting to close will run up against each other.
MonitoredTask status = TaskMonitor.get().createStatus(
@ -1537,7 +1539,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK_EXCEPTION_PATH",
justification="I think FindBugs is confused")
private Map<byte[], List<StoreFile>> doClose(final boolean abort, MonitoredTask status)
private Map<byte[], List<HStoreFile>> doClose(boolean abort, MonitoredTask status)
throws IOException {
if (isClosed()) {
LOG.warn("Region " + this + " already closed");
@ -1632,13 +1634,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
}
Map<byte[], List<StoreFile>> result = new TreeMap<>(Bytes.BYTES_COMPARATOR);
Map<byte[], List<HStoreFile>> result = new TreeMap<>(Bytes.BYTES_COMPARATOR);
if (!stores.isEmpty()) {
// initialize the thread pool for closing stores in parallel.
ThreadPoolExecutor storeCloserThreadPool =
getStoreOpenAndCloseThreadPool("StoreCloserThread-" +
getRegionInfo().getRegionNameAsString());
CompletionService<Pair<byte[], Collection<StoreFile>>> completionService =
CompletionService<Pair<byte[], Collection<HStoreFile>>> completionService =
new ExecutorCompletionService<>(storeCloserThreadPool);
// close each store in parallel
@ -1654,18 +1656,18 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
}
completionService
.submit(new Callable<Pair<byte[], Collection<StoreFile>>>() {
.submit(new Callable<Pair<byte[], Collection<HStoreFile>>>() {
@Override
public Pair<byte[], Collection<StoreFile>> call() throws IOException {
public Pair<byte[], Collection<HStoreFile>> call() throws IOException {
return new Pair<>(store.getColumnFamilyDescriptor().getName(), store.close());
}
});
}
try {
for (int i = 0; i < stores.size(); i++) {
Future<Pair<byte[], Collection<StoreFile>>> future = completionService.take();
Pair<byte[], Collection<StoreFile>> storeFiles = future.get();
List<StoreFile> familyFiles = result.get(storeFiles.getFirst());
Future<Pair<byte[], Collection<HStoreFile>>> future = completionService.take();
Pair<byte[], Collection<HStoreFile>> storeFiles = future.get();
List<HStoreFile> familyFiles = result.get(storeFiles.getFirst());
if (familyFiles == null) {
familyFiles = new ArrayList<>();
result.put(storeFiles.getFirst(), familyFiles);
@ -1874,11 +1876,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
public long getOldestHfileTs(boolean majorCompactionOnly) throws IOException {
long result = Long.MAX_VALUE;
for (HStore store : stores.values()) {
Collection<StoreFile> storeFiles = store.getStorefiles();
Collection<HStoreFile> storeFiles = store.getStorefiles();
if (storeFiles == null) {
continue;
}
for (StoreFile file : storeFiles) {
for (HStoreFile file : storeFiles) {
StoreFileReader sfReader = file.getReader();
if (sfReader == null) {
continue;
@ -1888,7 +1890,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
continue;
}
if (majorCompactionOnly) {
byte[] val = reader.loadFileInfo().get(StoreFile.MAJOR_COMPACTION_KEY);
byte[] val = reader.loadFileInfo().get(MAJOR_COMPACTION_KEY);
if (val == null || !Bytes.toBoolean(val)) {
continue;
}
@ -4182,7 +4184,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// If this flag is set, make use of the hfile archiving by making recovered.edits a fake
// column family. Have to fake out file type too by casting our recovered.edits as storefiles
String fakeFamilyName = WALSplitter.getRegionDirRecoveredEditsDir(regiondir).getName();
Set<StoreFile> fakeStoreFiles = new HashSet<>(files.size());
Set<HStoreFile> fakeStoreFiles = new HashSet<>(files.size());
for (Path file: files) {
fakeStoreFiles.add(
new HStoreFile(getRegionFileSystem().getFileSystem(), file, this.conf, null, null, true));
@ -5296,11 +5298,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
throw new IllegalArgumentException(
"No column family : " + new String(column) + " available");
}
Collection<StoreFile> storeFiles = store.getStorefiles();
Collection<HStoreFile> storeFiles = store.getStorefiles();
if (storeFiles == null) {
continue;
}
for (StoreFile storeFile : storeFiles) {
for (HStoreFile storeFile : storeFiles) {
storeFileNames.add(storeFile.getPath().toString());
}

View File

@ -25,7 +25,7 @@ import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.UUID;
import org.apache.commons.logging.Log;
@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.fs.HFileSystem;
@ -54,6 +53,7 @@ import org.apache.hadoop.hbase.util.FSHDFSUtils;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
@ -481,20 +481,6 @@ public class HRegionFileSystem {
return dstPath;
}
/**
* Moves multiple store files to the relative region's family store directory.
* @param storeFiles list of store files divided by family
* @throws IOException
*/
void commitStoreFiles(final Map<byte[], List<StoreFile>> storeFiles) throws IOException {
for (Map.Entry<byte[], List<StoreFile>> es: storeFiles.entrySet()) {
String familyName = Bytes.toString(es.getKey());
for (StoreFile sf: es.getValue()) {
commitStoreFile(familyName, sf.getPath());
}
}
}
/**
* Archives the specified store file from the specified family.
* @param familyName Family that contains the store files
@ -513,7 +499,7 @@ public class HRegionFileSystem {
* @param storeFiles set of store files to remove
* @throws IOException if the archiving fails
*/
public void removeStoreFiles(final String familyName, final Collection<StoreFile> storeFiles)
public void removeStoreFiles(String familyName, Collection<HStoreFile> storeFiles)
throws IOException {
HFileArchiver.archiveStoreFiles(this.conf, this.fs, this.regionInfoForFs,
this.tableDir, Bytes.toBytes(familyName), storeFiles);
@ -671,9 +657,8 @@ public class HRegionFileSystem {
* @return Path to created reference.
* @throws IOException
*/
public Path splitStoreFile(final HRegionInfo hri, final String familyName, final StoreFile f,
final byte[] splitRow, final boolean top, RegionSplitPolicy splitPolicy)
throws IOException {
public Path splitStoreFile(HRegionInfo hri, String familyName, HStoreFile f, byte[] splitRow,
boolean top, RegionSplitPolicy splitPolicy) throws IOException {
if (splitPolicy == null || !splitPolicy.skipStoreFileRangeCheck(familyName)) {
// Check whether the split row lies in the range of the store file
// If it is outside the range, return directly.
@ -682,28 +667,28 @@ public class HRegionFileSystem {
if (top) {
//check if larger than last key.
Cell splitKey = CellUtil.createFirstOnRow(splitRow);
Cell lastKey = f.getLastKey();
Optional<Cell> lastKey = f.getLastKey();
// If lastKey is null means storefile is empty.
if (lastKey == null) {
if (!lastKey.isPresent()) {
return null;
}
if (f.getComparator().compare(splitKey, lastKey) > 0) {
if (f.getComparator().compare(splitKey, lastKey.get()) > 0) {
return null;
}
} else {
//check if smaller than first key
Cell splitKey = CellUtil.createLastOnRow(splitRow);
Cell firstKey = f.getFirstKey();
Optional<Cell> firstKey = f.getFirstKey();
// If firstKey is null means storefile is empty.
if (firstKey == null) {
if (!firstKey.isPresent()) {
return null;
}
if (f.getComparator().compare(splitKey, firstKey) < 0) {
if (f.getComparator().compare(splitKey, firstKey.get()) < 0) {
return null;
}
}
} finally {
f.closeReader(f.getCacheConf() != null ? f.getCacheConf().shouldEvictOnClose() : true);
f.closeStoreFile(f.getCacheConf() != null ? f.getCacheConf().shouldEvictOnClose() : true);
}
}
@ -791,9 +776,8 @@ public class HRegionFileSystem {
* @return Path to created reference.
* @throws IOException
*/
public Path mergeStoreFile(final HRegionInfo mergedRegion, final String familyName,
final StoreFile f, final Path mergedDir)
throws IOException {
public Path mergeStoreFile(HRegionInfo mergedRegion, String familyName, HStoreFile f,
Path mergedDir) throws IOException {
Path referenceDir = new Path(new Path(mergedDir,
mergedRegion.getEncodedName()), familyName);
// A whole reference to the store file.

View File

@ -18,7 +18,6 @@
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.Thread.UncaughtExceptionHandler;
@ -84,7 +83,6 @@ import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.YouAreDeadException;
import org.apache.hadoop.hbase.ZNodeClearer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionUtils;
@ -141,6 +139,44 @@ import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad;
import org.apache.hadoop.hbase.security.Superusers;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CompressionTest;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.JSONBean;
import org.apache.hadoop.hbase.util.JvmPauseMonitor;
import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
import org.apache.hadoop.hbase.util.Sleeper;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.NettyAsyncFSWALConfigHelper;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.RecoveringRegionWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.data.Stat;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Maps;
@ -177,42 +213,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProto
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRSFatalErrorRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.RegionServerStatusProtos.ReportRegionStateTransitionResponse;
import org.apache.hadoop.hbase.trace.SpanReceiverHost;
import org.apache.hadoop.hbase.util.Addressing;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CompressionTest;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.JSONBean;
import org.apache.hadoop.hbase.util.JvmPauseMonitor;
import org.apache.hadoop.hbase.util.NettyEventLoopGroupConfig;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
import org.apache.hadoop.hbase.util.Sleeper;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
import org.apache.hadoop.hbase.wal.NettyAsyncFSWALConfigHelper;
import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALFactory;
import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.RecoveringRegionWatcher;
import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.StringUtils;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.data.Stat;
import sun.misc.Signal;
import sun.misc.SignalHandler;
@ -3172,7 +3172,7 @@ public class HRegionServer extends HasThread implements
for (int i = 0; i < regionEncodedName.size(); ++i) {
Region regionToClose = this.getFromOnlineRegions(regionEncodedName.get(i));
if (regionToClose != null) {
Map<byte[], List<StoreFile>> hstoreFiles = null;
Map<byte[], List<HStoreFile>> hstoreFiles = null;
Exception exceptionToThrow = null;
try{
hstoreFiles = ((HRegion)regionToClose).close(false);

View File

@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.Collections;
import java.util.Comparator;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalLong;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
@ -32,14 +32,15 @@ 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.CellComparator;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.io.hfile.BlockType;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.util.BloomFilterFactory;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
@ -61,8 +62,50 @@ public class HStoreFile implements StoreFile {
private static final Log LOG = LogFactory.getLog(HStoreFile.class.getName());
public static final String STORE_FILE_READER_NO_READAHEAD = "hbase.store.reader.no-readahead";
private static final boolean DEFAULT_STORE_FILE_READER_NO_READAHEAD = false;
// Keys for fileinfo values in HFile
/** Max Sequence ID in FileInfo */
public static final byte[] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
/** Major compaction flag in FileInfo */
public static final byte[] MAJOR_COMPACTION_KEY = Bytes.toBytes("MAJOR_COMPACTION_KEY");
/** Minor compaction flag in FileInfo */
public static final byte[] EXCLUDE_FROM_MINOR_COMPACTION_KEY =
Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION");
/** Bloom filter Type in FileInfo */
public static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE");
/** Delete Family Count in FileInfo */
public static final byte[] DELETE_FAMILY_COUNT = Bytes.toBytes("DELETE_FAMILY_COUNT");
/** Last Bloom filter key in FileInfo */
public static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
/** Key for Timerange information in metadata */
public static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
/** Key for timestamp of earliest-put in metadata */
public static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS");
/** Key for the number of mob cells in metadata */
public static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT");
/** Meta key set when store file is a result of a bulk load */
public static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK");
public static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP");
/**
* Key for skipping resetting sequence id in metadata. For bulk loaded hfiles, the scanner resets
* the cell seqId with the latest one, if this metadata is set as true, the reset is skipped.
*/
public static final byte[] SKIP_RESET_SEQ_ID = Bytes.toBytes("SKIP_RESET_SEQ_ID");
private final StoreFileInfo fileInfo;
private final FileSystem fs;
@ -90,29 +133,28 @@ public class HStoreFile implements StoreFile {
private long maxMemstoreTS = -1;
// firstKey, lastkey and cellComparator will be set when openReader.
private Cell firstKey;
private Optional<Cell> firstKey;
private Cell lastKey;
private Optional<Cell> lastKey;
private Comparator<Cell> comparator;
private CellComparator comparator;
@Override
public CacheConfig getCacheConf() {
return cacheConf;
}
@Override
public Cell getFirstKey() {
public Optional<Cell> getFirstKey() {
return firstKey;
}
@Override
public Cell getLastKey() {
public Optional<Cell> getLastKey() {
return lastKey;
}
@Override
public Comparator<Cell> getComparator() {
public CellComparator getComparator() {
return comparator;
}
@ -144,27 +186,6 @@ public class HStoreFile implements StoreFile {
*/
private final BloomType cfBloomType;
/**
* Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
* depending on the underlying files (10-20MB?).
* @param fs The current file system to use.
* @param p The path of the file.
* @param conf The current configuration.
* @param cacheConf The cache configuration and block cache reference.
* @param cfBloomType The bloom type to use for this store file as specified by column family
* configuration. This may or may not be the same as the Bloom filter type actually
* present in the HFile, because column family configuration might change. If this is
* {@link BloomType#NONE}, the existing Bloom filter is ignored.
* @deprecated Now we will specific whether the StoreFile is for primary replica when
* constructing, so please use {@link #HStoreFile(FileSystem, Path, Configuration,
* CacheConfig, BloomType, boolean)} directly.
*/
@Deprecated
public HStoreFile(final FileSystem fs, final Path p, final Configuration conf,
final CacheConfig cacheConf, final BloomType cfBloomType) throws IOException {
this(fs, new StoreFileInfo(conf, fs, p), conf, cacheConf, cfBloomType);
}
/**
* Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
* depending on the underlying files (10-20MB?).
@ -184,27 +205,6 @@ public class HStoreFile implements StoreFile {
this(fs, new StoreFileInfo(conf, fs, p), conf, cacheConf, cfBloomType, primaryReplica);
}
/**
* Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
* depending on the underlying files (10-20MB?).
* @param fs The current file system to use.
* @param fileInfo The store file information.
* @param conf The current configuration.
* @param cacheConf The cache configuration and block cache reference.
* @param cfBloomType The bloom type to use for this store file as specified by column family
* configuration. This may or may not be the same as the Bloom filter type actually
* present in the HFile, because column family configuration might change. If this is
* {@link BloomType#NONE}, the existing Bloom filter is ignored.
* @deprecated Now we will specific whether the StoreFile is for primary replica when
* constructing, so please use {@link #HStoreFile(FileSystem, StoreFileInfo,
* Configuration, CacheConfig, BloomType, boolean)} directly.
*/
@Deprecated
public HStoreFile(final FileSystem fs, final StoreFileInfo fileInfo, final Configuration conf,
final CacheConfig cacheConf, final BloomType cfBloomType) throws IOException {
this(fs, fileInfo, conf, cacheConf, cfBloomType, true);
}
/**
* Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
* depending on the underlying files (10-20MB?).
@ -235,7 +235,10 @@ public class HStoreFile implements StoreFile {
this.primaryReplica = primaryReplica;
}
@Override
/**
* @return the StoreFile object associated to this StoreFile. null if the StoreFile is not a
* reference.
*/
public StoreFileInfo getFileInfo() {
return this.fileInfo;
}
@ -283,7 +286,11 @@ public class HStoreFile implements StoreFile {
return fileInfo.getModificationTime();
}
@Override
/**
* Only used by the Striped Compaction Policy
* @param key
* @return value associated with the metadata key
*/
public byte[] getMetadataValue(byte[] key) {
return metadataMap.get(key);
}
@ -299,7 +306,6 @@ public class HStoreFile implements StoreFile {
return bulkLoadedHFile || (metadataMap != null && metadataMap.containsKey(BULKLOAD_TIME_KEY));
}
@Override
public boolean isCompactedAway() {
return compactedAway;
}
@ -309,7 +315,9 @@ public class HStoreFile implements StoreFile {
return refCount.get();
}
@Override
/**
* @return true if the file is still used in reads
*/
public boolean isReferencedInReads() {
int rc = refCount.get();
assert rc >= 0; // we should not go negative.
@ -331,7 +339,7 @@ public class HStoreFile implements StoreFile {
/**
* Opens reader on this store file. Called by Constructor.
* @throws IOException
* @see #closeReader(boolean)
* @see #closeStoreFile(boolean)
*/
private void open() throws IOException {
if (this.reader != null) {
@ -440,7 +448,9 @@ public class HStoreFile implements StoreFile {
comparator = reader.getComparator();
}
@Override
/**
* Initialize the reader used for pread.
*/
public void initReader() throws IOException {
if (reader == null) {
try {
@ -448,7 +458,7 @@ public class HStoreFile implements StoreFile {
} catch (Exception e) {
try {
boolean evictOnClose = cacheConf != null ? cacheConf.shouldEvictOnClose() : true;
this.closeReader(evictOnClose);
this.closeStoreFile(evictOnClose);
} catch (IOException ee) {
LOG.warn("failed to close reader", ee);
}
@ -465,14 +475,22 @@ public class HStoreFile implements StoreFile {
return reader;
}
@Override
/**
* Get a scanner which uses pread.
* <p>
* Must be called after initReader.
*/
public StoreFileScanner getPreadScanner(boolean cacheBlocks, long readPt, long scannerOrder,
boolean canOptimizeForNonNullColumn) {
return getReader().getStoreFileScanner(cacheBlocks, true, false, readPt, scannerOrder,
canOptimizeForNonNullColumn);
}
@Override
/**
* Get a scanner which uses streaming read.
* <p>
* Must be called after initReader.
*/
public StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks,
boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn)
throws IOException {
@ -480,31 +498,37 @@ public class HStoreFile implements StoreFile {
isCompaction, readPt, scannerOrder, canOptimizeForNonNullColumn);
}
@Override
/**
* @return Current reader. Must call initReader first else returns null.
* @see #initReader()
*/
public StoreFileReader getReader() {
return this.reader;
}
@Override
public synchronized void closeReader(boolean evictOnClose)
throws IOException {
/**
* @param evictOnClose whether to evict blocks belonging to this file
* @throws IOException
*/
public synchronized void closeStoreFile(boolean evictOnClose) throws IOException {
if (this.reader != null) {
this.reader.close(evictOnClose);
this.reader = null;
}
}
@Override
public void markCompactedAway() {
this.compactedAway = true;
/**
* Delete this file
* @throws IOException
*/
public void deleteStoreFile() throws IOException {
boolean evictOnClose = cacheConf != null ? cacheConf.shouldEvictOnClose() : true;
closeStoreFile(evictOnClose);
this.fs.delete(getPath(), true);
}
@Override
public void deleteReader() throws IOException {
boolean evictOnClose =
cacheConf != null? cacheConf.shouldEvictOnClose(): true;
closeReader(evictOnClose);
this.fs.delete(getPath(), true);
public void markCompactedAway() {
this.compactedAway = true;
}
@Override

View File

@ -246,30 +246,27 @@ public class MemStoreCompactor {
MemStoreSegmentsIterator iterator = null;
switch (action) {
case COMPACT:
iterator =
new MemStoreCompactorSegmentsIterator(versionedList.getStoreSegments(),
compactingMemStore.getComparator(),
compactionKVMax, compactingMemStore.getStore());
case COMPACT:
iterator = new MemStoreCompactorSegmentsIterator(versionedList.getStoreSegments(),
compactingMemStore.getComparator(), compactionKVMax, compactingMemStore.getStore());
result = SegmentFactory.instance().createImmutableSegmentByCompaction(
result = SegmentFactory.instance().createImmutableSegmentByCompaction(
compactingMemStore.getConfiguration(), compactingMemStore.getComparator(), iterator,
versionedList.getNumOfCells(), compactingMemStore.getIndexType());
iterator.close();
break;
case MERGE:
iterator =
new MemStoreMergerSegmentsIterator(versionedList.getStoreSegments(),
compactingMemStore.getComparator(),
compactionKVMax);
iterator.close();
break;
case MERGE:
iterator = new MemStoreMergerSegmentsIterator(versionedList.getStoreSegments(),
compactingMemStore.getComparator(), compactionKVMax);
result = SegmentFactory.instance().createImmutableSegmentByMerge(
result = SegmentFactory.instance().createImmutableSegmentByMerge(
compactingMemStore.getConfiguration(), compactingMemStore.getComparator(), iterator,
versionedList.getNumOfCells(), versionedList.getStoreSegments(),
compactingMemStore.getIndexType());
iterator.close();
break;
default: throw new RuntimeException("Unknown action " + action); // sanity check
iterator.close();
break;
default:
throw new RuntimeException("Unknown action " + action); // sanity check
}
return result;

View File

@ -45,10 +45,8 @@ public class MemStoreCompactorSegmentsIterator extends MemStoreSegmentsIterator
private StoreScanner compactingScanner;
// C-tor
public MemStoreCompactorSegmentsIterator(
List<ImmutableSegment> segments,
CellComparator comparator, int compactionKVMax, Store store
) throws IOException {
public MemStoreCompactorSegmentsIterator(List<ImmutableSegment> segments,
CellComparator comparator, int compactionKVMax, HStore store) throws IOException {
super(compactionKVMax);
List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>();
@ -108,7 +106,7 @@ public class MemStoreCompactorSegmentsIterator extends MemStoreSegmentsIterator
* Creates the scanner for compacting the pipeline.
* @return the scanner
*/
private StoreScanner createScanner(Store store, List<KeyValueScanner> scanners)
private StoreScanner createScanner(HStore store, List<KeyValueScanner> scanners)
throws IOException {
// Get all available versions
return new StoreScanner(store, store.getScanInfo(), OptionalInt.of(Integer.MAX_VALUE), scanners,

View File

@ -23,9 +23,9 @@ import java.util.List;
import java.util.NavigableSet;
import org.apache.hadoop.hbase.Cell;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Scanner scans both the memstore and the MOB Store. Coalesce KeyValue stream into
@ -39,7 +39,7 @@ public class MobStoreScanner extends StoreScanner {
private boolean readEmptyValueOnMobCellMiss = false;
private final HMobStore mobStore;
public MobStoreScanner(Store store, ScanInfo scanInfo, Scan scan,
public MobStoreScanner(HStore store, ScanInfo scanInfo, Scan scan,
final NavigableSet<byte[]> columns, long readPt) throws IOException {
super(store, scanInfo, scan, columns, readPt);
cacheMobBlocks = MobUtils.isCacheMobBlocks(scan);

View File

@ -499,7 +499,7 @@ public class RegionCoprocessorHost
* {@link RegionObserver#preCompactScannerOpen(ObserverContext, Store, List, ScanType, long,
* InternalScanner, CompactionLifeCycleTracker, long)}
*/
public InternalScanner preCompactScannerOpen(Store store, List<StoreFileScanner> scanners,
public InternalScanner preCompactScannerOpen(HStore store, List<StoreFileScanner> scanners,
ScanType scanType, long earliestPutTs, CompactionLifeCycleTracker tracker, User user,
long readPoint) throws IOException {
return execOperationWithResult(null,
@ -514,7 +514,7 @@ public class RegionCoprocessorHost
}
/**
* Called prior to selecting the {@link StoreFile}s for compaction from the list of currently
* Called prior to selecting the {@link HStoreFile}s for compaction from the list of currently
* available candidates.
* @param store The store where compaction is being requested
* @param candidates The currently available store files
@ -522,7 +522,7 @@ public class RegionCoprocessorHost
* @return If {@code true}, skip the normal selection process and use the current list
* @throws IOException
*/
public boolean preCompactSelection(Store store, List<StoreFile> candidates,
public boolean preCompactSelection(HStore store, List<HStoreFile> candidates,
CompactionLifeCycleTracker tracker, User user) throws IOException {
return execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
@Override
@ -534,13 +534,13 @@ public class RegionCoprocessorHost
}
/**
* Called after the {@link StoreFile}s to be compacted have been selected from the available
* Called after the {@link HStoreFile}s to be compacted have been selected from the available
* candidates.
* @param store The store where compaction is being requested
* @param selected The store files selected to compact
* @param tracker used to track the life cycle of a compaction
*/
public void postCompactSelection(Store store, ImmutableList<StoreFile> selected,
public void postCompactSelection(HStore store, ImmutableList<HStoreFile> selected,
CompactionLifeCycleTracker tracker, User user) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
@Override
@ -559,7 +559,7 @@ public class RegionCoprocessorHost
* @param tracker used to track the life cycle of a compaction
* @throws IOException
*/
public InternalScanner preCompact(Store store, InternalScanner scanner, ScanType scanType,
public InternalScanner preCompact(HStore store, InternalScanner scanner, ScanType scanType,
CompactionLifeCycleTracker tracker, User user) throws IOException {
return execOperationWithResult(false, scanner,
coprocessors.isEmpty() ? null : new RegionOperationWithResult<InternalScanner>(user) {
@ -578,7 +578,7 @@ public class RegionCoprocessorHost
* @param tracker used to track the life cycle of a compaction
* @throws IOException
*/
public void postCompact(Store store, StoreFile resultFile, CompactionLifeCycleTracker tracker,
public void postCompact(HStore store, HStoreFile resultFile, CompactionLifeCycleTracker tracker,
User user) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new RegionOperation(user) {
@Override
@ -593,7 +593,7 @@ public class RegionCoprocessorHost
* Invoked before a memstore flush
* @throws IOException
*/
public InternalScanner preFlush(final Store store, final InternalScanner scanner)
public InternalScanner preFlush(HStore store, final InternalScanner scanner)
throws IOException {
return execOperationWithResult(false, scanner,
coprocessors.isEmpty() ? null : new RegionOperationWithResult<InternalScanner>() {
@ -623,16 +623,16 @@ public class RegionCoprocessorHost
* See
* {@link RegionObserver#preFlushScannerOpen(ObserverContext, Store, List, InternalScanner, long)}
*/
public InternalScanner preFlushScannerOpen(final Store store,
final List<KeyValueScanner> scanners, final long readPoint) throws IOException {
public InternalScanner preFlushScannerOpen(HStore store, List<KeyValueScanner> scanners,
long readPoint) throws IOException {
return execOperationWithResult(null,
coprocessors.isEmpty() ? null : new RegionOperationWithResult<InternalScanner>() {
@Override
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
throws IOException {
setResult(oserver.preFlushScannerOpen(ctx, store, scanners, getResult(), readPoint));
}
});
coprocessors.isEmpty() ? null : new RegionOperationWithResult<InternalScanner>() {
@Override
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
throws IOException {
setResult(oserver.preFlushScannerOpen(ctx, store, scanners, getResult(), readPoint));
}
});
}
/**
@ -653,7 +653,7 @@ public class RegionCoprocessorHost
* Invoked after a memstore flush
* @throws IOException
*/
public void postFlush(final Store store, final StoreFile storeFile) throws IOException {
public void postFlush(HStore store, HStoreFile storeFile) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new RegionOperation() {
@Override
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
@ -1136,16 +1136,16 @@ public class RegionCoprocessorHost
* See
* {@link RegionObserver#preStoreScannerOpen(ObserverContext, Store, Scan, NavigableSet, KeyValueScanner, long)}
*/
public KeyValueScanner preStoreScannerOpen(final Store store, final Scan scan,
final NavigableSet<byte[]> targetCols, final long readPt) throws IOException {
public KeyValueScanner preStoreScannerOpen(HStore store, Scan scan,
NavigableSet<byte[]> targetCols, long readPt) throws IOException {
return execOperationWithResult(null,
coprocessors.isEmpty() ? null : new RegionOperationWithResult<KeyValueScanner>() {
@Override
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
throws IOException {
setResult(oserver.preStoreScannerOpen(ctx, store, scan, targetCols, getResult(), readPt));
}
});
coprocessors.isEmpty() ? null : new RegionOperationWithResult<KeyValueScanner>() {
@Override
public void call(RegionObserver oserver, ObserverContext<RegionCoprocessorEnvironment> ctx)
throws IOException {
setResult(oserver.preStoreScannerOpen(ctx, store, scan, targetCols, getResult(), readPt));
}
});
}
/**

View File

@ -19,14 +19,15 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.List;
import java.util.Optional;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
@ -80,12 +81,12 @@ public abstract class RegionSplitPolicy extends Configured {
byte[] splitPointFromLargestStore = null;
long largestStoreSize = 0;
for (Store s : stores) {
byte[] splitPoint = s.getSplitPoint();
for (HStore s : stores) {
Optional<byte[]> splitPoint = s.getSplitPoint();
// Store also returns null if it has references as way of indicating it is not splittable
long storeSize = s.getSize();
if (splitPoint != null && largestStoreSize < storeSize) {
splitPointFromLargestStore = splitPoint;
if (splitPoint.isPresent() && largestStoreSize < storeSize) {
splitPointFromLargestStore = splitPoint.get();
largestStoreSize = storeSize;
}
}
@ -131,7 +132,7 @@ public abstract class RegionSplitPolicy extends Configured {
/**
* In {@link HRegionFileSystem#splitStoreFile(org.apache.hadoop.hbase.HRegionInfo, String,
* StoreFile, byte[], boolean, RegionSplitPolicy)} we are not creating the split reference
* HStoreFile, byte[], boolean, RegionSplitPolicy)} we are not creating the split reference
* if split row not lies in the StoreFile range. But in some use cases we may need to create
* the split reference even when the split row not lies in the range. This method can be used
* to decide, whether to skip the the StoreFile range check or not.

View File

@ -23,9 +23,9 @@ import java.util.List;
import java.util.NavigableSet;
import org.apache.hadoop.hbase.Cell;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* ReversedMobStoreScanner extends from ReversedStoreScanner, and is used to support
@ -40,7 +40,7 @@ public class ReversedMobStoreScanner extends ReversedStoreScanner {
private boolean readEmptyValueOnMobCellMiss = false;
protected final HMobStore mobStore;
ReversedMobStoreScanner(Store store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
ReversedMobStoreScanner(HStore store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
long readPt) throws IOException {
super(store, scanInfo, scan, columns, readPt);
cacheMobBlocks = MobUtils.isCacheMobBlocks(scan);

View File

@ -22,12 +22,12 @@ import java.io.IOException;
import java.util.List;
import java.util.NavigableSet;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.yetus.audience.InterfaceAudience;
/**
* ReversedStoreScanner extends from StoreScanner, and is used to support
@ -46,7 +46,7 @@ class ReversedStoreScanner extends StoreScanner implements KeyValueScanner {
* @param columns which columns we are scanning
* @throws IOException
*/
ReversedStoreScanner(Store store, ScanInfo scanInfo, Scan scan,
ReversedStoreScanner(HStore store, ScanInfo scanInfo, Scan scan,
NavigableSet<byte[]> columns, long readPt)
throws IOException {
super(store, scanInfo, scan, columns, readPt);

View File

@ -28,8 +28,6 @@ import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
@ -40,10 +38,10 @@ import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionProgress;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.security.User;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
/**
* Interface for objects that hold a column family in a Region. Its a memstore and a set of zero or
@ -63,9 +61,9 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
// General Accessors
CellComparator getComparator();
Collection<StoreFile> getStorefiles();
Collection<? extends StoreFile> getStorefiles();
Collection<StoreFile> getCompactedFiles();
Collection<? extends StoreFile> getCompactedFiles();
/**
* Close all the readers We don't need to worry about subsequent requests because the Region
@ -73,7 +71,7 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
* @return the {@link StoreFile StoreFiles} that were previously being used.
* @throws IOException on failure
*/
Collection<StoreFile> close() throws IOException;
Collection<? extends StoreFile> close() throws IOException;
/**
* Return a scanner for both the memstore and the HStore files. Assumes we are not in a
@ -86,105 +84,6 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
KeyValueScanner getScanner(Scan scan, final NavigableSet<byte[]> targetCols, long readPt)
throws IOException;
/**
* Get all scanners with no filtering based on TTL (that happens further down the line).
* @param cacheBlocks cache the blocks or not
* @param usePread true to use pread, false if not
* @param isCompaction true if the scanner is created for compaction
* @param matcher the scan query matcher
* @param startRow the start row
* @param stopRow the stop row
* @param readPt the read point of the current scan
* @return all scanners for this store
*/
default List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean isGet, boolean usePread,
boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, byte[] stopRow, long readPt)
throws IOException {
return getScanners(cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow, false,
readPt);
}
/**
* Get all scanners with no filtering based on TTL (that happens further down the line).
* @param cacheBlocks cache the blocks or not
* @param usePread true to use pread, false if not
* @param isCompaction true if the scanner is created for compaction
* @param matcher the scan query matcher
* @param startRow the start row
* @param includeStartRow true to include start row, false if not
* @param stopRow the stop row
* @param includeStopRow true to include stop row, false if not
* @param readPt the read point of the current scan
* @return all scanners for this store
*/
List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean usePread, boolean isCompaction,
ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow,
boolean includeStopRow, long readPt) throws IOException;
/**
* Recreates the scanners on the current list of active store file scanners
* @param currentFileScanners the current set of active store file scanners
* @param cacheBlocks cache the blocks or not
* @param usePread use pread or not
* @param isCompaction is the scanner for compaction
* @param matcher the scan query matcher
* @param startRow the scan's start row
* @param includeStartRow should the scan include the start row
* @param stopRow the scan's stop row
* @param includeStopRow should the scan include the stop row
* @param readPt the read point of the current scane
* @param includeMemstoreScanner whether the current scanner should include memstorescanner
* @return list of scanners recreated on the current Scanners
* @throws IOException
*/
List<KeyValueScanner> recreateScanners(List<KeyValueScanner> currentFileScanners,
boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher,
byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt,
boolean includeMemstoreScanner) throws IOException;
/**
* Create scanners on the given files and if needed on the memstore with no filtering based on TTL
* (that happens further down the line).
* @param files the list of files on which the scanners has to be created
* @param cacheBlocks cache the blocks or not
* @param usePread true to use pread, false if not
* @param isCompaction true if the scanner is created for compaction
* @param matcher the scan query matcher
* @param startRow the start row
* @param stopRow the stop row
* @param readPt the read point of the current scan
* @param includeMemstoreScanner true if memstore has to be included
* @return scanners on the given files and on the memstore if specified
*/
default List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks,
boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher,
byte[] startRow, byte[] stopRow, long readPt, boolean includeMemstoreScanner)
throws IOException {
return getScanners(files, cacheBlocks, usePread, isCompaction, matcher, startRow, true, stopRow,
false, readPt, includeMemstoreScanner);
}
/**
* Create scanners on the given files and if needed on the memstore with no filtering based on TTL
* (that happens further down the line).
* @param files the list of files on which the scanners has to be created
* @param cacheBlocks ache the blocks or not
* @param usePread true to use pread, false if not
* @param isCompaction true if the scanner is created for compaction
* @param matcher the scan query matcher
* @param startRow the start row
* @param includeStartRow true to include start row, false if not
* @param stopRow the stop row
* @param includeStopRow true to include stop row, false if not
* @param readPt the read point of the current scan
* @param includeMemstoreScanner true if memstore has to be included
* @return scanners on the given files and on the memstore if specified
*/
List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks, boolean usePread,
boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow,
byte[] stopRow, boolean includeStopRow, long readPt, boolean includeMemstoreScanner)
throws IOException;
ScanInfo getScanInfo();
/**
@ -194,7 +93,6 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
FileSystem getFileSystem();
/**
* @param maxKeyCount
* @param compression Compression algorithm to use
@ -269,10 +167,10 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
* @deprecated see compact(CompactionContext, ThroughputController, User)
*/
@Deprecated
List<StoreFile> compact(CompactionContext compaction,
List<? extends StoreFile> compact(CompactionContext compaction,
ThroughputController throughputController) throws IOException;
List<StoreFile> compact(CompactionContext compaction,
List<? extends StoreFile> compact(CompactionContext compaction,
ThroughputController throughputController, User user) throws IOException;
/**
@ -297,10 +195,9 @@ public interface Store extends HeapSize, StoreConfigInformation, PropagatingConf
boolean canSplit();
/**
* Determines if Store should be split
* @return byte[] if store should be split, null otherwise.
* Determines if Store should be split.
*/
byte[] getSplitPoint();
Optional<byte[]> getSplitPoint();
// General accessors into the state of the store
// TODO abstract some of this out into a metrics class

View File

@ -22,13 +22,13 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.List;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionPolicy;
import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
/**
* StoreEngine is a factory that can create the objects necessary for HStore to operate.
@ -84,7 +84,7 @@ public abstract class StoreEngine<SF extends StoreFlusher,
* @param filesCompacting Files currently compacting
* @return whether a compaction selection is possible
*/
public abstract boolean needsCompaction(List<StoreFile> filesCompacting);
public abstract boolean needsCompaction(List<HStoreFile> filesCompacting);
/**
* Creates an instance of a compaction context specific to this engine.
@ -97,13 +97,13 @@ public abstract class StoreEngine<SF extends StoreFlusher,
* Create the StoreEngine's components.
*/
protected abstract void createComponents(
Configuration conf, Store store, CellComparator kvComparator) throws IOException;
Configuration conf, HStore store, CellComparator cellComparator) throws IOException;
private void createComponentsOnce(
Configuration conf, Store store, CellComparator kvComparator) throws IOException {
Configuration conf, HStore store, CellComparator cellComparator) throws IOException {
assert compactor == null && compactionPolicy == null
&& storeFileManager == null && storeFlusher == null;
createComponents(conf, store, kvComparator);
createComponents(conf, store, cellComparator);
assert compactor != null && compactionPolicy != null
&& storeFileManager != null && storeFlusher != null;
}
@ -113,16 +113,16 @@ public abstract class StoreEngine<SF extends StoreFlusher,
* @param store The store. An unfortunate dependency needed due to it
* being passed to coprocessors via the compactor.
* @param conf Store configuration.
* @param kvComparator KVComparator for storeFileManager.
* @param cellComparator CellComparator for storeFileManager.
* @return StoreEngine to use.
*/
public static StoreEngine<?, ?, ?, ?> create(
Store store, Configuration conf, CellComparator kvComparator) throws IOException {
HStore store, Configuration conf, CellComparator cellComparator) throws IOException {
String className = conf.get(STORE_ENGINE_CLASS_KEY, DEFAULT_STORE_ENGINE_CLASS.getName());
try {
StoreEngine<?,?,?,?> se = ReflectionUtils.instantiateWithCustomCtor(
className, new Class[] { }, new Object[] { });
se.createComponentsOnce(conf, store, kvComparator);
se.createComponentsOnce(conf, store, cellComparator);
return se;
} catch (Exception e) {
throw new IOException("Unable to load configured store engine '" + className + "'", e);

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.Comparator;
import java.util.Optional;
import java.util.OptionalLong;
import org.apache.hadoop.fs.Path;
@ -27,73 +28,38 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.util.Bytes;
/**
* An interface to describe a store data file.
* <p>
* <strong>NOTICE: </strong>this interface is mainly designed for coprocessor, so it will not expose
* all the internal APIs for a 'store file'. If you are implementing something inside HBase, i.e,
* not a coprocessor hook, usually you should use {@link HStoreFile} directly as it is the only
* implementation of this interface.
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving
public interface StoreFile {
static final String STORE_FILE_READER_NO_READAHEAD = "hbase.store.reader.no-readahead";
// Keys for fileinfo values in HFile
/** Max Sequence ID in FileInfo */
static final byte[] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
/** Major compaction flag in FileInfo */
static final byte[] MAJOR_COMPACTION_KEY = Bytes.toBytes("MAJOR_COMPACTION_KEY");
/** Minor compaction flag in FileInfo */
static final byte[] EXCLUDE_FROM_MINOR_COMPACTION_KEY =
Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION");
/** Bloom filter Type in FileInfo */
static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE");
/** Delete Family Count in FileInfo */
static final byte[] DELETE_FAMILY_COUNT = Bytes.toBytes("DELETE_FAMILY_COUNT");
/** Last Bloom filter key in FileInfo */
static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
/** Key for Timerange information in metadata */
static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
/** Key for timestamp of earliest-put in metadata */
static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS");
/** Key for the number of mob cells in metadata */
static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT");
/** Meta key set when store file is a result of a bulk load */
static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK");
static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP");
/**
* Get the first key in this store file.
*/
Optional<Cell> getFirstKey();
/**
* Key for skipping resetting sequence id in metadata. For bulk loaded hfiles, the scanner resets
* the cell seqId with the latest one, if this metadata is set as true, the reset is skipped.
* Get the last key in this store file.
*/
static final byte[] SKIP_RESET_SEQ_ID = Bytes.toBytes("SKIP_RESET_SEQ_ID");
CacheConfig getCacheConf();
Cell getFirstKey();
Cell getLastKey();
Optional<Cell> getLastKey();
/**
* Get the comparator for comparing two cells.
*/
Comparator<Cell> getComparator();
long getMaxMemstoreTS();
/**
* @return the StoreFile object associated to this StoreFile. null if the StoreFile is not a
* reference.
* Get max of the MemstoreTS in the KV's in this store file.
*/
StoreFileInfo getFileInfo();
long getMaxMemstoreTS();
/**
* @return Path or null if this StoreFile was made with a Stream.
@ -130,14 +96,11 @@ public interface StoreFile {
*/
long getMaxSequenceId();
long getModificationTimeStamp() throws IOException;
/**
* Only used by the Striped Compaction Policy
* @param key
* @return value associated with the metadata key
* Get the modification time of this store file. Usually will access the file system so throws
* IOException.
*/
byte[] getMetadataValue(byte[] key);
long getModificationTimeStamp() throws IOException;
/**
* Check if this storefile was created by bulk load. When a hfile is bulk loaded into HBase, we
@ -149,13 +112,6 @@ public interface StoreFile {
*/
boolean isBulkLoadResult();
boolean isCompactedAway();
/**
* @return true if the file is still used in reads
*/
boolean isReferencedInReads();
/**
* Return the timestamp at which this bulk load file was generated.
*/
@ -167,50 +123,18 @@ public interface StoreFile {
*/
HDFSBlocksDistribution getHDFSBlockDistribution();
/**
* Initialize the reader used for pread.
*/
void initReader() throws IOException;
/**
* Must be called after initReader.
*/
StoreFileScanner getPreadScanner(boolean cacheBlocks, long readPt, long scannerOrder,
boolean canOptimizeForNonNullColumn);
StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks,
boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn)
throws IOException;
/**
* @return Current reader. Must call initReader first else returns null.
* @see #initReader()
*/
StoreFileReader getReader();
/**
* @param evictOnClose whether to evict blocks belonging to this file
* @throws IOException
*/
void closeReader(boolean evictOnClose) throws IOException;
/**
* Marks the status of the file as compactedAway.
*/
void markCompactedAway();
/**
* Delete this file
* @throws IOException
*/
void deleteReader() throws IOException;
/**
* @return a length description of this StoreFile, suitable for debug output
*/
String toStringDetailed();
/**
* Get the min timestamp of all the cells in the store file.
*/
OptionalLong getMinimumTimestamp();
/**
* Get the max timestamp of all the cells in the store file.
*/
OptionalLong getMaximumTimestamp();
}

View File

@ -26,17 +26,17 @@ import java.util.Comparator;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Useful comparators for comparing StoreFiles.
* Useful comparators for comparing store files.
*/
@InterfaceAudience.Private
final class StoreFileComparators {
/**
* Comparator that compares based on the Sequence Ids of the the StoreFiles. Bulk loads that did
* Comparator that compares based on the Sequence Ids of the the store files. Bulk loads that did
* not request a seq ID are given a seq id of -1; thus, they are placed before all non- bulk
* loads, and bulk loads with sequence Id. Among these files, the size is used to determine the
* ordering, then bulkLoadTime. If there are ties, the path name is used as a tie-breaker.
*/
public static final Comparator<StoreFile> SEQ_ID =
public static final Comparator<HStoreFile> SEQ_ID =
Ordering.compound(ImmutableList.of(Ordering.natural().onResultOf(new GetSeqId()),
Ordering.natural().onResultOf(new GetFileSize()).reverse(),
Ordering.natural().onResultOf(new GetBulkTime()),
@ -46,23 +46,23 @@ final class StoreFileComparators {
* Comparator for time-aware compaction. SeqId is still the first ordering criterion to maintain
* MVCC.
*/
public static final Comparator<StoreFile> SEQ_ID_MAX_TIMESTAMP =
public static final Comparator<HStoreFile> SEQ_ID_MAX_TIMESTAMP =
Ordering.compound(ImmutableList.of(Ordering.natural().onResultOf(new GetSeqId()),
Ordering.natural().onResultOf(new GetMaxTimestamp()),
Ordering.natural().onResultOf(new GetFileSize()).reverse(),
Ordering.natural().onResultOf(new GetBulkTime()),
Ordering.natural().onResultOf(new GetPathName())));
private static class GetSeqId implements Function<StoreFile, Long> {
private static class GetSeqId implements Function<HStoreFile, Long> {
@Override
public Long apply(StoreFile sf) {
public Long apply(HStoreFile sf) {
return sf.getMaxSequenceId();
}
}
private static class GetFileSize implements Function<StoreFile, Long> {
private static class GetFileSize implements Function<HStoreFile, Long> {
@Override
public Long apply(StoreFile sf) {
public Long apply(HStoreFile sf) {
if (sf.getReader() != null) {
return sf.getReader().length();
} else {
@ -73,23 +73,23 @@ final class StoreFileComparators {
}
}
private static class GetBulkTime implements Function<StoreFile, Long> {
private static class GetBulkTime implements Function<HStoreFile, Long> {
@Override
public Long apply(StoreFile sf) {
public Long apply(HStoreFile sf) {
return sf.getBulkLoadTimestamp().orElse(Long.MAX_VALUE);
}
}
private static class GetPathName implements Function<StoreFile, String> {
private static class GetPathName implements Function<HStoreFile, String> {
@Override
public String apply(StoreFile sf) {
public String apply(HStoreFile sf) {
return sf.getPath().getName();
}
}
private static class GetMaxTimestamp implements Function<StoreFile, Long> {
private static class GetMaxTimestamp implements Function<HStoreFile, Long> {
@Override
public Long apply(StoreFile sf) {
public Long apply(HStoreFile sf) {
return sf.getMaximumTimestamp().orElse(Long.MAX_VALUE);
}
}

View File

@ -23,13 +23,14 @@ import java.util.Collection;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableCollection;
import java.util.Optional;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableCollection;
/**
* Manages the store files and basic metadata about that that determines the logical structure
* (e.g. what files to return for scan, how to determine split point, and such).
@ -45,13 +46,13 @@ public interface StoreFileManager {
* Loads the initial store files into empty StoreFileManager.
* @param storeFiles The files to load.
*/
void loadFiles(List<StoreFile> storeFiles);
void loadFiles(List<HStoreFile> storeFiles);
/**
* Adds new files, either for from MemStore flush or bulk insert, into the structure.
* @param sfs New store files.
*/
void insertNewFiles(Collection<StoreFile> sfs) throws IOException;
void insertNewFiles(Collection<HStoreFile> sfs) throws IOException;
/**
* Adds only the new compaction results into the structure.
@ -59,34 +60,34 @@ public interface StoreFileManager {
* @param results The resulting files for the compaction.
*/
void addCompactionResults(
Collection<StoreFile> compactedFiles, Collection<StoreFile> results) throws IOException;
Collection<HStoreFile> compactedFiles, Collection<HStoreFile> results) throws IOException;
/**
* Remove the compacted files
* @param compactedFiles the list of compacted files
* @throws IOException
*/
void removeCompactedFiles(Collection<StoreFile> compactedFiles) throws IOException;
void removeCompactedFiles(Collection<HStoreFile> compactedFiles) throws IOException;
/**
* Clears all the files currently in use and returns them.
* @return The files previously in use.
*/
ImmutableCollection<StoreFile> clearFiles();
ImmutableCollection<HStoreFile> clearFiles();
/**
* Clears all the compacted files and returns them. This method is expected to be
* accessed single threaded.
* @return The files compacted previously.
*/
Collection<StoreFile> clearCompactedFiles();
Collection<HStoreFile> clearCompactedFiles();
/**
* Gets the snapshot of the store files currently in use. Can be used for things like metrics
* and checks; should not assume anything about relations between store files in the list.
* @return The list of StoreFiles.
*/
Collection<StoreFile> getStorefiles();
Collection<HStoreFile> getStorefiles();
/**
* List of compacted files inside this store that needs to be excluded in reads
@ -95,7 +96,7 @@ public interface StoreFileManager {
* compacted files are done.
* @return the list of compacted files
*/
Collection<StoreFile> getCompactedfiles();
Collection<HStoreFile> getCompactedfiles();
/**
* Returns the number of files currently in use.
@ -115,7 +116,7 @@ public interface StoreFileManager {
* @param stopRow Stop row of the request.
* @return The list of files that are to be read for this request.
*/
Collection<StoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow, byte[] stopRow,
Collection<HStoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow, byte[] stopRow,
boolean includeStopRow);
/**
@ -124,9 +125,7 @@ public interface StoreFileManager {
* @return The files that may have the key less than or equal to targetKey, in reverse
* order of new-ness, and preference for target key.
*/
Iterator<StoreFile> getCandidateFilesForRowKeyBefore(
KeyValue targetKey
);
Iterator<HStoreFile> getCandidateFilesForRowKeyBefore(KeyValue targetKey);
/**
* Updates the candidate list for finding row key before. Based on the list of candidates
@ -139,17 +138,16 @@ public interface StoreFileManager {
* @param candidate The current best candidate found.
* @return The list to replace candidateFiles.
*/
Iterator<StoreFile> updateCandidateFilesForRowKeyBefore(
Iterator<StoreFile> candidateFiles, KeyValue targetKey, Cell candidate
);
Iterator<HStoreFile> updateCandidateFilesForRowKeyBefore(Iterator<HStoreFile> candidateFiles,
KeyValue targetKey, Cell candidate);
/**
* Gets the split point for the split of this set of store files (approx. middle).
* @return The mid-point, or null if no split is possible.
* @return The mid-point if possible.
* @throws IOException
*/
byte[] getSplitPoint() throws IOException;
Optional<byte[]> getSplitPoint() throws IOException;
/**
* @return The store compaction priority.
@ -161,7 +159,7 @@ public interface StoreFileManager {
* @param filesCompacting Files that are currently compacting.
* @return The files which don't have any necessary data according to TTL and other criteria.
*/
Collection<StoreFile> getUnneededFiles(long maxTs, List<StoreFile> filesCompacting);
Collection<HStoreFile> getUnneededFiles(long maxTs, List<HStoreFile> filesCompacting);
/**
* @return the compaction pressure used for compaction throughput tuning.
@ -171,7 +169,7 @@ public interface StoreFileManager {
/**
* @return the comparator used to sort storefiles. Usually, the
* {@link StoreFile#getMaxSequenceId()} is the first priority.
* {@link HStoreFile#getMaxSequenceId()} is the first priority.
*/
Comparator<StoreFile> getStoreFileComparator();
Comparator<HStoreFile> getStoreFileComparator();
}

View File

@ -1,5 +1,4 @@
/*
*
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@ -18,9 +17,14 @@
*/
package org.apache.hadoop.hbase.regionserver;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.DELETE_FAMILY_COUNT;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.LAST_BLOOM_KEY;
import java.io.DataInput;
import java.io.IOException;
import java.util.Map;
import java.util.Optional;
import java.util.SortedSet;
import java.util.concurrent.atomic.AtomicInteger;
@ -414,40 +418,40 @@ public class StoreFileReader {
* @return true if there is overlap, false otherwise
*/
public boolean passesKeyRangeFilter(Scan scan) {
if (this.getFirstKey() == null || this.getLastKey() == null) {
Optional<Cell> firstKeyKV = this.getFirstKey();
Optional<Cell> lastKeyKV = this.getLastKey();
if (!firstKeyKV.isPresent() || !lastKeyKV.isPresent()) {
// the file is empty
return false;
}
if (Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW)
&& Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
if (Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW) &&
Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
return true;
}
byte[] smallestScanRow = scan.isReversed() ? scan.getStopRow() : scan.getStartRow();
byte[] largestScanRow = scan.isReversed() ? scan.getStartRow() : scan.getStopRow();
Cell firstKeyKV = this.getFirstKey();
Cell lastKeyKV = this.getLastKey();
boolean nonOverLapping = (getComparator().compareRows(firstKeyKV,
largestScanRow, 0, largestScanRow.length) > 0
&& !Bytes
.equals(scan.isReversed() ? scan.getStartRow() : scan.getStopRow(),
HConstants.EMPTY_END_ROW))
|| getComparator().compareRows(lastKeyKV, smallestScanRow, 0, smallestScanRow.length) < 0;
boolean nonOverLapping = (getComparator()
.compareRows(firstKeyKV.get(), largestScanRow, 0, largestScanRow.length) > 0 &&
!Bytes.equals(scan.isReversed() ? scan.getStartRow() : scan.getStopRow(),
HConstants.EMPTY_END_ROW)) ||
getComparator().compareRows(lastKeyKV.get(), smallestScanRow, 0,
smallestScanRow.length) < 0;
return !nonOverLapping;
}
public Map<byte[], byte[]> loadFileInfo() throws IOException {
Map<byte [], byte []> fi = reader.loadFileInfo();
byte[] b = fi.get(StoreFile.BLOOM_FILTER_TYPE_KEY);
byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
if (b != null) {
bloomFilterType = BloomType.valueOf(Bytes.toString(b));
}
lastBloomKey = fi.get(StoreFile.LAST_BLOOM_KEY);
lastBloomKey = fi.get(LAST_BLOOM_KEY);
if(bloomFilterType == BloomType.ROWCOL) {
lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue(lastBloomKey, 0, lastBloomKey.length);
}
byte[] cnt = fi.get(StoreFile.DELETE_FAMILY_COUNT);
byte[] cnt = fi.get(DELETE_FAMILY_COUNT);
if (cnt != null) {
deleteFamilyCnt = Bytes.toLong(cnt);
}
@ -537,16 +541,16 @@ public class StoreFileReader {
this.deleteFamilyBloomFilter = null;
}
public Cell getLastKey() {
public Optional<Cell> getLastKey() {
return reader.getLastKey();
}
public byte[] getLastRowKey() {
public Optional<byte[]> getLastRowKey() {
return reader.getLastRowKey();
}
public Cell midkey() throws IOException {
return reader.midkey();
public Optional<Cell> midKey() throws IOException {
return reader.midKey();
}
public long length() {
@ -565,7 +569,7 @@ public class StoreFileReader {
return deleteFamilyCnt;
}
public Cell getFirstKey() {
public Optional<Cell> getFirstKey() {
return reader.getFirstKey();
}

View File

@ -25,6 +25,7 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.PriorityQueue;
import java.util.concurrent.atomic.LongAdder;
@ -102,7 +103,7 @@ public class StoreFileScanner implements KeyValueScanner {
/**
* Return an array of scanners corresponding to the given set of store files.
*/
public static List<StoreFileScanner> getScannersForStoreFiles(Collection<StoreFile> files,
public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreFile> files,
boolean cacheBlocks, boolean usePread, long readPt) throws IOException {
return getScannersForStoreFiles(files, cacheBlocks, usePread, false, false, readPt);
}
@ -110,7 +111,7 @@ public class StoreFileScanner implements KeyValueScanner {
/**
* Return an array of scanners corresponding to the given set of store files.
*/
public static List<StoreFileScanner> getScannersForStoreFiles(Collection<StoreFile> files,
public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreFile> files,
boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean useDropBehind,
long readPt) throws IOException {
return getScannersForStoreFiles(files, cacheBlocks, usePread, isCompaction, useDropBehind, null,
@ -121,7 +122,7 @@ public class StoreFileScanner implements KeyValueScanner {
* Return an array of scanners corresponding to the given set of store files, And set the
* ScanQueryMatcher for each store file scanner for further optimization
*/
public static List<StoreFileScanner> getScannersForStoreFiles(Collection<StoreFile> files,
public static List<StoreFileScanner> getScannersForStoreFiles(Collection<HStoreFile> files,
boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean canUseDrop,
ScanQueryMatcher matcher, long readPt) throws IOException {
if (files.isEmpty()) {
@ -129,15 +130,15 @@ public class StoreFileScanner implements KeyValueScanner {
}
List<StoreFileScanner> scanners = new ArrayList<>(files.size());
boolean canOptimizeForNonNullColumn = matcher != null ? !matcher.hasNullColumnInQuery() : false;
PriorityQueue<StoreFile> sortedFiles =
PriorityQueue<HStoreFile> sortedFiles =
new PriorityQueue<>(files.size(), StoreFileComparators.SEQ_ID);
for (StoreFile file : files) {
for (HStoreFile file : files) {
// The sort function needs metadata so we need to open reader first before sorting the list.
file.initReader();
sortedFiles.add(file);
}
for (int i = 0, n = files.size(); i < n; i++) {
StoreFile sf = sortedFiles.remove();
HStoreFile sf = sortedFiles.remove();
StoreFileScanner scanner;
if (usePread) {
scanner = sf.getPreadScanner(cacheBlocks, readPt, i, canOptimizeForNonNullColumn);
@ -154,10 +155,10 @@ public class StoreFileScanner implements KeyValueScanner {
* Get scanners for compaction. We will create a separated reader for each store file to avoid
* contention with normal read request.
*/
public static List<StoreFileScanner> getScannersForCompaction(Collection<StoreFile> files,
public static List<StoreFileScanner> getScannersForCompaction(Collection<HStoreFile> files,
boolean canUseDropBehind, long readPt) throws IOException {
List<StoreFileScanner> scanners = new ArrayList<>(files.size());
List<StoreFile> sortedFiles = new ArrayList<>(files);
List<HStoreFile> sortedFiles = new ArrayList<>(files);
Collections.sort(sortedFiles, StoreFileComparators.SEQ_ID);
boolean succ = false;
try {
@ -537,12 +538,11 @@ public class StoreFileScanner implements KeyValueScanner {
@Override
public boolean seekToLastRow() throws IOException {
byte[] lastRow = reader.getLastRowKey();
if (lastRow == null) {
Optional<byte[]> lastRow = reader.getLastRowKey();
if (!lastRow.isPresent()) {
return false;
}
Cell seekKey = CellUtil
.createFirstOnRow(lastRow, 0, (short) lastRow.length);
Cell seekKey = CellUtil.createFirstOnRow(lastRow.get());
if (seek(seekKey)) {
return true;
} else {

View File

@ -1,5 +1,4 @@
/*
*
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@ -18,7 +17,13 @@
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.DELETE_FAMILY_COUNT;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.EARLIEST_PUT_TS;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAX_SEQ_ID_KEY;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.MOB_CELLS_COUNT;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY;
import java.io.IOException;
import java.net.InetSocketAddress;
@ -33,10 +38,9 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
@ -48,6 +52,9 @@ import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.RowBloomContext;
import org.apache.hadoop.hbase.util.RowColBloomContext;
import org.apache.hadoop.io.WritableUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
/**
* A StoreFile writer. Use this to read/write HBase Store Files. It is package
@ -185,10 +192,9 @@ public class StoreFileWriter implements CellSink, ShipperListener {
* @throws IOException problem writing to FS
*/
public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
throws IOException {
writer.appendFileInfo(StoreFile.MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
writer.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY,
Bytes.toBytes(majorCompaction));
throws IOException {
writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
writer.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction));
appendTrackedTimestampsToMetadata();
}
@ -202,9 +208,9 @@ public class StoreFileWriter implements CellSink, ShipperListener {
*/
public void appendMetadata(final long maxSequenceId, final boolean majorCompaction,
final long mobCellsCount) throws IOException {
writer.appendFileInfo(StoreFile.MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
writer.appendFileInfo(StoreFile.MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction));
writer.appendFileInfo(StoreFile.MOB_CELLS_COUNT, Bytes.toBytes(mobCellsCount));
writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
writer.appendFileInfo(MAJOR_COMPACTION_KEY, Bytes.toBytes(majorCompaction));
writer.appendFileInfo(MOB_CELLS_COUNT, Bytes.toBytes(mobCellsCount));
appendTrackedTimestampsToMetadata();
}
@ -212,8 +218,8 @@ public class StoreFileWriter implements CellSink, ShipperListener {
* Add TimestampRange and earliest put timestamp to Metadata
*/
public void appendTrackedTimestampsToMetadata() throws IOException {
appendFileInfo(StoreFile.TIMERANGE_KEY, WritableUtils.toByteArray(timeRangeTracker));
appendFileInfo(StoreFile.EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
appendFileInfo(TIMERANGE_KEY, WritableUtils.toByteArray(timeRangeTracker));
appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
}
/**
@ -310,8 +316,7 @@ public class StoreFileWriter implements CellSink, ShipperListener {
// add the general Bloom filter writer and append file info
if (hasGeneralBloom) {
writer.addGeneralBloomFilter(generalBloomFilterWriter);
writer.appendFileInfo(StoreFile.BLOOM_FILTER_TYPE_KEY,
Bytes.toBytes(bloomType.toString()));
writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY, Bytes.toBytes(bloomType.toString()));
bloomContext.addLastBloomKey(writer);
}
return hasGeneralBloom;
@ -327,8 +332,7 @@ public class StoreFileWriter implements CellSink, ShipperListener {
// append file info about the number of delete family kvs
// even if there is no delete family Bloom.
writer.appendFileInfo(StoreFile.DELETE_FAMILY_COUNT,
Bytes.toBytes(this.deleteFamilyCnt));
writer.appendFileInfo(DELETE_FAMILY_COUNT, Bytes.toBytes(this.deleteFamilyCnt));
return hasDeleteFamilyBloom;
}
@ -501,7 +505,7 @@ public class StoreFileWriter implements CellSink, ShipperListener {
}
// set block storage policy for temp path
String policyName = this.conf.get(HColumnDescriptor.STORAGE_POLICY);
String policyName = this.conf.get(ColumnFamilyDescriptorBuilder.STORAGE_POLICY);
if (null == policyName) {
policyName = this.conf.get(HStore.BLOCK_STORAGE_POLICY_KEY);
}

View File

@ -29,10 +29,10 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Store flusher interface. Turns a snapshot of memstore into a set of store files (usually one).
@ -41,9 +41,9 @@ import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
@InterfaceAudience.Private
abstract class StoreFlusher {
protected Configuration conf;
protected Store store;
protected HStore store;
public StoreFlusher(Configuration conf, Store store) {
public StoreFlusher(Configuration conf, HStore store) {
this.conf = conf;
this.store = store;
}

View File

@ -16,7 +16,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
@ -38,7 +37,6 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.IsolationLevel;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.executor.ExecutorService;
@ -51,6 +49,7 @@ import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
import org.apache.hadoop.hbase.regionserver.querymatcher.UserScanQueryMatcher;
import org.apache.hadoop.hbase.util.CollectionUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
@ -67,7 +66,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
implements KeyValueScanner, InternalScanner, ChangedReadersObserver {
private static final Log LOG = LogFactory.getLog(StoreScanner.class);
// In unit tests, the store could be null
protected final Optional<Store> store;
protected final Optional<HStore> store;
private ScanQueryMatcher matcher;
protected KeyValueHeap heap;
private boolean cacheBlocks;
@ -147,7 +146,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
// Indicates whether there was flush during the course of the scan
private volatile boolean flushed = false;
// generally we get one file from a flush
private final List<StoreFile> flushedStoreFiles = new ArrayList<>(1);
private final List<HStoreFile> flushedStoreFiles = new ArrayList<>(1);
// Since CompactingMemstore is now default, we get three memstore scanners from a flush
private final List<KeyValueScanner> memStoreScannersAfterFlush = new ArrayList<>(3);
// The current list of scanners
@ -160,7 +159,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
private boolean topChanged = false;
/** An internal constructor. */
private StoreScanner(Optional<Store> store, Scan scan, ScanInfo scanInfo,
private StoreScanner(Optional<HStore> store, Scan scan, ScanInfo scanInfo,
int numColumns, long readPt, boolean cacheBlocks, ScanType scanType) {
this.readPt = readPt;
this.store = store;
@ -223,7 +222,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
* @param columns which columns we are scanning
* @throws IOException
*/
public StoreScanner(Store store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
public StoreScanner(HStore store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
long readPt) throws IOException {
this(Optional.of(store), scan, scanInfo, columns != null ? columns.size() : 0, readPt,
scan.getCacheBlocks(), ScanType.USER_SCAN);
@ -275,7 +274,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
* @param scanners ancillary scanners
* @param smallestReadPoint the readPoint that we should use for tracking versions
*/
public StoreScanner(Store store, ScanInfo scanInfo, OptionalInt maxVersions,
public StoreScanner(HStore store, ScanInfo scanInfo, OptionalInt maxVersions,
List<? extends KeyValueScanner> scanners, ScanType scanType, long smallestReadPoint,
long earliestPutTs) throws IOException {
this(store, scanInfo, maxVersions, scanners, scanType, smallestReadPoint, earliestPutTs, null,
@ -292,20 +291,20 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
* @param dropDeletesFromRow The inclusive left bound of the range; can be EMPTY_START_ROW.
* @param dropDeletesToRow The exclusive right bound of the range; can be EMPTY_END_ROW.
*/
public StoreScanner(Store store, ScanInfo scanInfo, OptionalInt maxVersions,
public StoreScanner(HStore store, ScanInfo scanInfo, OptionalInt maxVersions,
List<? extends KeyValueScanner> scanners, long smallestReadPoint, long earliestPutTs,
byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException {
this(store, scanInfo, maxVersions, scanners, ScanType.COMPACT_RETAIN_DELETES, smallestReadPoint,
earliestPutTs, dropDeletesFromRow, dropDeletesToRow);
}
private StoreScanner(Store store, ScanInfo scanInfo, OptionalInt maxVersions,
private StoreScanner(HStore store, ScanInfo scanInfo, OptionalInt maxVersions,
List<? extends KeyValueScanner> scanners, ScanType scanType, long smallestReadPoint,
long earliestPutTs, byte[] dropDeletesFromRow, byte[] dropDeletesToRow) throws IOException {
this(Optional.of(store),
maxVersions.isPresent() ? new Scan().readVersions(maxVersions.getAsInt())
: SCAN_FOR_COMPACTION,
scanInfo, 0, ((HStore) store).getHRegion().getReadPoint(IsolationLevel.READ_COMMITTED),
scanInfo, 0, store.getHRegion().getReadPoint(IsolationLevel.READ_COMMITTED),
false, scanType);
assert scanType != ScanType.USER_SCAN;
matcher =
@ -844,9 +843,9 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
// Implementation of ChangedReadersObserver
@Override
public void updateReaders(List<StoreFile> sfs, List<KeyValueScanner> memStoreScanners) throws IOException {
if (CollectionUtils.isEmpty(sfs)
&& CollectionUtils.isEmpty(memStoreScanners)) {
public void updateReaders(List<HStoreFile> sfs, List<KeyValueScanner> memStoreScanners)
throws IOException {
if (CollectionUtils.isEmpty(sfs) && CollectionUtils.isEmpty(memStoreScanners)) {
return;
}
flushLock.lock();
@ -868,7 +867,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
*/
protected final boolean reopenAfterFlush() throws IOException {
// here we can make sure that we have a Store instance.
Store store = this.store.get();
HStore store = this.store.get();
Cell lastTop = heap.peek();
// When we have the scan object, should we not pass it to getScanners() to get a limited set of
// scanners? We did so in the constructor and we could have done it now by storing the scan
@ -996,7 +995,7 @@ public class StoreScanner extends NonReversedNonLazyKeyValueScanner
List<KeyValueScanner> newCurrentScanners;
KeyValueHeap newHeap;
// We must have a store instance here
Store store = this.store.get();
HStore store = this.store.get();
try {
// recreate the scanners on the current file scanners
fileScanners = store.recreateScanners(scannersToClose, cacheBlocks, false, false,

View File

@ -41,7 +41,7 @@ public class StoreUtils {
/**
* Creates a deterministic hash code for store file collection.
*/
public static OptionalInt getDeterministicRandomSeed(Collection<StoreFile> files) {
public static OptionalInt getDeterministicRandomSeed(Collection<HStoreFile> files) {
return files.stream().mapToInt(f -> f.getPath().getName().hashCode()).findFirst();
}
@ -49,24 +49,17 @@ public class StoreUtils {
* Determines whether any files in the collection are references.
* @param files The files.
*/
public static boolean hasReferences(final Collection<StoreFile> files) {
if (files != null) {
for (StoreFile hsf: files) {
if (hsf.isReference()) {
return true;
}
}
}
return false;
public static boolean hasReferences(Collection<HStoreFile> files) {
// TODO: make sure that we won't pass null here in the future.
return files != null ? files.stream().anyMatch(HStoreFile::isReference) : false;
}
/**
* Gets lowest timestamp from candidate StoreFiles
*/
public static long getLowestTimestamp(final Collection<StoreFile> candidates)
throws IOException {
public static long getLowestTimestamp(Collection<HStoreFile> candidates) throws IOException {
long minTs = Long.MAX_VALUE;
for (StoreFile storeFile : candidates) {
for (HStoreFile storeFile : candidates) {
minTs = Math.min(minTs, storeFile.getModificationTimeStamp());
}
return minTs;
@ -77,7 +70,7 @@ public class StoreUtils {
* @param candidates The files to choose from.
* @return The largest file; null if no file has a reader.
*/
static Optional<StoreFile> getLargestFile(Collection<StoreFile> candidates) {
static Optional<HStoreFile> getLargestFile(Collection<HStoreFile> candidates) {
return candidates.stream().filter(f -> f.getReader() != null)
.max((f1, f2) -> Long.compare(f1.getReader().length(), f2.getReader().length()));
}
@ -89,29 +82,19 @@ public class StoreUtils {
* @return 0 if no non-bulk-load files are provided or, this is Store that does not yet have any
* store files.
*/
public static long getMaxMemstoreTSInList(Collection<StoreFile> sfs) {
long max = 0;
for (StoreFile sf : sfs) {
if (!sf.isBulkLoadResult()) {
max = Math.max(max, sf.getMaxMemstoreTS());
}
}
return max;
public static long getMaxMemstoreTSInList(Collection<HStoreFile> sfs) {
return sfs.stream().filter(sf -> !sf.isBulkLoadResult()).mapToLong(HStoreFile::getMaxMemstoreTS)
.max().orElse(0L);
}
/**
* Return the highest sequence ID found across all storefiles in
* the given list.
* Return the highest sequence ID found across all storefiles in the given list.
* @param sfs
* @return 0 if no non-bulk-load files are provided or, this is Store that
* does not yet have any store files.
* @return 0 if no non-bulk-load files are provided or, this is Store that does not yet have any
* store files.
*/
public static long getMaxSequenceIdInList(Collection<StoreFile> sfs) {
long max = 0;
for (StoreFile sf : sfs) {
max = Math.max(max, sf.getMaxSequenceId());
}
return max;
public static long getMaxSequenceIdInList(Collection<HStoreFile> sfs) {
return sfs.stream().mapToLong(HStoreFile::getMaxSequenceId).max().orElse(0L);
}
/**
@ -120,7 +103,7 @@ public class StoreUtils {
* @param comparator Comparator used to compare KVs.
* @return The split point row, or null if splitting is not possible, or reader is null.
*/
static Optional<byte[]> getFileSplitPoint(StoreFile file, CellComparator comparator)
static Optional<byte[]> getFileSplitPoint(HStoreFile file, CellComparator comparator)
throws IOException {
StoreFileReader reader = file.getReader();
if (reader == null) {
@ -130,20 +113,31 @@ public class StoreUtils {
// Get first, last, and mid keys. Midkey is the key that starts block
// in middle of hfile. Has column and timestamp. Need to return just
// the row we want to split on as midkey.
Cell midkey = reader.midkey();
if (midkey != null) {
Cell firstKey = reader.getFirstKey();
Cell lastKey = reader.getLastKey();
// if the midkey is the same as the first or last keys, we cannot (ever) split this region.
if (comparator.compareRows(midkey, firstKey) == 0 ||
comparator.compareRows(midkey, lastKey) == 0) {
if (LOG.isDebugEnabled()) {
LOG.debug("cannot split because midkey is the same as first or last row");
}
return Optional.empty();
}
return Optional.of(CellUtil.cloneRow(midkey));
Optional<Cell> optionalMidKey = reader.midKey();
if (!optionalMidKey.isPresent()) {
return Optional.empty();
}
return Optional.empty();
Cell midKey = optionalMidKey.get();
Cell firstKey = reader.getFirstKey().get();
Cell lastKey = reader.getLastKey().get();
// if the midkey is the same as the first or last keys, we cannot (ever) split this region.
if (comparator.compareRows(midKey, firstKey) == 0 ||
comparator.compareRows(midKey, lastKey) == 0) {
if (LOG.isDebugEnabled()) {
LOG.debug("cannot split because midkey is the same as first or last row");
}
return Optional.empty();
}
return Optional.of(CellUtil.cloneRow(midKey));
}
/**
* Gets the mid point of the largest file passed in as split point.
*/
static Optional<byte[]> getSplitPoint(Collection<HStoreFile> storefiles,
CellComparator comparator) throws IOException {
Optional<HStoreFile> largestFile = StoreUtils.getLargestFile(storefiles);
return largestFile.isPresent() ? StoreUtils.getFileSplitPoint(largestFile.get(), comparator)
: Optional.empty();
}
}

View File

@ -47,7 +47,7 @@ public class StripeStoreEngine extends StoreEngine<StripeStoreFlusher,
private StripeStoreConfig config;
@Override
public boolean needsCompaction(List<StoreFile> filesCompacting) {
public boolean needsCompaction(List<HStoreFile> filesCompacting) {
return this.compactionPolicy.needsCompactions(this.storeFileManager, filesCompacting);
}
@ -58,7 +58,7 @@ public class StripeStoreEngine extends StoreEngine<StripeStoreFlusher,
@Override
protected void createComponents(
Configuration conf, Store store, CellComparator comparator) throws IOException {
Configuration conf, HStore store, CellComparator comparator) throws IOException {
this.config = new StripeStoreConfig(conf, store);
this.compactionPolicy = new StripeCompactionPolicy(conf, store, config);
this.storeFileManager = new StripeStoreFileManager(comparator, conf, this.config);
@ -74,12 +74,12 @@ public class StripeStoreEngine extends StoreEngine<StripeStoreFlusher,
private StripeCompactionPolicy.StripeCompactionRequest stripeRequest = null;
@Override
public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) {
return compactionPolicy.preSelectFilesForCoprocessor(storeFileManager, filesCompacting);
}
@Override
public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
public boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction,
boolean mayUseOffPeak, boolean forceMajor) throws IOException {
this.stripeRequest = compactionPolicy.selectCompaction(
storeFileManager, filesCompacting, mayUseOffPeak);

View File

@ -39,11 +39,11 @@ import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ConcatenatedLists;
import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableCollection;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
@ -100,19 +100,19 @@ public class StripeStoreFileManager
* same index, except the last one. Inside each list, the files are in reverse order by
* seqNum. Note that the length of this is one higher than that of stripeEndKeys.
*/
public ArrayList<ImmutableList<StoreFile>> stripeFiles = new ArrayList<>();
public ArrayList<ImmutableList<HStoreFile>> stripeFiles = new ArrayList<>();
/** Level 0. The files are in reverse order by seqNum. */
public ImmutableList<StoreFile> level0Files = ImmutableList.<StoreFile>of();
public ImmutableList<HStoreFile> level0Files = ImmutableList.of();
/** Cached list of all files in the structure, to return from some calls */
public ImmutableList<StoreFile> allFilesCached = ImmutableList.<StoreFile>of();
private ImmutableList<StoreFile> allCompactedFilesCached = ImmutableList.<StoreFile>of();
public ImmutableList<HStoreFile> allFilesCached = ImmutableList.of();
private ImmutableList<HStoreFile> allCompactedFilesCached = ImmutableList.of();
}
private State state = null;
/** Cached file metadata (or overrides as the case may be) */
private HashMap<StoreFile, byte[]> fileStarts = new HashMap<>();
private HashMap<StoreFile, byte[]> fileEnds = new HashMap<>();
private HashMap<HStoreFile, byte[]> fileStarts = new HashMap<>();
private HashMap<HStoreFile, byte[]> fileEnds = new HashMap<>();
/** Normally invalid key is null, but in the map null is the result for "no key"; so use
* the following constant value in these maps instead. Note that this is a constant and
* we use it to compare by reference when we read from the map. */
@ -132,17 +132,17 @@ public class StripeStoreFileManager
}
@Override
public void loadFiles(List<StoreFile> storeFiles) {
public void loadFiles(List<HStoreFile> storeFiles) {
loadUnclassifiedStoreFiles(storeFiles);
}
@Override
public Collection<StoreFile> getStorefiles() {
public Collection<HStoreFile> getStorefiles() {
return state.allFilesCached;
}
@Override
public Collection<StoreFile> getCompactedfiles() {
public Collection<HStoreFile> getCompactedfiles() {
return state.allCompactedFilesCached;
}
@ -152,7 +152,7 @@ public class StripeStoreFileManager
}
@Override
public void insertNewFiles(Collection<StoreFile> sfs) throws IOException {
public void insertNewFiles(Collection<HStoreFile> sfs) throws IOException {
CompactionOrFlushMergeCopy cmc = new CompactionOrFlushMergeCopy(true);
// Passing null does not cause NPE??
cmc.mergeResults(null, sfs);
@ -160,8 +160,8 @@ public class StripeStoreFileManager
}
@Override
public ImmutableCollection<StoreFile> clearFiles() {
ImmutableCollection<StoreFile> result = state.allFilesCached;
public ImmutableCollection<HStoreFile> clearFiles() {
ImmutableCollection<HStoreFile> result = state.allFilesCached;
this.state = new State();
this.fileStarts.clear();
this.fileEnds.clear();
@ -169,8 +169,8 @@ public class StripeStoreFileManager
}
@Override
public ImmutableCollection<StoreFile> clearCompactedFiles() {
ImmutableCollection<StoreFile> result = state.allCompactedFilesCached;
public ImmutableCollection<HStoreFile> clearCompactedFiles() {
ImmutableCollection<HStoreFile> result = state.allCompactedFilesCached;
this.state = new State();
return result;
}
@ -183,7 +183,7 @@ public class StripeStoreFileManager
/** See {@link StoreFileManager#getCandidateFilesForRowKeyBefore(KeyValue)}
* for details on this methods. */
@Override
public Iterator<StoreFile> getCandidateFilesForRowKeyBefore(final KeyValue targetKey) {
public Iterator<HStoreFile> getCandidateFilesForRowKeyBefore(final KeyValue targetKey) {
KeyBeforeConcatenatedLists result = new KeyBeforeConcatenatedLists();
// Order matters for this call.
result.addSublist(state.level0Files);
@ -200,14 +200,14 @@ public class StripeStoreFileManager
* {@link StoreFileManager#updateCandidateFilesForRowKeyBefore(Iterator, KeyValue, Cell)}
* for details on this methods. */
@Override
public Iterator<StoreFile> updateCandidateFilesForRowKeyBefore(
Iterator<StoreFile> candidateFiles, final KeyValue targetKey, final Cell candidate) {
public Iterator<HStoreFile> updateCandidateFilesForRowKeyBefore(
Iterator<HStoreFile> candidateFiles, final KeyValue targetKey, final Cell candidate) {
KeyBeforeConcatenatedLists.Iterator original =
(KeyBeforeConcatenatedLists.Iterator)candidateFiles;
assert original != null;
ArrayList<List<StoreFile>> components = original.getComponents();
ArrayList<List<HStoreFile>> components = original.getComponents();
for (int firstIrrelevant = 0; firstIrrelevant < components.size(); ++firstIrrelevant) {
StoreFile sf = components.get(firstIrrelevant).get(0);
HStoreFile sf = components.get(firstIrrelevant).get(0);
byte[] endKey = endOf(sf);
// Entries are ordered as such: L0, then stripes in reverse order. We never remove
// level 0; we remove the stripe, and all subsequent ones, as soon as we find the
@ -221,12 +221,6 @@ public class StripeStoreFileManager
return original;
}
private byte[] getSplitPoint(Collection<StoreFile> sfs) throws IOException {
Optional<StoreFile> largestFile = StoreUtils.getLargestFile(sfs);
return largestFile.isPresent()
? StoreUtils.getFileSplitPoint(largestFile.get(), cellComparator).orElse(null) : null;
}
/**
* Override of getSplitPoint that determines the split point as the boundary between two
* stripes, unless it causes significant imbalance between split sides' sizes. In that
@ -235,8 +229,10 @@ public class StripeStoreFileManager
* @return The split point, or null if no split is possible.
*/
@Override
public byte[] getSplitPoint() throws IOException {
if (this.getStorefileCount() == 0) return null;
public Optional<byte[]> getSplitPoint() throws IOException {
if (this.getStorefileCount() == 0) {
return Optional.empty();
}
if (state.stripeFiles.size() <= 1) {
return getSplitPointFromAllFiles();
}
@ -265,7 +261,9 @@ public class StripeStoreFileManager
if (ratio < 1) {
ratio = 1 / ratio;
}
if (config.getMaxSplitImbalance() > ratio) return state.stripeEndRows[leftIndex];
if (config.getMaxSplitImbalance() > ratio) {
return Optional.of(state.stripeEndRows[leftIndex]);
}
// If the difference between the sides is too large, we could get the proportional key on
// the a stripe to equalize the difference, but there's no proportional key method at the
@ -278,18 +276,21 @@ public class StripeStoreFileManager
if (newRatio < 1) {
newRatio = 1 / newRatio;
}
if (newRatio >= ratio) return state.stripeEndRows[leftIndex];
if (newRatio >= ratio) {
return Optional.of(state.stripeEndRows[leftIndex]);
}
LOG.debug("Splitting the stripe - ratio w/o split " + ratio + ", ratio with split "
+ newRatio + " configured ratio " + config.getMaxSplitImbalance());
// Ok, we may get better ratio, get it.
return getSplitPoint(state.stripeFiles.get(isRightLarger ? rightIndex : leftIndex));
// OK, we may get better ratio, get it.
return StoreUtils.getSplitPoint(state.stripeFiles.get(isRightLarger ? rightIndex : leftIndex),
cellComparator);
}
private byte[] getSplitPointFromAllFiles() throws IOException {
ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<>();
private Optional<byte[]> getSplitPointFromAllFiles() throws IOException {
ConcatenatedLists<HStoreFile> sfs = new ConcatenatedLists<>();
sfs.addSublist(state.level0Files);
sfs.addAllSublists(state.stripeFiles);
return getSplitPoint(sfs);
return StoreUtils.getSplitPoint(sfs, cellComparator);
}
private double getMidStripeSplitRatio(long smallerSize, long largerSize, long lastLargerSize) {
@ -297,7 +298,7 @@ public class StripeStoreFileManager
}
@Override
public Collection<StoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow,
public Collection<HStoreFile> getFilesForScan(byte[] startRow, boolean includeStartRow,
byte[] stopRow, boolean includeStopRow) {
if (state.stripeFiles.isEmpty()) {
return state.level0Files; // There's just L0.
@ -313,7 +314,7 @@ public class StripeStoreFileManager
return state.allFilesCached; // We need to read all files.
}
ConcatenatedLists<StoreFile> result = new ConcatenatedLists<>();
ConcatenatedLists<HStoreFile> result = new ConcatenatedLists<>();
result.addAllSublists(state.stripeFiles.subList(firstStripe, lastStripe + 1));
result.addSublist(state.level0Files);
return result;
@ -321,7 +322,7 @@ public class StripeStoreFileManager
@Override
public void addCompactionResults(
Collection<StoreFile> compactedFiles, Collection<StoreFile> results) throws IOException {
Collection<HStoreFile> compactedFiles, Collection<HStoreFile> results) throws IOException {
// See class comment for the assumptions we make here.
LOG.debug("Attempting to merge compaction results: " + compactedFiles.size()
+ " files replaced by " + results.size());
@ -337,14 +338,14 @@ public class StripeStoreFileManager
// Let a background thread close the actual reader on these compacted files and also
// ensure to evict the blocks from block cache so that they are no longer in
// cache
private void markCompactedAway(Collection<StoreFile> compactedFiles) {
for (StoreFile file : compactedFiles) {
private void markCompactedAway(Collection<HStoreFile> compactedFiles) {
for (HStoreFile file : compactedFiles) {
file.markCompactedAway();
}
}
@Override
public void removeCompactedFiles(Collection<StoreFile> compactedFiles) throws IOException {
public void removeCompactedFiles(Collection<HStoreFile> compactedFiles) throws IOException {
// See class comment for the assumptions we make here.
LOG.debug("Attempting to delete compaction results: " + compactedFiles.size());
// In order to be able to fail in the middle of the operation, we'll operate on lazy
@ -378,7 +379,7 @@ public class StripeStoreFileManager
*/
private long getStripeFilesSize(int stripeIndex) {
long result = 0;
for (StoreFile sf : state.stripeFiles.get(stripeIndex)) {
for (HStoreFile sf : state.stripeFiles.get(stripeIndex)) {
result += sf.getReader().length();
}
return result;
@ -391,13 +392,13 @@ public class StripeStoreFileManager
* go to level 0.
* @param storeFiles Store files to add.
*/
private void loadUnclassifiedStoreFiles(List<StoreFile> storeFiles) {
private void loadUnclassifiedStoreFiles(List<HStoreFile> storeFiles) {
LOG.debug("Attempting to load " + storeFiles.size() + " store files.");
TreeMap<byte[], ArrayList<StoreFile>> candidateStripes = new TreeMap<>(MAP_COMPARATOR);
ArrayList<StoreFile> level0Files = new ArrayList<>();
TreeMap<byte[], ArrayList<HStoreFile>> candidateStripes = new TreeMap<>(MAP_COMPARATOR);
ArrayList<HStoreFile> level0Files = new ArrayList<>();
// Separate the files into tentative stripes; then validate. Currently, we rely on metadata.
// If needed, we could dynamically determine the stripes in future.
for (StoreFile sf : storeFiles) {
for (HStoreFile sf : storeFiles) {
byte[] startRow = startOf(sf), endRow = endOf(sf);
// Validate the range and put the files into place.
if (isInvalid(startRow) || isInvalid(endRow)) {
@ -410,7 +411,7 @@ public class StripeStoreFileManager
insertFileIntoStripe(level0Files, sf); // Bad metadata - goes to L0 also.
ensureLevel0Metadata(sf);
} else {
ArrayList<StoreFile> stripe = candidateStripes.get(endRow);
ArrayList<HStoreFile> stripe = candidateStripes.get(endRow);
if (stripe == null) {
stripe = new ArrayList<>();
candidateStripes.put(endRow, stripe);
@ -423,14 +424,14 @@ public class StripeStoreFileManager
boolean hasOverlaps = false;
byte[] expectedStartRow = null; // first stripe can start wherever
Iterator<Map.Entry<byte[], ArrayList<StoreFile>>> entryIter =
Iterator<Map.Entry<byte[], ArrayList<HStoreFile>>> entryIter =
candidateStripes.entrySet().iterator();
while (entryIter.hasNext()) {
Map.Entry<byte[], ArrayList<StoreFile>> entry = entryIter.next();
ArrayList<StoreFile> files = entry.getValue();
Map.Entry<byte[], ArrayList<HStoreFile>> entry = entryIter.next();
ArrayList<HStoreFile> files = entry.getValue();
// Validate the file start rows, and remove the bad ones to level 0.
for (int i = 0; i < files.size(); ++i) {
StoreFile sf = files.get(i);
HStoreFile sf = files.get(i);
byte[] startRow = startOf(sf);
if (expectedStartRow == null) {
expectedStartRow = startRow; // ensure that first stripe is still consistent
@ -439,7 +440,7 @@ public class StripeStoreFileManager
LOG.warn("Store file doesn't fit into the tentative stripes - expected to start at ["
+ Bytes.toString(expectedStartRow) + "], but starts at [" + Bytes.toString(startRow)
+ "], to L0 it goes");
StoreFile badSf = files.remove(i);
HStoreFile badSf = files.remove(i);
insertFileIntoStripe(level0Files, badSf);
ensureLevel0Metadata(badSf);
--i;
@ -459,7 +460,7 @@ public class StripeStoreFileManager
// as open keys anyway, and log the message.
// If there were errors, we'll play it safe and dump everything into L0.
if (!candidateStripes.isEmpty()) {
StoreFile firstFile = candidateStripes.firstEntry().getValue().get(0);
HStoreFile firstFile = candidateStripes.firstEntry().getValue().get(0);
boolean isOpen = isOpen(startOf(firstFile)) && isOpen(candidateStripes.lastKey());
if (!isOpen) {
LOG.warn("The range of the loaded files does not cover full key space: from ["
@ -470,8 +471,8 @@ public class StripeStoreFileManager
ensureEdgeStripeMetadata(candidateStripes.lastEntry().getValue(), false);
} else {
LOG.warn("Inconsistent files, everything goes to L0.");
for (ArrayList<StoreFile> files : candidateStripes.values()) {
for (StoreFile sf : files) {
for (ArrayList<HStoreFile> files : candidateStripes.values()) {
for (HStoreFile sf : files) {
insertFileIntoStripe(level0Files, sf);
ensureLevel0Metadata(sf);
}
@ -486,9 +487,9 @@ public class StripeStoreFileManager
state.level0Files = ImmutableList.copyOf(level0Files);
state.stripeFiles = new ArrayList<>(candidateStripes.size());
state.stripeEndRows = new byte[Math.max(0, candidateStripes.size() - 1)][];
ArrayList<StoreFile> newAllFiles = new ArrayList<>(level0Files);
ArrayList<HStoreFile> newAllFiles = new ArrayList<>(level0Files);
int i = candidateStripes.size() - 1;
for (Map.Entry<byte[], ArrayList<StoreFile>> entry : candidateStripes.entrySet()) {
for (Map.Entry<byte[], ArrayList<HStoreFile>> entry : candidateStripes.entrySet()) {
state.stripeFiles.add(ImmutableList.copyOf(entry.getValue()));
newAllFiles.addAll(entry.getValue());
if (i > 0) {
@ -501,14 +502,14 @@ public class StripeStoreFileManager
debugDumpState("Files loaded");
}
private void ensureEdgeStripeMetadata(ArrayList<StoreFile> stripe, boolean isFirst) {
HashMap<StoreFile, byte[]> targetMap = isFirst ? fileStarts : fileEnds;
for (StoreFile sf : stripe) {
private void ensureEdgeStripeMetadata(ArrayList<HStoreFile> stripe, boolean isFirst) {
HashMap<HStoreFile, byte[]> targetMap = isFirst ? fileStarts : fileEnds;
for (HStoreFile sf : stripe) {
targetMap.put(sf, OPEN_KEY);
}
}
private void ensureLevel0Metadata(StoreFile sf) {
private void ensureLevel0Metadata(HStoreFile sf) {
if (!isInvalid(startOf(sf))) this.fileStarts.put(sf, INVALID_KEY_IN_MAP);
if (!isInvalid(endOf(sf))) this.fileEnds.put(sf, INVALID_KEY_IN_MAP);
}
@ -616,7 +617,7 @@ public class StripeStoreFileManager
}
private byte[] startOf(StoreFile sf) {
private byte[] startOf(HStoreFile sf) {
byte[] result = fileStarts.get(sf);
// result and INVALID_KEY_IN_MAP are compared _only_ by reference on purpose here as the latter
@ -627,7 +628,7 @@ public class StripeStoreFileManager
: result == INVALID_KEY_IN_MAP ? INVALID_KEY : result;
}
private byte[] endOf(StoreFile sf) {
private byte[] endOf(HStoreFile sf) {
byte[] result = fileEnds.get(sf);
// result and INVALID_KEY_IN_MAP are compared _only_ by reference on purpose here as the latter
@ -643,7 +644,7 @@ public class StripeStoreFileManager
* @param stripe Stripe copy to insert into.
* @param sf File to insert.
*/
private static void insertFileIntoStripe(ArrayList<StoreFile> stripe, StoreFile sf) {
private static void insertFileIntoStripe(ArrayList<HStoreFile> stripe, HStoreFile sf) {
// The only operation for which sorting of the files matters is KeyBefore. Therefore,
// we will store the file in reverse order by seqNum from the outset.
for (int insertBefore = 0; ; ++insertBefore) {
@ -664,20 +665,20 @@ public class StripeStoreFileManager
* but will quickly cut down on them as it finds something in the more likely ones; thus,
* the above allow us to avoid unnecessary copying of a bunch of lists.
*/
private static class KeyBeforeConcatenatedLists extends ConcatenatedLists<StoreFile> {
private static class KeyBeforeConcatenatedLists extends ConcatenatedLists<HStoreFile> {
@Override
public java.util.Iterator<StoreFile> iterator() {
public java.util.Iterator<HStoreFile> iterator() {
return new Iterator();
}
public class Iterator extends ConcatenatedLists<StoreFile>.Iterator {
public ArrayList<List<StoreFile>> getComponents() {
public class Iterator extends ConcatenatedLists<HStoreFile>.Iterator {
public ArrayList<List<HStoreFile>> getComponents() {
return components;
}
public void removeComponents(int startIndex) {
List<List<StoreFile>> subList = components.subList(startIndex, components.size());
for (List<StoreFile> entry : subList) {
List<List<HStoreFile>> subList = components.subList(startIndex, components.size());
for (List<HStoreFile> entry : subList) {
size -= entry.size();
}
assert size >= 0;
@ -690,7 +691,7 @@ public class StripeStoreFileManager
throw new IllegalStateException("No element to remove");
}
this.nextWasCalled = false;
List<StoreFile> src = components.get(currentComponent);
List<HStoreFile> src = components.get(currentComponent);
if (src instanceof ImmutableList<?>) {
src = new ArrayList<>(src);
components.set(currentComponent, src);
@ -711,14 +712,14 @@ public class StripeStoreFileManager
* then creates a new state object and puts it in place.
*/
private class CompactionOrFlushMergeCopy {
private ArrayList<List<StoreFile>> stripeFiles = null;
private ArrayList<StoreFile> level0Files = null;
private ArrayList<List<HStoreFile>> stripeFiles = null;
private ArrayList<HStoreFile> level0Files = null;
private ArrayList<byte[]> stripeEndRows = null;
private Collection<StoreFile> compactedFiles = null;
private Collection<StoreFile> results = null;
private Collection<HStoreFile> compactedFiles = null;
private Collection<HStoreFile> results = null;
private List<StoreFile> l0Results = new ArrayList<>();
private List<HStoreFile> l0Results = new ArrayList<>();
private final boolean isFlush;
public CompactionOrFlushMergeCopy(boolean isFlush) {
@ -727,14 +728,14 @@ public class StripeStoreFileManager
this.isFlush = isFlush;
}
private void mergeResults(Collection<StoreFile> compactedFiles, Collection<StoreFile> results)
private void mergeResults(Collection<HStoreFile> compactedFiles, Collection<HStoreFile> results)
throws IOException {
assert this.compactedFiles == null && this.results == null;
this.compactedFiles = compactedFiles;
this.results = results;
// Do logical processing.
if (!isFlush) removeCompactedFiles();
TreeMap<byte[], StoreFile> newStripes = processResults();
TreeMap<byte[], HStoreFile> newStripes = processResults();
if (newStripes != null) {
processNewCandidateStripes(newStripes);
}
@ -744,7 +745,7 @@ public class StripeStoreFileManager
updateMetadataMaps();
}
private void deleteResults(Collection<StoreFile> compactedFiles) throws IOException {
private void deleteResults(Collection<HStoreFile> compactedFiles) throws IOException {
this.compactedFiles = compactedFiles;
// Create new state and update parent.
State state = createNewState(true);
@ -762,13 +763,13 @@ public class StripeStoreFileManager
newState.stripeEndRows = (this.stripeEndRows == null) ? oldState.stripeEndRows
: this.stripeEndRows.toArray(new byte[this.stripeEndRows.size()][]);
newState.stripeFiles = new ArrayList<>(this.stripeFiles.size());
for (List<StoreFile> newStripe : this.stripeFiles) {
for (List<HStoreFile> newStripe : this.stripeFiles) {
newState.stripeFiles.add(newStripe instanceof ImmutableList<?>
? (ImmutableList<StoreFile>)newStripe : ImmutableList.copyOf(newStripe));
? (ImmutableList<HStoreFile>)newStripe : ImmutableList.copyOf(newStripe));
}
List<StoreFile> newAllFiles = new ArrayList<>(oldState.allFilesCached);
List<StoreFile> newAllCompactedFiles = new ArrayList<>(oldState.allCompactedFilesCached);
List<HStoreFile> newAllFiles = new ArrayList<>(oldState.allFilesCached);
List<HStoreFile> newAllCompactedFiles = new ArrayList<>(oldState.allCompactedFilesCached);
if (!isFlush) {
newAllFiles.removeAll(compactedFiles);
if (delCompactedFiles) {
@ -788,13 +789,13 @@ public class StripeStoreFileManager
private void updateMetadataMaps() {
StripeStoreFileManager parent = StripeStoreFileManager.this;
if (!isFlush) {
for (StoreFile sf : this.compactedFiles) {
for (HStoreFile sf : this.compactedFiles) {
parent.fileStarts.remove(sf);
parent.fileEnds.remove(sf);
}
}
if (this.l0Results != null) {
for (StoreFile sf : this.l0Results) {
for (HStoreFile sf : this.l0Results) {
parent.ensureLevel0Metadata(sf);
}
}
@ -804,14 +805,14 @@ public class StripeStoreFileManager
* @param index Index of the stripe we need.
* @return A lazy stripe copy from current stripes.
*/
private final ArrayList<StoreFile> getStripeCopy(int index) {
List<StoreFile> stripeCopy = this.stripeFiles.get(index);
ArrayList<StoreFile> result = null;
private final ArrayList<HStoreFile> getStripeCopy(int index) {
List<HStoreFile> stripeCopy = this.stripeFiles.get(index);
ArrayList<HStoreFile> result = null;
if (stripeCopy instanceof ImmutableList<?>) {
result = new ArrayList<>(stripeCopy);
this.stripeFiles.set(index, result);
} else {
result = (ArrayList<StoreFile>)stripeCopy;
result = (ArrayList<HStoreFile>)stripeCopy;
}
return result;
}
@ -819,7 +820,7 @@ public class StripeStoreFileManager
/**
* @return A lazy L0 copy from current state.
*/
private final ArrayList<StoreFile> getLevel0Copy() {
private final ArrayList<HStoreFile> getLevel0Copy() {
if (this.level0Files == null) {
this.level0Files = new ArrayList<>(StripeStoreFileManager.this.state.level0Files);
}
@ -831,9 +832,9 @@ public class StripeStoreFileManager
* or to the list of new candidate stripes.
* @return New candidate stripes.
*/
private TreeMap<byte[], StoreFile> processResults() throws IOException {
TreeMap<byte[], StoreFile> newStripes = null;
for (StoreFile sf : this.results) {
private TreeMap<byte[], HStoreFile> processResults() throws IOException {
TreeMap<byte[], HStoreFile> newStripes = null;
for (HStoreFile sf : this.results) {
byte[] startRow = startOf(sf), endRow = endOf(sf);
if (isInvalid(endRow) || isInvalid(startRow)) {
if (!isFlush) {
@ -856,7 +857,7 @@ public class StripeStoreFileManager
if (newStripes == null) {
newStripes = new TreeMap<>(MAP_COMPARATOR);
}
StoreFile oldSf = newStripes.put(endRow, sf);
HStoreFile oldSf = newStripes.put(endRow, sf);
if (oldSf != null) {
throw new IOException("Compactor has produced multiple files for the stripe ending in ["
+ Bytes.toString(endRow) + "], found " + sf.getPath() + " and " + oldSf.getPath());
@ -870,9 +871,9 @@ public class StripeStoreFileManager
* @param compactedFiles Compacted files.
*/
private void removeCompactedFiles() throws IOException {
for (StoreFile oldFile : this.compactedFiles) {
for (HStoreFile oldFile : this.compactedFiles) {
byte[] oldEndRow = endOf(oldFile);
List<StoreFile> source = null;
List<HStoreFile> source = null;
if (isInvalid(oldEndRow)) {
source = getLevel0Copy();
} else {
@ -895,7 +896,7 @@ public class StripeStoreFileManager
* @param newStripes New stripes - files by end row.
*/
private void processNewCandidateStripes(
TreeMap<byte[], StoreFile> newStripes) throws IOException {
TreeMap<byte[], HStoreFile> newStripes) throws IOException {
// Validate that the removed and added aggregate ranges still make for a full key space.
boolean hasStripes = !this.stripeFiles.isEmpty();
this.stripeEndRows = new ArrayList<>(Arrays.asList(StripeStoreFileManager.this.state.stripeEndRows));
@ -907,7 +908,7 @@ public class StripeStoreFileManager
}
boolean canAddNewStripes = true;
Collection<StoreFile> filesForL0 = null;
Collection<HStoreFile> filesForL0 = null;
if (hasStripes) {
// Determine which stripes will need to be removed because they conflict with new stripes.
// The new boundaries should match old stripe boundaries, so we should get exact matches.
@ -921,7 +922,7 @@ public class StripeStoreFileManager
int removeTo = findStripeIndexByEndRow(lastEndRow);
if (removeTo < 0) throw new IOException("Compaction is trying to add a bad range.");
// See if there are files in the stripes we are trying to replace.
ArrayList<StoreFile> conflictingFiles = new ArrayList<>();
ArrayList<HStoreFile> conflictingFiles = new ArrayList<>();
for (int removeIndex = removeTo; removeIndex >= removeFrom; --removeIndex) {
conflictingFiles.addAll(this.stripeFiles.get(removeIndex));
}
@ -942,7 +943,7 @@ public class StripeStoreFileManager
filesForL0 = conflictingFiles;
}
if (filesForL0 != null) {
for (StoreFile sf : filesForL0) {
for (HStoreFile sf : filesForL0) {
insertFileIntoStripe(getLevel0Copy(), sf);
}
l0Results.addAll(filesForL0);
@ -966,7 +967,7 @@ public class StripeStoreFileManager
// Now, insert new stripes. The total ranges match, so we can insert where we removed.
byte[] previousEndRow = null;
int insertAt = removeFrom;
for (Map.Entry<byte[], StoreFile> newStripe : newStripes.entrySet()) {
for (Map.Entry<byte[], HStoreFile> newStripe : newStripes.entrySet()) {
if (previousEndRow != null) {
// Validate that the ranges are contiguous.
assert !isOpen(previousEndRow);
@ -977,7 +978,7 @@ public class StripeStoreFileManager
}
}
// Add the new stripe.
ArrayList<StoreFile> tmp = new ArrayList<>();
ArrayList<HStoreFile> tmp = new ArrayList<>();
tmp.add(newStripe.getValue());
stripeFiles.add(insertAt, tmp);
previousEndRow = newStripe.getKey();
@ -990,7 +991,7 @@ public class StripeStoreFileManager
}
@Override
public List<StoreFile> getLevel0Files() {
public List<HStoreFile> getLevel0Files() {
return this.state.level0Files;
}
@ -1005,7 +1006,7 @@ public class StripeStoreFileManager
}
@Override
public ArrayList<ImmutableList<StoreFile>> getStripes() {
public ArrayList<ImmutableList<HStoreFile>> getStripes() {
return this.state.stripeFiles;
}
@ -1015,22 +1016,22 @@ public class StripeStoreFileManager
}
@Override
public Collection<StoreFile> getUnneededFiles(long maxTs, List<StoreFile> filesCompacting) {
public Collection<HStoreFile> getUnneededFiles(long maxTs, List<HStoreFile> filesCompacting) {
// 1) We can never get rid of the last file which has the maximum seqid in a stripe.
// 2) Files that are not the latest can't become one due to (1), so the rest are fair game.
State state = this.state;
Collection<StoreFile> expiredStoreFiles = null;
for (ImmutableList<StoreFile> stripe : state.stripeFiles) {
Collection<HStoreFile> expiredStoreFiles = null;
for (ImmutableList<HStoreFile> stripe : state.stripeFiles) {
expiredStoreFiles = findExpiredFiles(stripe, maxTs, filesCompacting, expiredStoreFiles);
}
return findExpiredFiles(state.level0Files, maxTs, filesCompacting, expiredStoreFiles);
}
private Collection<StoreFile> findExpiredFiles(ImmutableList<StoreFile> stripe, long maxTs,
List<StoreFile> filesCompacting, Collection<StoreFile> expiredStoreFiles) {
private Collection<HStoreFile> findExpiredFiles(ImmutableList<HStoreFile> stripe, long maxTs,
List<HStoreFile> filesCompacting, Collection<HStoreFile> expiredStoreFiles) {
// Order by seqnum is reversed.
for (int i = 1; i < stripe.size(); ++i) {
StoreFile sf = stripe.get(i);
HStoreFile sf = stripe.get(i);
synchronized (sf) {
long fileTs = sf.getReader().getMaxTimestamp();
if (fileTs < maxTs && !filesCompacting.contains(sf)) {
@ -1061,7 +1062,7 @@ public class StripeStoreFileManager
// we flush data to stripe directly.
int delta = stateLocal.level0Files.isEmpty() ? 0 : 1;
double max = 0.0;
for (ImmutableList<StoreFile> stripeFile : stateLocal.stripeFiles) {
for (ImmutableList<HStoreFile> stripeFile : stateLocal.stripeFiles) {
int stripeFileCount = stripeFile.size();
double normCount =
(double) (stripeFileCount + delta - config.getStripeCompactMinFiles())
@ -1079,7 +1080,7 @@ public class StripeStoreFileManager
}
@Override
public Comparator<StoreFile> getStoreFileComparator() {
public Comparator<HStoreFile> getStoreFileComparator() {
return StoreFileComparators.SEQ_ID;
}
}

View File

@ -26,13 +26,13 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
@ -47,7 +47,7 @@ public class StripeStoreFlusher extends StoreFlusher {
private final StripeCompactionPolicy policy;
private final StripeCompactionPolicy.StripeInformationProvider stripes;
public StripeStoreFlusher(Configuration conf, Store store,
public StripeStoreFlusher(Configuration conf, HStore store,
StripeCompactionPolicy policy, StripeStoreFileManager stripes) {
super(conf, store);
this.policy = policy;

View File

@ -24,13 +24,13 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter;
import org.apache.hadoop.hbase.regionserver.AbstractMultiFileWriter.WriterFactory;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.regionserver.StoreScanner;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Base class for implementing a Compactor which will generate multiple output files after
@ -42,7 +42,7 @@ public abstract class AbstractMultiOutputCompactor<T extends AbstractMultiFileWr
private static final Log LOG = LogFactory.getLog(AbstractMultiOutputCompactor.class);
public AbstractMultiOutputCompactor(Configuration conf, Store store) {
public AbstractMultiOutputCompactor(Configuration conf, HStore store) {
super(conf, store);
}

View File

@ -21,11 +21,11 @@ package org.apache.hadoop.hbase.regionserver.compactions;
import java.io.IOException;
import java.util.List;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.security.User;
import org.apache.yetus.audience.InterfaceAudience;
/**
@ -44,7 +44,7 @@ public abstract class CompactionContext {
* @param filesCompacting files currently compacting
* @return the list of files that can theoretically be compacted.
*/
public abstract List<StoreFile> preSelect(final List<StoreFile> filesCompacting);
public abstract List<HStoreFile> preSelect(List<HStoreFile> filesCompacting);
/**
* Called to select files for compaction. Must fill in the request field if successful.
@ -54,9 +54,8 @@ public abstract class CompactionContext {
* @param forceMajor Whether to force major compaction.
* @return Whether the selection succeeded. Selection may be empty and lead to no compaction.
*/
public abstract boolean select(
final List<StoreFile> filesCompacting, final boolean isUserCompaction,
final boolean mayUseOffPeak, final boolean forceMajor) throws IOException;
public abstract boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction,
boolean mayUseOffPeak, boolean forceMajor) throws IOException;
/**
* Forces external selection to be applied for this compaction.

View File

@ -22,10 +22,10 @@ package org.apache.hadoop.hbase.regionserver.compactions;
import java.io.IOException;
import java.util.Collection;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A compaction policy determines how to select files for compaction,
@ -45,8 +45,8 @@ public abstract class CompactionPolicy {
* @param filesToCompact Files to compact. Can be null.
* @return True if we should run a major compaction.
*/
public abstract boolean shouldPerformMajorCompaction(
final Collection<StoreFile> filesToCompact) throws IOException;
public abstract boolean shouldPerformMajorCompaction(Collection<HStoreFile> filesToCompact)
throws IOException;
/**
* @param compactionSize Total size of some compaction

View File

@ -21,8 +21,8 @@ package org.apache.hadoop.hbase.regionserver.compactions;
import java.util.Collection;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
import org.apache.yetus.audience.InterfaceAudience;
@ -40,7 +40,7 @@ public class CompactionRequest {
private enum DisplayCompactionType { MINOR, ALL_FILES, MAJOR }
private DisplayCompactionType isMajor = DisplayCompactionType.MINOR;
private int priority = Store.NO_PRIORITY;
private Collection<StoreFile> filesToCompact;
private Collection<HStoreFile> filesToCompact;
// CompactRequest object creation time.
private long selectionTime;
@ -51,19 +51,19 @@ public class CompactionRequest {
private long totalSize = -1L;
private CompactionLifeCycleTracker tracker = CompactionLifeCycleTracker.DUMMY;
public CompactionRequest(Collection<StoreFile> files) {
public CompactionRequest(Collection<HStoreFile> files) {
this.selectionTime = EnvironmentEdgeManager.currentTime();
this.timeInNanos = System.nanoTime();
this.filesToCompact = Preconditions.checkNotNull(files, "files for compaction can not null");
recalculateSize();
}
public void updateFiles(Collection<StoreFile> files) {
public void updateFiles(Collection<HStoreFile> files) {
this.filesToCompact = Preconditions.checkNotNull(files, "files for compaction can not null");
recalculateSize();
}
public Collection<StoreFile> getFiles() {
public Collection<HStoreFile> getFiles() {
return this.filesToCompact;
}
@ -152,7 +152,7 @@ public class CompactionRequest {
* @param files files that should be included in the compaction
*/
private void recalculateSize() {
this.totalSize = filesToCompact.stream().map(StoreFile::getReader)
this.totalSize = filesToCompact.stream().map(HStoreFile::getReader)
.mapToLong(r -> r != null ? r.length() : 0L).sum();
}
}

View File

@ -17,6 +17,9 @@
*/
package org.apache.hadoop.hbase.regionserver.compactions;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.EARLIEST_PUT_TS;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.TIMERANGE_KEY;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
@ -33,19 +36,17 @@ import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFile.FileInfo;
import org.apache.hadoop.hbase.regionserver.CellSink;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.ScannerContext;
import org.apache.hadoop.hbase.regionserver.ShipperListener;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileReader;
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
@ -57,6 +58,7 @@ import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.io.Closeables;
@ -70,7 +72,7 @@ public abstract class Compactor<T extends CellSink> {
protected static final long COMPACTION_PROGRESS_LOG_INTERVAL = 60 * 1000;
protected volatile CompactionProgress progress;
protected final Configuration conf;
protected final Store store;
protected final HStore store;
protected final int compactionKVMax;
protected final Compression.Algorithm compactionCompression;
@ -88,7 +90,7 @@ public abstract class Compactor<T extends CellSink> {
private boolean dropCacheMinor;
//TODO: depending on Store is not good but, realistically, all compactors currently do.
Compactor(final Configuration conf, final Store store) {
Compactor(Configuration conf, HStore store) {
this.conf = conf;
this.store = store;
this.compactionKVMax =
@ -137,12 +139,12 @@ public abstract class Compactor<T extends CellSink> {
* @return The result.
*/
protected FileDetails getFileDetails(
Collection<StoreFile> filesToCompact, boolean allFiles) throws IOException {
Collection<HStoreFile> filesToCompact, boolean allFiles) throws IOException {
FileDetails fd = new FileDetails();
long oldestHFileTimeStampToKeepMVCC = System.currentTimeMillis() -
(1000L * 60 * 60 * 24 * this.keepSeqIdPeriod);
for (StoreFile file : filesToCompact) {
for (HStoreFile file : filesToCompact) {
if(allFiles && (file.getModificationTimeStamp() < oldestHFileTimeStampToKeepMVCC)) {
// when isAllFiles is true, all files are compacted so we can calculate the smallest
// MVCC value to keep
@ -184,7 +186,7 @@ public abstract class Compactor<T extends CellSink> {
// This is used to remove family delete marker during compaction.
long earliestPutTs = 0;
if (allFiles) {
tmp = fileInfo.get(StoreFile.EARLIEST_PUT_TS);
tmp = fileInfo.get(EARLIEST_PUT_TS);
if (tmp == null) {
// There's a file with no information, must be an old one
// assume we have very old puts
@ -194,7 +196,7 @@ public abstract class Compactor<T extends CellSink> {
fd.earliestPutTs = Math.min(fd.earliestPutTs, earliestPutTs);
}
}
tmp = fileInfo.get(StoreFile.TIMERANGE_KEY);
tmp = fileInfo.get(TIMERANGE_KEY);
TimeRangeTracker trt = TimeRangeTracker.getTimeRangeTracker(tmp);
fd.latestPutTs = trt == null? HConstants.LATEST_TIMESTAMP: trt.getMax();
if (LOG.isDebugEnabled()) {
@ -215,7 +217,7 @@ public abstract class Compactor<T extends CellSink> {
* @param filesToCompact Files.
* @return Scanners.
*/
protected List<StoreFileScanner> createFileScanners(Collection<StoreFile> filesToCompact,
protected List<StoreFileScanner> createFileScanners(Collection<HStoreFile> filesToCompact,
long smallestReadPoint, boolean useDropBehind) throws IOException {
return StoreFileScanner.getScannersForCompaction(filesToCompact, useDropBehind,
smallestReadPoint);
@ -498,7 +500,7 @@ public abstract class Compactor<T extends CellSink> {
* @param earliestPutTs Earliest put across all files.
* @return A compaction scanner.
*/
protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
protected InternalScanner createScanner(HStore store, List<StoreFileScanner> scanners,
ScanType scanType, long smallestReadPoint, long earliestPutTs) throws IOException {
return new StoreScanner(store, store.getScanInfo(), OptionalInt.empty(), scanners, scanType,
smallestReadPoint, earliestPutTs);
@ -513,7 +515,7 @@ public abstract class Compactor<T extends CellSink> {
* @param dropDeletesToRow Drop deletes ending with this row, exclusive. Can be null.
* @return A compaction scanner.
*/
protected InternalScanner createScanner(Store store, List<StoreFileScanner> scanners,
protected InternalScanner createScanner(HStore store, List<StoreFileScanner> scanners,
long smallestReadPoint, long earliestPutTs, byte[] dropDeletesFromRow,
byte[] dropDeletesToRow) throws IOException {
return new StoreScanner(store, store.getScanInfo(), OptionalInt.empty(), scanners,

View File

@ -30,14 +30,14 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.ReflectionUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterators;
@ -98,9 +98,9 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
*/
@Override
@VisibleForTesting
public boolean needsCompaction(final Collection<StoreFile> storeFiles,
final List<StoreFile> filesCompacting) {
ArrayList<StoreFile> candidates = new ArrayList<>(storeFiles);
public boolean needsCompaction(Collection<HStoreFile> storeFiles,
List<HStoreFile> filesCompacting) {
ArrayList<HStoreFile> candidates = new ArrayList<>(storeFiles);
try {
return !selectMinorCompaction(candidates, false, true).getFiles().isEmpty();
} catch (Exception e) {
@ -109,8 +109,8 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
}
}
public boolean shouldPerformMajorCompaction(final Collection<StoreFile> filesToCompact)
throws IOException {
public boolean shouldPerformMajorCompaction(Collection<HStoreFile> filesToCompact)
throws IOException {
long mcTime = getNextMajorCompactTime(filesToCompact);
if (filesToCompact == null || mcTime == 0) {
if (LOG.isDebugEnabled()) {
@ -135,7 +135,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
List<Long> boundaries = getCompactBoundariesForMajor(filesToCompact, now);
boolean[] filesInWindow = new boolean[boundaries.size()];
for (StoreFile file: filesToCompact) {
for (HStoreFile file: filesToCompact) {
OptionalLong minTimestamp = file.getMinimumTimestamp();
long oldest = minTimestamp.isPresent() ? now - minTimestamp.getAsLong() : Long.MIN_VALUE;
if (cfTTL != Long.MAX_VALUE && oldest >= cfTTL) {
@ -186,7 +186,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
}
@Override
protected CompactionRequest createCompactionRequest(ArrayList<StoreFile> candidateSelection,
protected CompactionRequest createCompactionRequest(ArrayList<HStoreFile> candidateSelection,
boolean tryingMajor, boolean mayUseOffPeak, boolean mayBeStuck) throws IOException {
CompactionRequest result = tryingMajor ? selectMajorCompaction(candidateSelection)
: selectMinorCompaction(candidateSelection, mayUseOffPeak, mayBeStuck);
@ -196,7 +196,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
return result;
}
public CompactionRequest selectMajorCompaction(ArrayList<StoreFile> candidateSelection) {
public CompactionRequest selectMajorCompaction(ArrayList<HStoreFile> candidateSelection) {
long now = EnvironmentEdgeManager.currentTime();
return new DateTieredCompactionRequest(candidateSelection,
this.getCompactBoundariesForMajor(candidateSelection, now));
@ -210,15 +210,15 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
* by seqId and maxTimestamp in descending order and build the time windows. All the out-of-order
* data into the same compaction windows, guaranteeing contiguous compaction based on sequence id.
*/
public CompactionRequest selectMinorCompaction(ArrayList<StoreFile> candidateSelection,
public CompactionRequest selectMinorCompaction(ArrayList<HStoreFile> candidateSelection,
boolean mayUseOffPeak, boolean mayBeStuck) throws IOException {
long now = EnvironmentEdgeManager.currentTime();
long oldestToCompact = getOldestToCompact(comConf.getDateTieredMaxStoreFileAgeMillis(), now);
List<Pair<StoreFile, Long>> storefileMaxTimestampPairs =
List<Pair<HStoreFile, Long>> storefileMaxTimestampPairs =
Lists.newArrayListWithCapacity(candidateSelection.size());
long maxTimestampSeen = Long.MIN_VALUE;
for (StoreFile storeFile : candidateSelection) {
for (HStoreFile storeFile : candidateSelection) {
// if there is out-of-order data,
// we put them in the same window as the last file in increasing order
maxTimestampSeen =
@ -229,7 +229,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
CompactionWindow window = getIncomingWindow(now);
int minThreshold = comConf.getDateTieredIncomingWindowMin();
PeekingIterator<Pair<StoreFile, Long>> it =
PeekingIterator<Pair<HStoreFile, Long>> it =
Iterators.peekingIterator(storefileMaxTimestampPairs.iterator());
while (it.hasNext()) {
if (window.compareToTimestamp(oldestToCompact) < 0) {
@ -242,7 +242,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
minThreshold = comConf.getMinFilesToCompact();
} else {
// The file is within the target window
ArrayList<StoreFile> fileList = Lists.newArrayList();
ArrayList<HStoreFile> fileList = Lists.newArrayList();
// Add all files in the same window. For incoming window
// we tolerate files with future data although it is sub-optimal
while (it.hasNext() && window.compareToTimestamp(it.peek().getSecond()) <= 0) {
@ -261,10 +261,10 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
}
}
// A non-null file list is expected by HStore
return new CompactionRequest(Collections.<StoreFile> emptyList());
return new CompactionRequest(Collections.emptyList());
}
private DateTieredCompactionRequest generateCompactionRequest(ArrayList<StoreFile> storeFiles,
private DateTieredCompactionRequest generateCompactionRequest(ArrayList<HStoreFile> storeFiles,
CompactionWindow window, boolean mayUseOffPeak, boolean mayBeStuck, int minThreshold)
throws IOException {
// The files has to be in ascending order for ratio-based compaction to work right
@ -273,7 +273,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
// Compact everything in the window if have more files than comConf.maxBlockingFiles
compactionPolicyPerWindow.setMinThreshold(minThreshold);
ArrayList<StoreFile> storeFileSelection = mayBeStuck ? storeFiles
ArrayList<HStoreFile> storeFileSelection = mayBeStuck ? storeFiles
: compactionPolicyPerWindow.applyCompactionPolicy(storeFiles, mayUseOffPeak, false);
if (storeFileSelection != null && !storeFileSelection.isEmpty()) {
// If there is any file in the window excluded from compaction,
@ -291,7 +291,7 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
/**
* Return a list of boundaries for multiple compaction output in ascending order.
*/
private List<Long> getCompactBoundariesForMajor(Collection<StoreFile> filesToCompact, long now) {
private List<Long> getCompactBoundariesForMajor(Collection<HStoreFile> filesToCompact, long now) {
long minTimestamp =
filesToCompact.stream().mapToLong(f -> f.getMinimumTimestamp().orElse(Long.MAX_VALUE)).min()
.orElse(Long.MAX_VALUE);

View File

@ -21,14 +21,14 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="EQ_DOESNT_OVERRIDE_EQUALS",
justification="It is intended to use the same equal method as superclass")
public class DateTieredCompactionRequest extends CompactionRequest {
private List<Long> boundaries;
public DateTieredCompactionRequest(Collection<StoreFile> files, List<Long> boundaryList) {
public DateTieredCompactionRequest(Collection<HStoreFile> files, List<Long> boundaryList) {
super(files);
boundaries = boundaryList;
}

View File

@ -24,13 +24,13 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreUtils;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.security.User;
import org.apache.yetus.audience.InterfaceAudience;
/**
* This compactor will generate StoreFile for different time ranges.
@ -40,7 +40,7 @@ public class DateTieredCompactor extends AbstractMultiOutputCompactor<DateTiered
private static final Log LOG = LogFactory.getLog(DateTieredCompactor.class);
public DateTieredCompactor(Configuration conf, Store store) {
public DateTieredCompactor(Configuration conf, HStore store) {
super(conf, store);
}

View File

@ -25,14 +25,14 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.security.User;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
@ -44,7 +44,7 @@ import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
public class DefaultCompactor extends Compactor<StoreFileWriter> {
private static final Log LOG = LogFactory.getLog(DefaultCompactor.class);
public DefaultCompactor(final Configuration conf, final Store store) {
public DefaultCompactor(Configuration conf, HStore store) {
super(conf, store);
}
@ -76,7 +76,7 @@ public class DefaultCompactor extends Compactor<StoreFileWriter> {
* made it through the compaction.
* @throws IOException
*/
public List<Path> compactForTesting(final Collection<StoreFile> filesToCompact, boolean isMajor)
public List<Path> compactForTesting(Collection<HStoreFile> filesToCompact, boolean isMajor)
throws IOException {
CompactionRequest cr = new CompactionRequest(filesToCompact);
cr.setIsMajor(isMajor, isMajor);

View File

@ -26,9 +26,9 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Class to pick which files if any to compact together.
@ -51,21 +51,20 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
}
@Override
protected final ArrayList<StoreFile> applyCompactionPolicy(final ArrayList<StoreFile> candidates,
final boolean mayUseOffPeak, final boolean mightBeStuck) throws IOException {
return new ArrayList<>(applyCompactionPolicy(candidates, mightBeStuck,
mayUseOffPeak, comConf.getMinFilesToCompact(), comConf.getMaxFilesToCompact()));
protected final ArrayList<HStoreFile> applyCompactionPolicy(ArrayList<HStoreFile> candidates,
boolean mayUseOffPeak, boolean mightBeStuck) throws IOException {
return new ArrayList<>(applyCompactionPolicy(candidates, mightBeStuck, mayUseOffPeak,
comConf.getMinFilesToCompact(), comConf.getMaxFilesToCompact()));
}
public List<StoreFile> applyCompactionPolicy(final List<StoreFile> candidates,
boolean mightBeStuck, boolean mayUseOffPeak, int minFiles, int maxFiles) {
public List<HStoreFile> applyCompactionPolicy(List<HStoreFile> candidates, boolean mightBeStuck,
boolean mayUseOffPeak, int minFiles, int maxFiles) {
final double currentRatio = mayUseOffPeak
? comConf.getCompactionRatioOffPeak() : comConf.getCompactionRatio();
// Start off choosing nothing.
List<StoreFile> bestSelection = new ArrayList<>(0);
List<StoreFile> smallest = mightBeStuck ? new ArrayList<>(0) : null;
List<HStoreFile> bestSelection = new ArrayList<>(0);
List<HStoreFile> smallest = mightBeStuck ? new ArrayList<>(0) : null;
long bestSize = 0;
long smallestSize = Long.MAX_VALUE;
@ -75,7 +74,7 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
// Consider every different sub list permutation in between start and end with min files.
for (int currentEnd = start + minFiles - 1;
currentEnd < candidates.size(); currentEnd++) {
List<StoreFile> potentialMatchFiles = candidates.subList(start, currentEnd + 1);
List<HStoreFile> potentialMatchFiles = candidates.subList(start, currentEnd + 1);
// Sanity checks
if (potentialMatchFiles.size() < minFiles) {
@ -125,8 +124,8 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
return new ArrayList<>(bestSelection);
}
private boolean isBetterSelection(List<StoreFile> bestSelection,
long bestSize, List<StoreFile> selection, long size, boolean mightBeStuck) {
private boolean isBetterSelection(List<HStoreFile> bestSelection, long bestSize,
List<HStoreFile> selection, long size, boolean mightBeStuck) {
if (mightBeStuck && bestSize > 0 && size > 0) {
// Keep the selection that removes most files for least size. That penaltizes adding
// large files to compaction, but not small files, so we don't become totally inefficient
@ -146,13 +145,8 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
* @param potentialMatchFiles StoreFile list.
* @return Sum of StoreFile.getReader().length();
*/
private long getTotalStoreSize(final List<StoreFile> potentialMatchFiles) {
long size = 0;
for (StoreFile s:potentialMatchFiles) {
size += s.getReader().length();
}
return size;
private long getTotalStoreSize(List<HStoreFile> potentialMatchFiles) {
return potentialMatchFiles.stream().mapToLong(sf -> sf.getReader().length()).sum();
}
/**
@ -163,14 +157,14 @@ public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
* @param currentRatio The ratio to use.
* @return a boolean if these files satisfy the ratio constraints.
*/
private boolean filesInRatio(final List<StoreFile> files, final double currentRatio) {
private boolean filesInRatio(List<HStoreFile> files, double currentRatio) {
if (files.size() < 2) {
return true;
}
long totalFileSize = getTotalStoreSize(files);
for (StoreFile file : files) {
for (HStoreFile file : files) {
long singleFileSize = file.getReader().length();
long sumAllOtherFileSizes = totalFileSize - singleFileSize;

View File

@ -26,11 +26,11 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
/**
*
@ -55,10 +55,9 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
}
@Override
public CompactionRequest selectCompaction(Collection<StoreFile> candidateFiles,
List<StoreFile> filesCompacting, boolean isUserCompaction, boolean mayUseOffPeak,
public CompactionRequest selectCompaction(Collection<HStoreFile> candidateFiles,
List<HStoreFile> filesCompacting, boolean isUserCompaction, boolean mayUseOffPeak,
boolean forceMajor) throws IOException {
if(forceMajor){
LOG.warn("Major compaction is not supported for FIFO compaction policy. Ignore the flag.");
}
@ -70,13 +69,13 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
}
// Nothing to compact
Collection<StoreFile> toCompact = getExpiredStores(candidateFiles, filesCompacting);
Collection<HStoreFile> toCompact = getExpiredStores(candidateFiles, filesCompacting);
CompactionRequest result = new CompactionRequest(toCompact);
return result;
}
@Override
public boolean shouldPerformMajorCompaction(Collection<StoreFile> filesToCompact)
public boolean shouldPerformMajorCompaction(Collection<HStoreFile> filesToCompact)
throws IOException {
boolean isAfterSplit = StoreUtils.hasReferences(filesToCompact);
if(isAfterSplit){
@ -87,8 +86,8 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
}
@Override
public boolean needsCompaction(Collection<StoreFile> storeFiles,
List<StoreFile> filesCompacting) {
public boolean needsCompaction(Collection<HStoreFile> storeFiles,
List<HStoreFile> filesCompacting) {
boolean isAfterSplit = StoreUtils.hasReferences(storeFiles);
if(isAfterSplit){
LOG.info("Split detected, delegate to the parent policy.");
@ -97,9 +96,9 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
return hasExpiredStores(storeFiles);
}
private boolean hasExpiredStores(Collection<StoreFile> files) {
private boolean hasExpiredStores(Collection<HStoreFile> files) {
long currentTime = EnvironmentEdgeManager.currentTime();
for(StoreFile sf: files){
for(HStoreFile sf: files){
// Check MIN_VERSIONS is in HStore removeUnneededFiles
long maxTs = sf.getReader().getMaxTimestamp();
long maxTtl = storeConfigInfo.getStoreFileTtl();
@ -113,11 +112,11 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
return false;
}
private Collection<StoreFile> getExpiredStores(Collection<StoreFile> files,
Collection<StoreFile> filesCompacting) {
private Collection<HStoreFile> getExpiredStores(Collection<HStoreFile> files,
Collection<HStoreFile> filesCompacting) {
long currentTime = EnvironmentEdgeManager.currentTime();
Collection<StoreFile> expiredStores = new ArrayList<>();
for(StoreFile sf: files){
Collection<HStoreFile> expiredStores = new ArrayList<>();
for(HStoreFile sf: files){
// Check MIN_VERSIONS is in HStore removeUnneededFiles
long maxTs = sf.getReader().getMaxTimestamp();
long maxTtl = storeConfigInfo.getStoreFileTtl();

View File

@ -28,13 +28,13 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
/**
* The default algorithm for selecting files for compaction.
@ -55,7 +55,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
* @return True if we should run a major compaction.
*/
@Override
public boolean shouldPerformMajorCompaction(final Collection<StoreFile> filesToCompact)
public boolean shouldPerformMajorCompaction(Collection<HStoreFile> filesToCompact)
throws IOException {
boolean result = false;
long mcTime = getNextMajorCompactTime(filesToCompact);
@ -79,7 +79,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
}
if (filesToCompact.size() == 1) {
// Single file
StoreFile sf = filesToCompact.iterator().next();
HStoreFile sf = filesToCompact.iterator().next();
OptionalLong minTimestamp = sf.getMinimumTimestamp();
long oldest = minTimestamp.isPresent() ? now - minTimestamp.getAsLong() : Long.MIN_VALUE;
if (sf.isMajorCompactionResult() && (cfTTL == Long.MAX_VALUE || oldest < cfTTL)) {
@ -113,7 +113,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
}
@Override
protected CompactionRequest createCompactionRequest(ArrayList<StoreFile>
protected CompactionRequest createCompactionRequest(ArrayList<HStoreFile>
candidateSelection, boolean tryingMajor, boolean mayUseOffPeak, boolean mayBeStuck)
throws IOException {
if (!tryingMajor) {
@ -155,7 +155,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
* @param candidates pre-filtrate
* @return filtered subset
*/
protected ArrayList<StoreFile> applyCompactionPolicy(ArrayList<StoreFile> candidates,
protected ArrayList<HStoreFile> applyCompactionPolicy(ArrayList<HStoreFile> candidates,
boolean mayUseOffPeak, boolean mayBeStuck) throws IOException {
if (candidates.isEmpty()) {
return candidates;
@ -174,7 +174,7 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
long[] fileSizes = new long[countOfFiles];
long[] sumSize = new long[countOfFiles];
for (int i = countOfFiles - 1; i >= 0; --i) {
StoreFile file = candidates.get(i);
HStoreFile file = candidates.get(i);
fileSizes[i] = file.getReader().length();
// calculate the sum of fileSizes[i,i+maxFilesToCompact-1) for algo
int tooFar = i + comConf.getMaxFilesToCompact() - 1;
@ -209,8 +209,8 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
* @param filesCompacting files being scheduled to compact.
* @return true to schedule a request.
*/
public boolean needsCompaction(final Collection<StoreFile> storeFiles,
final List<StoreFile> filesCompacting) {
public boolean needsCompaction(Collection<HStoreFile> storeFiles,
List<HStoreFile> filesCompacting) {
int numCandidates = storeFiles.size() - filesCompacting.size();
return numCandidates >= comConf.getMinFilesToCompact();
}

View File

@ -20,10 +20,10 @@ import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
import org.apache.hadoop.hbase.shaded.com.google.common.base.Predicate;
@ -42,8 +42,8 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
super(conf, storeConfigInfo);
}
public List<StoreFile> preSelectCompactionForCoprocessor(final Collection<StoreFile> candidates,
final List<StoreFile> filesCompacting) {
public List<HStoreFile> preSelectCompactionForCoprocessor(Collection<HStoreFile> candidates,
List<HStoreFile> filesCompacting) {
return getCurrentEligibleFiles(new ArrayList<>(candidates), filesCompacting);
}
@ -53,11 +53,11 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
* on seqId for data consistency.
* @return subset copy of candidate list that meets compaction criteria
*/
public CompactionRequest selectCompaction(Collection<StoreFile> candidateFiles,
final List<StoreFile> filesCompacting, final boolean isUserCompaction,
final boolean mayUseOffPeak, final boolean forceMajor) throws IOException {
public CompactionRequest selectCompaction(Collection<HStoreFile> candidateFiles,
List<HStoreFile> filesCompacting, boolean isUserCompaction, boolean mayUseOffPeak,
boolean forceMajor) throws IOException {
// Preliminary compaction subject to filters
ArrayList<StoreFile> candidateSelection = new ArrayList<>(candidateFiles);
ArrayList<HStoreFile> candidateSelection = new ArrayList<>(candidateFiles);
// Stuck and not compacting enough (estimate). It is not guaranteed that we will be
// able to compact more if stuck and compacting, because ratio policy excludes some
// non-compacting files from consideration during compaction (see getCurrentEligibleFiles).
@ -88,7 +88,7 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
CompactionRequest result = createCompactionRequest(candidateSelection,
isTryingMajor || isAfterSplit, mayUseOffPeak, mayBeStuck);
ArrayList<StoreFile> filesToCompact = Lists.newArrayList(result.getFiles());
ArrayList<HStoreFile> filesToCompact = Lists.newArrayList(result.getFiles());
removeExcessFiles(filesToCompact, isUserCompaction, isTryingMajor);
result.updateFiles(filesToCompact);
@ -99,16 +99,16 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
return result;
}
protected abstract CompactionRequest createCompactionRequest(ArrayList<StoreFile>
candidateSelection, boolean tryingMajor, boolean mayUseOffPeak, boolean mayBeStuck)
throws IOException;
protected abstract CompactionRequest createCompactionRequest(
ArrayList<HStoreFile> candidateSelection, boolean tryingMajor, boolean mayUseOffPeak,
boolean mayBeStuck) throws IOException;
/*
/**
* @param filesToCompact Files to compact. Can be null.
* @return True if we should run a major compaction.
*/
public abstract boolean shouldPerformMajorCompaction(final Collection<StoreFile> filesToCompact)
throws IOException;
public abstract boolean shouldPerformMajorCompaction(Collection<HStoreFile> filesToCompact)
throws IOException;
/**
* Used calculation jitter
@ -119,7 +119,7 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
* @param filesToCompact
* @return When to run next major compaction
*/
public long getNextMajorCompactTime(Collection<StoreFile> filesToCompact) {
public long getNextMajorCompactTime(Collection<HStoreFile> filesToCompact) {
// default = 24hrs
long period = comConf.getMajorCompactionPeriod();
if (period <= 0) {
@ -154,16 +154,16 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
return compactionSize > comConf.getThrottlePoint();
}
public abstract boolean needsCompaction(final Collection<StoreFile> storeFiles,
final List<StoreFile> filesCompacting);
public abstract boolean needsCompaction(Collection<HStoreFile> storeFiles,
List<HStoreFile> filesCompacting);
protected ArrayList<StoreFile> getCurrentEligibleFiles(ArrayList<StoreFile> candidateFiles,
final List<StoreFile> filesCompacting) {
protected ArrayList<HStoreFile> getCurrentEligibleFiles(ArrayList<HStoreFile> candidateFiles,
final List<HStoreFile> filesCompacting) {
// candidates = all storefiles not already in compaction queue
if (!filesCompacting.isEmpty()) {
// exclude all files older than the newest file we're currently
// compacting. this allows us to preserve contiguity (HBASE-2856)
StoreFile last = filesCompacting.get(filesCompacting.size() - 1);
HStoreFile last = filesCompacting.get(filesCompacting.size() - 1);
int idx = candidateFiles.indexOf(last);
Preconditions.checkArgument(idx != -1);
candidateFiles.subList(0, idx + 1).clear();
@ -176,7 +176,7 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
* @return filtered subset exclude all files above maxCompactSize
* Also save all references. We MUST compact them
*/
protected ArrayList<StoreFile> skipLargeFiles(ArrayList<StoreFile> candidates,
protected ArrayList<HStoreFile> skipLargeFiles(ArrayList<HStoreFile> candidates,
boolean mayUseOffpeak) {
int pos = 0;
while (pos < candidates.size() && !candidates.get(pos).isReference()
@ -195,10 +195,10 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
* @param candidates pre-filtrate
* @return filtered subset exclude all bulk load files if configured
*/
protected ArrayList<StoreFile> filterBulk(ArrayList<StoreFile> candidates) {
candidates.removeAll(Collections2.filter(candidates, new Predicate<StoreFile>() {
protected ArrayList<HStoreFile> filterBulk(ArrayList<HStoreFile> candidates) {
candidates.removeAll(Collections2.filter(candidates, new Predicate<HStoreFile>() {
@Override
public boolean apply(StoreFile input) {
public boolean apply(HStoreFile input) {
return input.excludeFromMinorCompaction();
}
}));
@ -208,7 +208,7 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
/**
* @param candidates pre-filtrate
*/
protected void removeExcessFiles(ArrayList<StoreFile> candidates,
protected void removeExcessFiles(ArrayList<HStoreFile> candidates,
boolean isUserCompaction, boolean isMajorCompaction) {
int excess = candidates.size() - comConf.getMaxFilesToCompact();
if (excess > 0) {
@ -227,12 +227,12 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
* @param candidates pre-filtrate
* @return filtered subset forget the compactionSelection if we don't have enough files
*/
protected ArrayList<StoreFile> checkMinFilesCriteria(ArrayList<StoreFile> candidates,
int minFiles) {
protected ArrayList<HStoreFile> checkMinFilesCriteria(ArrayList<HStoreFile> candidates,
int minFiles) {
if (candidates.size() < minFiles) {
if (LOG.isDebugEnabled()) {
LOG.debug("Not compacting files because we only have " + candidates.size()
+ " files ready for compaction. Need " + minFiles + " to initiate.");
LOG.debug("Not compacting files because we only have " + candidates.size() +
" files ready for compaction. Need " + minFiles + " to initiate.");
}
candidates.clear();
}

View File

@ -30,9 +30,8 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreUtils;
import org.apache.hadoop.hbase.regionserver.StripeStoreConfig;
import org.apache.hadoop.hbase.regionserver.StripeStoreFlusher;
@ -42,6 +41,7 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ConcatenatedLists;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
@ -63,12 +63,12 @@ public class StripeCompactionPolicy extends CompactionPolicy {
stripePolicy = new ExploringCompactionPolicy(conf, storeConfigInfo);
}
public List<StoreFile> preSelectFilesForCoprocessor(StripeInformationProvider si,
List<StoreFile> filesCompacting) {
public List<HStoreFile> preSelectFilesForCoprocessor(StripeInformationProvider si,
List<HStoreFile> filesCompacting) {
// We sincerely hope nobody is messing with us with their coprocessors.
// If they do, they are very likely to shoot themselves in the foot.
// We'll just exclude all the filesCompacting from the list.
ArrayList<StoreFile> candidateFiles = new ArrayList<>(si.getStorefiles());
ArrayList<HStoreFile> candidateFiles = new ArrayList<>(si.getStorefiles());
candidateFiles.removeAll(filesCompacting);
return candidateFiles;
}
@ -102,7 +102,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
}
public StripeCompactionRequest selectCompaction(StripeInformationProvider si,
List<StoreFile> filesCompacting, boolean isOffpeak) throws IOException {
List<HStoreFile> filesCompacting, boolean isOffpeak) throws IOException {
// TODO: first cut - no parallel compactions. To have more fine grained control we
// probably need structure more sophisticated than a list.
if (!filesCompacting.isEmpty()) {
@ -116,7 +116,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
// This can happen due to region split. We can skip it later; for now preserve
// compact-all-things behavior.
Collection<StoreFile> allFiles = si.getStorefiles();
Collection<HStoreFile> allFiles = si.getStorefiles();
if (StoreUtils.hasReferences(allFiles)) {
LOG.debug("There are references in the store; compacting all files");
long targetKvs = estimateTargetKvs(allFiles, config.getInitialCount()).getFirst();
@ -127,7 +127,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
}
int stripeCount = si.getStripeCount();
List<StoreFile> l0Files = si.getLevel0Files();
List<HStoreFile> l0Files = si.getLevel0Files();
// See if we need to make new stripes.
boolean shouldCompactL0 = (this.config.getLevel0MinFiles() <= l0Files.size());
@ -157,7 +157,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
return selectSingleStripeCompaction(si, false, canDropDeletesNoL0, isOffpeak);
}
public boolean needsCompactions(StripeInformationProvider si, List<StoreFile> filesCompacting) {
public boolean needsCompactions(StripeInformationProvider si, List<HStoreFile> filesCompacting) {
// Approximation on whether we need compaction.
return filesCompacting.isEmpty()
&& (StoreUtils.hasReferences(si.getStorefiles())
@ -166,7 +166,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
}
@Override
public boolean shouldPerformMajorCompaction(Collection<StoreFile> filesToCompact)
public boolean shouldPerformMajorCompaction(Collection<HStoreFile> filesToCompact)
throws IOException {
return false; // there's never a major compaction!
}
@ -182,7 +182,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
*/
protected boolean needsSingleStripeCompaction(StripeInformationProvider si) {
int minFiles = this.config.getStripeCompactMinFiles();
for (List<StoreFile> stripe : si.getStripes()) {
for (List<HStoreFile> stripe : si.getStripes()) {
if (stripe.size() >= minFiles) return true;
}
return false;
@ -190,20 +190,20 @@ public class StripeCompactionPolicy extends CompactionPolicy {
protected StripeCompactionRequest selectSingleStripeCompaction(StripeInformationProvider si,
boolean includeL0, boolean canDropDeletesWithoutL0, boolean isOffpeak) throws IOException {
ArrayList<ImmutableList<StoreFile>> stripes = si.getStripes();
ArrayList<ImmutableList<HStoreFile>> stripes = si.getStripes();
int bqIndex = -1;
List<StoreFile> bqSelection = null;
List<HStoreFile> bqSelection = null;
int stripeCount = stripes.size();
long bqTotalSize = -1;
for (int i = 0; i < stripeCount; ++i) {
// If we want to compact L0 to drop deletes, we only want whole-stripe compactions.
// So, pass includeL0 as 2nd parameter to indicate that.
List<StoreFile> selection = selectSimpleCompaction(stripes.get(i),
List<HStoreFile> selection = selectSimpleCompaction(stripes.get(i),
!canDropDeletesWithoutL0 && includeL0, isOffpeak);
if (selection.isEmpty()) continue;
long size = 0;
for (StoreFile sf : selection) {
for (HStoreFile sf : selection) {
size += sf.getReader().length();
}
if (bqSelection == null || selection.size() > bqSelection.size() ||
@ -217,7 +217,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
LOG.debug("No good compaction is possible in any stripe");
return null;
}
List<StoreFile> filesToCompact = new ArrayList<>(bqSelection);
List<HStoreFile> filesToCompact = new ArrayList<>(bqSelection);
// See if we can, and need to, split this stripe.
int targetCount = 1;
long targetKvs = Long.MAX_VALUE;
@ -244,9 +244,9 @@ public class StripeCompactionPolicy extends CompactionPolicy {
StripeCompactionRequest req;
if (includeL0) {
assert hasAllFiles;
List<StoreFile> l0Files = si.getLevel0Files();
List<HStoreFile> l0Files = si.getLevel0Files();
LOG.debug("Adding " + l0Files.size() + " files to compaction to be able to drop deletes");
ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<>();
ConcatenatedLists<HStoreFile> sfs = new ConcatenatedLists<>();
sfs.addSublist(filesToCompact);
sfs.addSublist(l0Files);
req = new BoundaryStripeCompactionRequest(sfs, si.getStripeBoundaries());
@ -267,33 +267,16 @@ public class StripeCompactionPolicy extends CompactionPolicy {
* @param allFilesOnly Whether a compaction of all-or-none files is needed.
* @return The resulting selection.
*/
private List<StoreFile> selectSimpleCompaction(
List<StoreFile> sfs, boolean allFilesOnly, boolean isOffpeak) {
private List<HStoreFile> selectSimpleCompaction(
List<HStoreFile> sfs, boolean allFilesOnly, boolean isOffpeak) {
int minFilesLocal = Math.max(
allFilesOnly ? sfs.size() : 0, this.config.getStripeCompactMinFiles());
int maxFilesLocal = Math.max(this.config.getStripeCompactMaxFiles(), minFilesLocal);
return stripePolicy.applyCompactionPolicy(sfs, false, isOffpeak, minFilesLocal, maxFilesLocal);
}
/**
* Selects the compaction that compacts all files (to be removed later).
* @param si StoreFileManager.
* @param targetStripeCount Target stripe count.
* @param targetSize Target stripe size.
* @return The compaction.
*/
private StripeCompactionRequest selectCompactionOfAllFiles(StripeInformationProvider si,
int targetStripeCount, long targetSize) {
Collection<StoreFile> allFiles = si.getStorefiles();
SplitStripeCompactionRequest request = new SplitStripeCompactionRequest(
allFiles, OPEN_KEY, OPEN_KEY, targetStripeCount, targetSize);
request.setMajorRangeFull();
LOG.debug("Selecting a compaction that includes all " + allFiles.size() + " files");
return request;
}
private StripeCompactionRequest selectNewStripesCompaction(StripeInformationProvider si) {
List<StoreFile> l0Files = si.getLevel0Files();
List<HStoreFile> l0Files = si.getLevel0Files();
Pair<Long, Integer> kvsAndCount = estimateTargetKvs(l0Files, config.getInitialCount());
LOG.debug("Creating " + kvsAndCount.getSecond() + " initial stripes with "
+ kvsAndCount.getFirst() + " kvs each via L0 compaction of " + l0Files.size() + " files");
@ -312,9 +295,9 @@ public class StripeCompactionPolicy extends CompactionPolicy {
long timestampCutoff = EnvironmentEdgeManager.currentTime() - cfTtl;
// Merge the longest sequence of stripes where all files have expired, if any.
int start = -1, bestStart = -1, length = 0, bestLength = 0;
ArrayList<ImmutableList<StoreFile>> stripes = si.getStripes();
ArrayList<ImmutableList<HStoreFile>> stripes = si.getStripes();
OUTER: for (int i = 0; i < stripes.size(); ++i) {
for (StoreFile storeFile : stripes.get(i)) {
for (HStoreFile storeFile : stripes.get(i)) {
if (storeFile.getReader().getMaxTimestamp() < timestampCutoff) continue;
// Found non-expired file, this stripe has to stay.
if (length > bestLength) {
@ -345,7 +328,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
}
LOG.debug("Merging " + bestLength + " stripes to delete expired store files");
int endIndex = bestStart + bestLength - 1;
ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<>();
ConcatenatedLists<HStoreFile> sfs = new ConcatenatedLists<>();
sfs.addAllSublists(stripes.subList(bestStart, endIndex + 1));
SplitStripeCompactionRequest result = new SplitStripeCompactionRequest(sfs,
si.getStartRow(bestStart), si.getEndRow(endIndex), 1, Long.MAX_VALUE);
@ -355,23 +338,23 @@ public class StripeCompactionPolicy extends CompactionPolicy {
return result;
}
private static long getTotalKvCount(final Collection<StoreFile> candidates) {
private static long getTotalKvCount(final Collection<HStoreFile> candidates) {
long totalSize = 0;
for (StoreFile storeFile : candidates) {
for (HStoreFile storeFile : candidates) {
totalSize += storeFile.getReader().getEntries();
}
return totalSize;
}
public static long getTotalFileSize(final Collection<StoreFile> candidates) {
public static long getTotalFileSize(final Collection<HStoreFile> candidates) {
long totalSize = 0;
for (StoreFile storeFile : candidates) {
for (HStoreFile storeFile : candidates) {
totalSize += storeFile.getReader().length();
}
return totalSize;
}
private Pair<Long, Integer> estimateTargetKvs(Collection<StoreFile> files, double splitCount) {
private Pair<Long, Integer> estimateTargetKvs(Collection<HStoreFile> files, double splitCount) {
// If the size is larger than what we target, we don't want to split into proportionally
// larger parts and then have to split again very soon. So, we will increase the multiplier
// by one until we get small enough parts. E.g. 5Gb stripe that should have been split into
@ -452,7 +435,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
this.targetBoundaries = targetBoundaries;
}
public BoundaryStripeCompactionRequest(Collection<StoreFile> files,
public BoundaryStripeCompactionRequest(Collection<HStoreFile> files,
List<byte[]> targetBoundaries) {
this(new CompactionRequest(files), targetBoundaries);
}
@ -494,16 +477,11 @@ public class StripeCompactionPolicy extends CompactionPolicy {
}
public SplitStripeCompactionRequest(
CompactionRequest request, byte[] startRow, byte[] endRow, long targetKvs) {
this(request, startRow, endRow, Integer.MAX_VALUE, targetKvs);
}
public SplitStripeCompactionRequest(
Collection<StoreFile> files, byte[] startRow, byte[] endRow, long targetKvs) {
Collection<HStoreFile> files, byte[] startRow, byte[] endRow, long targetKvs) {
this(files, startRow, endRow, Integer.MAX_VALUE, targetKvs);
}
public SplitStripeCompactionRequest(Collection<StoreFile> files,
public SplitStripeCompactionRequest(Collection<HStoreFile> files,
byte[] startRow, byte[] endRow, int targetCount, long targetKvs) {
this(new CompactionRequest(files), startRow, endRow, targetCount, targetKvs);
}
@ -524,7 +502,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
/** The information about stripes that the policy needs to do its stuff */
public static interface StripeInformationProvider {
public Collection<StoreFile> getStorefiles();
public Collection<HStoreFile> getStorefiles();
/**
* Gets the start row for a given stripe.
@ -543,7 +521,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
/**
* @return Level 0 files.
*/
public List<StoreFile> getLevel0Files();
public List<HStoreFile> getLevel0Files();
/**
* @return All stripe boundaries; including the open ones on both ends.
@ -553,7 +531,7 @@ public class StripeCompactionPolicy extends CompactionPolicy {
/**
* @return The stripes.
*/
public ArrayList<ImmutableList<StoreFile>> getStripes();
public ArrayList<ImmutableList<HStoreFile>> getStripes();
/**
* @return Stripe count.

View File

@ -24,15 +24,15 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
import org.apache.hadoop.hbase.regionserver.StripeMultiFileWriter;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
/**
* This is the placeholder for stripe compactor. The implementation, as well as the proper javadoc,
@ -42,7 +42,7 @@ import org.apache.hadoop.hbase.util.Bytes;
public class StripeCompactor extends AbstractMultiOutputCompactor<StripeMultiFileWriter> {
private static final Log LOG = LogFactory.getLog(StripeCompactor.class);
public StripeCompactor(Configuration conf, Store store) {
public StripeCompactor(Configuration conf, HStore store) {
super(conf, store);
}

View File

@ -37,16 +37,21 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
@ -54,10 +59,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDataManifest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
/**
* Utility class to help read/write the Snapshot Manifest.
@ -228,20 +229,20 @@ public final class SnapshotManifest {
// 2. iterate through all the stores in the region
LOG.debug("Creating references for hfiles");
for (Store store : region.getStores()) {
for (HStore store : region.getStores()) {
// 2.1. build the snapshot reference for the store
Object familyData = visitor.familyOpen(regionData,
store.getColumnFamilyDescriptor().getName());
monitor.rethrowException();
List<StoreFile> storeFiles = new ArrayList<>(store.getStorefiles());
List<HStoreFile> storeFiles = new ArrayList<>(store.getStorefiles());
if (LOG.isDebugEnabled()) {
LOG.debug("Adding snapshot references for " + storeFiles + " hfiles");
}
// 2.2. iterate through all the store's files and create "references".
for (int i = 0, sz = storeFiles.size(); i < sz; i++) {
StoreFile storeFile = storeFiles.get(i);
HStoreFile storeFile = storeFiles.get(i);
monitor.rethrowException();
// create "reference" to this store file.

View File

@ -34,6 +34,7 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Optional;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
@ -702,7 +703,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
Multimap<ByteBuffer, LoadQueueItem> regionGroups, final LoadQueueItem item, final Table table,
final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
Path hfilePath = item.getFilePath();
byte[] first, last;
Optional<byte[]> first, last;
try (HFile.Reader hfr = HFile.createReader(hfilePath.getFileSystem(getConf()), hfilePath,
new CacheConfig(getConf()), true, getConf())) {
hfr.loadFileInfo();
@ -713,19 +714,19 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
return new Pair<>(null, hfilePath.getName());
}
LOG.info("Trying to load hfile=" + hfilePath + " first=" + Bytes.toStringBinary(first) +
" last=" + Bytes.toStringBinary(last));
if (first == null || last == null) {
assert first == null && last == null;
LOG.info("Trying to load hfile=" + hfilePath + " first=" + first.map(Bytes::toStringBinary) +
" last=" + last.map(Bytes::toStringBinary));
if (!first.isPresent() || !last.isPresent()) {
assert !first.isPresent() && !last.isPresent();
// TODO what if this is due to a bad HFile?
LOG.info("hfile " + hfilePath + " has no entries, skipping");
return null;
}
if (Bytes.compareTo(first, last) > 0) {
throw new IllegalArgumentException(
"Invalid range: " + Bytes.toStringBinary(first) + " > " + Bytes.toStringBinary(last));
if (Bytes.compareTo(first.get(), last.get()) > 0) {
throw new IllegalArgumentException("Invalid range: " + Bytes.toStringBinary(first.get()) +
" > " + Bytes.toStringBinary(last.get()));
}
int idx = Arrays.binarySearch(startEndKeys.getFirst(), first, Bytes.BYTES_COMPARATOR);
int idx = Arrays.binarySearch(startEndKeys.getFirst(), first.get(), Bytes.BYTES_COMPARATOR);
if (idx < 0) {
// not on boundary, returns -(insertion index). Calculate region it
// would be in.
@ -753,7 +754,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
"Please use hbck tool to fix it first.");
}
boolean lastKeyInRange = Bytes.compareTo(last, startEndKeys.getSecond()[idx]) < 0 ||
boolean lastKeyInRange = Bytes.compareTo(last.get(), startEndKeys.getSecond()[idx]) < 0 ||
Bytes.equals(startEndKeys.getSecond()[idx], HConstants.EMPTY_BYTE_ARRAY);
if (!lastKeyInRange) {
List<LoadQueueItem> lqis = splitStoreFile(item, table,
@ -834,8 +835,8 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
" for family " + builder.getNameAsString());
}
reader.loadFileInfo();
byte[] first = reader.getFirstRowKey();
byte[] last = reader.getLastRowKey();
byte[] first = reader.getFirstRowKey().get();
byte[] last = reader.getLastRowKey().get();
LOG.info("Trying to figure out region boundaries hfile=" + hfile + " first=" +
Bytes.toStringBinary(first) + " last=" + Bytes.toStringBinary(last));

View File

@ -1,5 +1,4 @@
/*
*
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@ -16,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.util;
import java.io.DataInput;
@ -26,17 +24,17 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter;
import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterBase;
import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Handles Bloom filter initialization based on configuration and serialized
* metadata in the reader and writer of {@link org.apache.hadoop.hbase.regionserver.StoreFile}.
* Handles Bloom filter initialization based on configuration and serialized metadata in the reader
* and writer of {@link org.apache.hadoop.hbase.regionserver.HStoreFile}.
*/
@InterfaceAudience.Private
public final class BloomFilterFactory {
@ -155,7 +153,7 @@ public final class BloomFilterFactory {
/**
* Creates a new general (Row or RowCol) Bloom filter at the time of
* {@link org.apache.hadoop.hbase.regionserver.StoreFile} writing.
* {@link org.apache.hadoop.hbase.regionserver.HStoreFile} writing.
*
* @param conf
* @param cacheConf
@ -203,7 +201,7 @@ public final class BloomFilterFactory {
/**
* Creates a new Delete Family Bloom filter at the time of
* {@link org.apache.hadoop.hbase.regionserver.StoreFile} writing.
* {@link org.apache.hadoop.hbase.regionserver.HStoreFile} writing.
* @param conf
* @param cacheConf
* @param maxKeys an estimate of the number of keys we expect to insert.

View File

@ -50,6 +50,7 @@ import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Optional;
import java.util.Set;
import java.util.SortedMap;
import java.util.SortedSet;
@ -858,13 +859,13 @@ public class HBaseFsck extends Configured implements Closeable {
new CacheConfig(getConf()), true, getConf());
if ((reader.getFirstKey() != null)
&& ((storeFirstKey == null) || (comparator.compare(storeFirstKey,
((KeyValue.KeyOnlyKeyValue) reader.getFirstKey()).getKey()) > 0))) {
storeFirstKey = ((KeyValue.KeyOnlyKeyValue)reader.getFirstKey()).getKey();
((KeyValue.KeyOnlyKeyValue) reader.getFirstKey().get()).getKey()) > 0))) {
storeFirstKey = ((KeyValue.KeyOnlyKeyValue)reader.getFirstKey().get()).getKey();
}
if ((reader.getLastKey() != null)
&& ((storeLastKey == null) || (comparator.compare(storeLastKey,
((KeyValue.KeyOnlyKeyValue)reader.getLastKey()).getKey())) < 0)) {
storeLastKey = ((KeyValue.KeyOnlyKeyValue)reader.getLastKey()).getKey();
((KeyValue.KeyOnlyKeyValue)reader.getLastKey().get()).getKey())) < 0)) {
storeLastKey = ((KeyValue.KeyOnlyKeyValue)reader.getLastKey().get()).getKey();
}
reader.close();
}
@ -961,10 +962,10 @@ public class HBaseFsck extends Configured implements Closeable {
CacheConfig cacheConf = new CacheConfig(getConf());
hf = HFile.createReader(fs, hfile.getPath(), cacheConf, true, getConf());
hf.loadFileInfo();
Cell startKv = hf.getFirstKey();
start = CellUtil.cloneRow(startKv);
Cell endKv = hf.getLastKey();
end = CellUtil.cloneRow(endKv);
Optional<Cell> startKv = hf.getFirstKey();
start = CellUtil.cloneRow(startKv.get());
Optional<Cell> endKv = hf.getLastKey();
end = CellUtil.cloneRow(endKv.get());
} catch (IOException ioe) {
LOG.warn("Problem reading orphan file " + hfile + ", skipping");
continue;

View File

@ -17,14 +17,15 @@
*/
package org.apache.hadoop.hbase.util;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.LAST_BLOOM_KEY;
import java.io.IOException;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Handles ROW bloom related context. It works with both ByteBufferedCell and byte[] backed cells
@ -39,7 +40,7 @@ public class RowBloomContext extends BloomContext {
public void addLastBloomKey(Writer writer) throws IOException {
if (this.getLastCell() != null) {
byte[] key = CellUtil.copyRow(this.getLastCell());
writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, key);
writer.appendFileInfo(LAST_BLOOM_KEY, key);
}
}

View File

@ -17,14 +17,15 @@
*/
package org.apache.hadoop.hbase.util;
import static org.apache.hadoop.hbase.regionserver.HStoreFile.LAST_BLOOM_KEY;
import java.io.IOException;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.HFile.Writer;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Handles ROWCOL bloom related context. It can work with both BytebufferdCells
@ -43,7 +44,7 @@ public class RowColBloomContext extends BloomContext {
Cell firstOnRow = CellUtil.createFirstOnRowCol(this.getLastCell());
// This copy happens only once when the writer is closed
byte[] key = CellUtil.getCellKeySerializedAsKeyValueKey(firstOnRow);
writer.appendFileInfo(StoreFile.LAST_BLOOM_KEY, key);
writer.appendFileInfo(LAST_BLOOM_KEY, key);
}
}

View File

@ -95,7 +95,7 @@
List<? extends Store> stores = region.getStores();
for (Store store : stores) {
String cf = store.getColumnFamilyName();
Collection<StoreFile> storeFiles = store.getStorefiles(); %>
Collection<? extends StoreFile> storeFiles = store.getStorefiles(); %>
<h3>Column Family: <%= cf %></h2>

View File

@ -20,8 +20,6 @@ package org.apache.hadoop.hbase;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
@ -41,16 +39,14 @@ import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -59,6 +55,10 @@ import org.apache.hadoop.hbase.wal.WAL;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
/**
* Test for the case where a regionserver going down has enough cycles to do damage to regions that
* have actually been assigned elsehwere.
@ -201,7 +201,7 @@ public class TestIOFencing {
}
@Override
protected void completeCompaction(Collection<StoreFile> compactedFiles) throws IOException {
protected void completeCompaction(Collection<HStoreFile> compactedFiles) throws IOException {
try {
r.compactionsWaiting.countDown();
r.compactionsBlocked.await();

View File

@ -49,12 +49,10 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.exceptions.MergeRegionException;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.master.LoadBalancer;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.util.Bytes;
@ -69,6 +67,9 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.MergeTableRegionsRequest;
/**
* Class to test HBaseAdmin.
* Spins up the minicluster once at test start and then takes it down afterward.
@ -1320,8 +1321,8 @@ public class TestAdmin1 {
List<HRegion> regions = TEST_UTIL.getMiniHBaseCluster().getRegions(tableName);
for (HRegion r : regions) {
Store store = r.getStore(Bytes.toBytes(fn));
for (StoreFile sf : store.getStorefiles()) {
HStore store = r.getStore(Bytes.toBytes(fn));
for (HStoreFile sf : store.getStorefiles()) {
assertTrue(sf.toString().contains(fn));
assertTrue("Column family " + fn + " should have 3 copies",
FSUtils.getDefaultReplication(TEST_UTIL.getTestFileSystem(), sf.getPath()) == (sf
@ -1329,7 +1330,7 @@ public class TestAdmin1 {
}
store = r.getStore(Bytes.toBytes(fn1));
for (StoreFile sf : store.getStorefiles()) {
for (HStoreFile sf : store.getStorefiles()) {
assertTrue(sf.toString().contains(fn1));
assertTrue("Column family " + fn1 + " should have only 1 copy", 1 == sf.getFileInfo()
.getFileStatus().getReplication());

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.CachedBlock;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
import org.apache.hadoop.hbase.regionserver.Region;
@ -253,14 +254,16 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
}
public static class CompactorRegionObserver implements RegionObserver {
@Override
public InternalScanner preCompactScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Store store, List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs,
InternalScanner s, CompactionLifeCycleTracker request, long readPoint) throws IOException {
return createCompactorScanner(store, scanners, scanType, earliestPutTs);
Store store, List<? extends KeyValueScanner> scanners, ScanType scanType,
long earliestPutTs, InternalScanner s, CompactionLifeCycleTracker request, long readPoint)
throws IOException {
return createCompactorScanner((HStore) store, scanners, scanType, earliestPutTs);
}
private InternalScanner createCompactorScanner(Store store,
private InternalScanner createCompactorScanner(HStore store,
List<? extends KeyValueScanner> scanners, ScanType scanType, long earliestPutTs)
throws IOException {
return new CompactorStoreScanner(store, store.getScanInfo(), OptionalInt.empty(), scanners,
@ -270,7 +273,7 @@ public class TestAvoidCellReferencesIntoShippedBlocks {
private static class CompactorStoreScanner extends StoreScanner {
public CompactorStoreScanner(Store store, ScanInfo scanInfo, OptionalInt maxVersions,
public CompactorStoreScanner(HStore store, ScanInfo scanInfo, OptionalInt maxVersions,
List<? extends KeyValueScanner> scanners, ScanType scanType, long smallestReadPoint,
long earliestPutTs) throws IOException {
super(store, scanInfo, maxVersions, scanners, scanType, smallestReadPoint, earliestPutTs);

View File

@ -54,6 +54,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.CompareOperator;
import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
@ -69,7 +70,6 @@ import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.ClusterStatus.Option;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
@ -105,6 +105,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateR
import org.apache.hadoop.hbase.regionserver.DelegatingKeyValueScanner;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
import org.apache.hadoop.hbase.regionserver.Region;
@ -554,7 +555,7 @@ public class TestFromClientSide {
}
class MyStoreScanner extends StoreScanner {
public MyStoreScanner(Store store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
public MyStoreScanner(HStore store, ScanInfo scanInfo, Scan scan, NavigableSet<byte[]> columns,
long readPt) throws IOException {
super(store, scanInfo, scan, columns, readPt);
}
@ -588,7 +589,7 @@ public class TestFromClientSide {
public KeyValueScanner preStoreScannerOpen(ObserverContext<RegionCoprocessorEnvironment> c,
Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s,
final long readPt) throws IOException {
return new MyStoreScanner(store, store.getScanInfo(), scan, targetCols, readPt);
return new MyStoreScanner((HStore) store, store.getScanInfo(), scan, targetCols, readPt);
}
}

View File

@ -19,6 +19,11 @@
package org.apache.hadoop.hbase.coprocessor;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.List;
import java.util.Map;
@ -57,16 +62,12 @@ import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileReader;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.WALEdit;
import org.apache.hadoop.hbase.wal.WALKey;
import static org.junit.Assert.assertArrayEquals;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
/**
* A sample region observer that tests the RegionObserver interface.
@ -202,13 +203,13 @@ public class SimpleRegionObserver implements RegionObserver {
@Override
public void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
List<StoreFile> candidates, CompactionLifeCycleTracker tracker) throws IOException {
List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker) throws IOException {
ctPreCompactSelect.incrementAndGet();
}
@Override
public void postCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
ImmutableList<StoreFile> selected, CompactionLifeCycleTracker tracker) {
ImmutableList<? extends StoreFile> selected, CompactionLifeCycleTracker tracker) {
ctPostCompactSelect.incrementAndGet();
}

View File

@ -63,7 +63,6 @@ import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreScanner;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.testclassification.CoprocessorTests;
@ -116,7 +115,7 @@ public class TestRegionObserverScannerOpenHook {
Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPt)
throws IOException {
scan.setFilter(new NoDataFilter());
return new StoreScanner(store, store.getScanInfo(), scan, targetCols, readPt);
return new StoreScanner((HStore) store, store.getScanInfo(), scan, targetCols, readPt);
}
}

View File

@ -96,7 +96,7 @@ public class TestHalfStoreFileReader {
HFile.Reader r = HFile.createReader(fs, p, cacheConf, true, conf);
r.loadFileInfo();
Cell midKV = r.midkey();
Cell midKV = r.midKey().get();
byte[] midkey = CellUtil.cloneRow(midKV);
// System.out.println("midkey: " + midKV + " or: " + Bytes.toStringBinary(midkey));
@ -155,7 +155,7 @@ public class TestHalfStoreFileReader {
HFile.Reader r = HFile.createReader(fs, p, cacheConf, true, conf);
r.loadFileInfo();
Cell midKV = r.midkey();
Cell midKV = r.midKey().get();
byte[] midkey = CellUtil.cloneRow(midKV);
Reference bottom = new Reference(midkey, Reference.Range.bottom);

View File

@ -164,8 +164,8 @@ public class TestHFile {
w.close();
Reader r = HFile.createReader(fs, f, cacheConf, true, conf);
r.loadFileInfo();
assertNull(r.getFirstKey());
assertNull(r.getLastKey());
assertFalse(r.getFirstKey().isPresent());
assertFalse(r.getLastKey().isPresent());
}
/**

View File

@ -571,7 +571,7 @@ public class TestHFileBlockIndex {
boolean hasArrayIndexOutOfBoundsException = false;
try {
// get the mid-key.
reader.midkey();
reader.midKey();
} catch (ArrayIndexOutOfBoundsException e) {
hasArrayIndexOutOfBoundsException = true;
} finally {
@ -649,8 +649,8 @@ public class TestHFileBlockIndex {
assertEquals(expectedNumLevels,
reader.getTrailer().getNumDataIndexLevels());
assertTrue(Bytes.equals(keys[0], ((KeyValue)reader.getFirstKey()).getKey()));
assertTrue(Bytes.equals(keys[NUM_KV - 1], ((KeyValue)reader.getLastKey()).getKey()));
assertTrue(Bytes.equals(keys[0], ((KeyValue)reader.getFirstKey().get()).getKey()));
assertTrue(Bytes.equals(keys[NUM_KV - 1], ((KeyValue)reader.getLastKey().get()).getKey()));
LOG.info("Last key: " + Bytes.toStringBinary(keys[NUM_KV - 1]));
for (boolean pread : new boolean[] { false, true }) {
@ -706,7 +706,7 @@ public class TestHFileBlockIndex {
// Validate the mid-key.
assertEquals(
Bytes.toStringBinary(blockKeys.get((blockKeys.size() - 1) / 2)),
reader.midkey());
reader.midKey());
assertEquals(UNCOMPRESSED_INDEX_SIZES[testI],
reader.getTrailer().getUncompressedDataIndexSize());

View File

@ -186,8 +186,8 @@ public class TestHFileSeek extends TestCase {
Reader reader = HFile.createReaderFromStream(path, fsdis,
fs.getFileStatus(path).getLen(), new CacheConfig(conf), conf);
reader.loadFileInfo();
KeySampler kSampler = new KeySampler(rng, ((KeyValue) reader.getFirstKey()).getKey(),
((KeyValue) reader.getLastKey()).getKey(), keyLenGen);
KeySampler kSampler = new KeySampler(rng, ((KeyValue) reader.getFirstKey().get()).getKey(),
((KeyValue) reader.getLastKey().get()).getKey(), keyLenGen);
HFileScanner scanner = reader.getScanner(false, USE_PREAD);
BytesWritable key = new BytesWritable();
timer.reset();

View File

@ -723,8 +723,8 @@ public class TestMobCompactor {
@Override
public void preCompactSelection(ObserverContext<RegionCoprocessorEnvironment> c, Store store,
List<StoreFile> candidates, CompactionLifeCycleTracker tracker) throws IOException {
List<? extends StoreFile> candidates, CompactionLifeCycleTracker tracker)
throws IOException {
int count = candidates.size();
if (count >= 2) {
for (int i = 0; i < count - 2; i++) {
@ -815,7 +815,7 @@ public class TestMobCompactor {
Assert.assertTrue(hasFiles);
Path path = files[0].getPath();
CacheConfig cacheConf = new CacheConfig(conf);
StoreFile sf = new HStoreFile(TEST_UTIL.getTestFileSystem(), path, conf, cacheConf,
HStoreFile sf = new HStoreFile(TEST_UTIL.getTestFileSystem(), path, conf, cacheConf,
BloomType.NONE, true);
sf.initReader();
HFile.Reader reader = sf.getReader().getHFileReader();

View File

@ -18,6 +18,7 @@
*/
package org.apache.hadoop.hbase.mob.compactions;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
@ -71,7 +72,6 @@ import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
import org.apache.hadoop.hbase.regionserver.ScanInfo;
import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.regionserver.StoreScanner;
@ -81,7 +81,6 @@ import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
@ -494,12 +493,12 @@ public class TestPartitionedMobCompactor {
PartitionedMobCompactionRequest request = select(files, isForceAllFiles);
// Make sure that there is no del Partitions
Assert.assertTrue(request.getDelPartitions().size() == 0);
assertTrue(request.getDelPartitions().size() == 0);
// Make sure that when there is no startKey/endKey for partition.
for (CompactionPartition p : request.getCompactionPartitions()) {
Assert.assertTrue(p.getStartKey() == null);
Assert.assertTrue(p.getEndKey() == null);
assertTrue(p.getStartKey() == null);
assertTrue(p.getEndKey() == null);
}
return null;
}
@ -530,18 +529,18 @@ public class TestPartitionedMobCompactor {
}
PartitionedMobCompactionRequest request = select(files, isForceAllFiles);
Assert.assertTrue(request.getDelPartitions().size() == delPartitionSize);
assertTrue(request.getDelPartitions().size() == delPartitionSize);
if (request.getDelPartitions().size() > 0) {
for (CompactionPartition p : request.getCompactionPartitions()) {
Assert.assertTrue(p.getStartKey() != null);
Assert.assertTrue(p.getEndKey() != null);
assertTrue(p.getStartKey() != null);
assertTrue(p.getEndKey() != null);
}
}
try {
for (CompactionDelPartition delPartition : request.getDelPartitions()) {
for (Path newDelPath : delPartition.listDelFiles()) {
StoreFile sf =
HStoreFile sf =
new HStoreFile(fs, newDelPath, conf, this.cacheConfig, BloomType.NONE, true);
// pre-create reader of a del file to avoid race condition when opening the reader in
// each partition.
@ -553,11 +552,11 @@ public class TestPartitionedMobCompactor {
// Make sure that CompactionDelPartitions does not overlap
CompactionDelPartition prevDelP = null;
for (CompactionDelPartition delP : request.getDelPartitions()) {
Assert.assertTrue(
assertTrue(
Bytes.compareTo(delP.getId().getStartKey(), delP.getId().getEndKey()) <= 0);
if (prevDelP != null) {
Assert.assertTrue(
assertTrue(
Bytes.compareTo(prevDelP.getId().getEndKey(), delP.getId().getStartKey()) < 0);
}
}
@ -567,7 +566,7 @@ public class TestPartitionedMobCompactor {
// Make sure that only del files within key range for a partition is included in compaction.
// compact the mob files by partitions in parallel.
for (CompactionPartition partition : request.getCompactionPartitions()) {
List<StoreFile> delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions());
List<HStoreFile> delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions());
if (!request.getDelPartitions().isEmpty()) {
if (!((Bytes.compareTo(request.getDelPartitions().get(0).getId().getStartKey(),
partition.getEndKey()) > 0) || (Bytes.compareTo(
@ -575,23 +574,23 @@ public class TestPartitionedMobCompactor {
.getEndKey(), partition.getStartKey()) < 0))) {
if (delFiles.size() > 0) {
Assert.assertTrue(delFiles.size() == 1);
assertTrue(delFiles.size() == 1);
affectedPartitions += delFiles.size();
Assert.assertTrue(Bytes.compareTo(partition.getStartKey(),
CellUtil.cloneRow(delFiles.get(0).getLastKey())) <= 0);
Assert.assertTrue(Bytes.compareTo(partition.getEndKey(),
CellUtil.cloneRow(delFiles.get(delFiles.size() - 1).getFirstKey())) >= 0);
assertTrue(Bytes.compareTo(partition.getStartKey(),
CellUtil.cloneRow(delFiles.get(0).getLastKey().get())) <= 0);
assertTrue(Bytes.compareTo(partition.getEndKey(),
CellUtil.cloneRow(delFiles.get(delFiles.size() - 1).getFirstKey().get())) >= 0);
}
}
}
}
// The del file is only included in one partition
Assert.assertTrue(affectedPartitions == PartitionsIncludeDelFiles);
assertTrue(affectedPartitions == PartitionsIncludeDelFiles);
} finally {
for (CompactionDelPartition delPartition : request.getDelPartitions()) {
for (StoreFile storeFile : delPartition.getStoreFiles()) {
for (HStoreFile storeFile : delPartition.getStoreFiles()) {
try {
storeFile.closeReader(true);
storeFile.closeStoreFile(true);
} catch (IOException e) {
LOG.warn("Failed to close the reader on store file " + storeFile.getPath(), e);
}
@ -679,19 +678,19 @@ public class TestPartitionedMobCompactor {
// Make sure that when there is no del files, there will be no startKey/endKey for partition.
if (request.getDelPartitions().size() == 0) {
for (CompactionPartition p : request.getCompactionPartitions()) {
Assert.assertTrue(p.getStartKey() == null);
Assert.assertTrue(p.getEndKey() == null);
assertTrue(p.getStartKey() == null);
assertTrue(p.getEndKey() == null);
}
}
// Make sure that CompactionDelPartitions does not overlap
CompactionDelPartition prevDelP = null;
for (CompactionDelPartition delP : request.getDelPartitions()) {
Assert.assertTrue(Bytes.compareTo(delP.getId().getStartKey(),
assertTrue(Bytes.compareTo(delP.getId().getStartKey(),
delP.getId().getEndKey()) <= 0);
if (prevDelP != null) {
Assert.assertTrue(Bytes.compareTo(prevDelP.getId().getEndKey(),
assertTrue(Bytes.compareTo(prevDelP.getId().getEndKey(),
delP.getId().getStartKey()) < 0);
}
}
@ -699,25 +698,24 @@ public class TestPartitionedMobCompactor {
// Make sure that only del files within key range for a partition is included in compaction.
// compact the mob files by partitions in parallel.
for (CompactionPartition partition : request.getCompactionPartitions()) {
List<StoreFile> delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions());
List<HStoreFile> delFiles = getListOfDelFilesForPartition(partition, request.getDelPartitions());
if (!request.getDelPartitions().isEmpty()) {
if (!((Bytes.compareTo(request.getDelPartitions().get(0).getId().getStartKey(),
partition.getEndKey()) > 0) || (Bytes.compareTo(
request.getDelPartitions().get(request.getDelPartitions().size() - 1).getId()
.getEndKey(), partition.getStartKey()) < 0))) {
if (delFiles.size() > 0) {
Assert.assertTrue(Bytes
.compareTo(partition.getStartKey(), delFiles.get(0).getFirstKey().getRowArray())
>= 0);
Assert.assertTrue(Bytes.compareTo(partition.getEndKey(),
delFiles.get(delFiles.size() - 1).getLastKey().getRowArray()) <= 0);
assertTrue(Bytes.compareTo(partition.getStartKey(),
delFiles.get(0).getFirstKey().get().getRowArray()) >= 0);
assertTrue(Bytes.compareTo(partition.getEndKey(),
delFiles.get(delFiles.size() - 1).getLastKey().get().getRowArray()) <= 0);
}
}
}
}
// assert the compaction type
Assert.assertEquals(type, request.type);
assertEquals(type, request.type);
// assert get the right partitions
compareCompactedPartitions(expected, request.compactionPartitions);
// assert get the right del files
@ -750,8 +748,8 @@ public class TestPartitionedMobCompactor {
}
List<Path> newDelPaths = compactDelFiles(request, delFilePaths);
// assert the del files are merged.
Assert.assertEquals(expectedFileCount, newDelPaths.size());
Assert.assertEquals(expectedCellCount, countDelCellsInDelFiles(newDelPaths));
assertEquals(expectedFileCount, newDelPaths.size());
assertEquals(expectedCellCount, countDelCellsInDelFiles(newDelPaths));
return null;
}
};
@ -784,9 +782,9 @@ public class TestPartitionedMobCompactor {
}
Collections.sort(expected);
Collections.sort(actualKeys);
Assert.assertEquals(expected.size(), actualKeys.size());
assertEquals(expected.size(), actualKeys.size());
for (int i = 0; i < expected.size(); i++) {
Assert.assertEquals(expected.get(i), actualKeys.get(i));
assertEquals(expected.get(i), actualKeys.get(i));
}
}
@ -802,7 +800,7 @@ public class TestPartitionedMobCompactor {
}
}
for (Path f : delFiles) {
Assert.assertTrue(delMap.containsKey(f));
assertTrue(delMap.containsKey(f));
}
}
@ -874,10 +872,10 @@ public class TestPartitionedMobCompactor {
* @return the cell size
*/
private int countDelCellsInDelFiles(List<Path> paths) throws IOException {
List<StoreFile> sfs = new ArrayList<>();
List<HStoreFile> sfs = new ArrayList<>();
int size = 0;
for (Path path : paths) {
StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
HStoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
sfs.add(sf);
}
List<KeyValueScanner> scanners = new ArrayList<>(StoreFileScanner.getScannersForStoreFiles(sfs,

View File

@ -38,16 +38,15 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter.Predicate;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HStore;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.regionserver.HStoreFile;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience;
import org.junit.rules.TestName;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.HashMultimap;
@ -435,10 +434,9 @@ public class SpaceQuotaHelperForTests {
@Override
public boolean evaluate() throws Exception {
for (HRegion region : cluster.getRegions(tn)) {
for (Store store : region.getStores()) {
HStore hstore = (HStore) store;
Collection<StoreFile> files =
hstore.getStoreEngine().getStoreFileManager().getCompactedfiles();
for (HStore store : region.getStores()) {
Collection<HStoreFile> files =
store.getStoreEngine().getStoreFileManager().getCompactedfiles();
if (null != files && !files.isEmpty()) {
LOG.debug(region.getRegionInfo().getEncodedName() + " still has compacted files");
return false;

View File

@ -19,9 +19,6 @@ package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.assertEquals;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@ -32,9 +29,12 @@ import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequ
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.ImmutableList;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
public class AbstractTestDateTieredCompactionPolicy extends TestCompactionPolicy {
protected ArrayList<StoreFile> sfCreate(long[] minTimestamps, long[] maxTimestamps, long[] sizes)
protected ArrayList<HStoreFile> sfCreate(long[] minTimestamps, long[] maxTimestamps, long[] sizes)
throws IOException {
ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge();
EnvironmentEdgeManager.injectEdge(timeMachine);
@ -45,17 +45,17 @@ public class AbstractTestDateTieredCompactionPolicy extends TestCompactionPolicy
ageInDisk.add(0L);
}
ArrayList<StoreFile> ret = Lists.newArrayList();
ArrayList<HStoreFile> ret = Lists.newArrayList();
for (int i = 0; i < sizes.length; i++) {
MockStoreFile msf =
new MockStoreFile(TEST_UTIL, TEST_FILE, sizes[i], ageInDisk.get(i), false, i);
MockHStoreFile msf =
new MockHStoreFile(TEST_UTIL, TEST_FILE, sizes[i], ageInDisk.get(i), false, i);
msf.setTimeRangeTracker(new TimeRangeTracker(minTimestamps[i], maxTimestamps[i]));
ret.add(msf);
}
return ret;
}
protected void compactEquals(long now, ArrayList<StoreFile> candidates, long[] expectedFileSizes,
protected void compactEquals(long now, ArrayList<HStoreFile> candidates, long[] expectedFileSizes,
long[] expectedBoundaries, boolean isMajor, boolean toCompact) throws IOException {
ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge();
EnvironmentEdgeManager.injectEdge(timeMachine);
@ -64,17 +64,17 @@ public class AbstractTestDateTieredCompactionPolicy extends TestCompactionPolicy
DateTieredCompactionPolicy policy =
(DateTieredCompactionPolicy) store.storeEngine.getCompactionPolicy();
if (isMajor) {
for (StoreFile file : candidates) {
((MockStoreFile) file).setIsMajor(true);
for (HStoreFile file : candidates) {
((MockHStoreFile) file).setIsMajor(true);
}
assertEquals(toCompact, policy.shouldPerformMajorCompaction(candidates));
request = (DateTieredCompactionRequest) policy.selectMajorCompaction(candidates);
} else {
assertEquals(toCompact, policy.needsCompaction(candidates, ImmutableList.<StoreFile> of()));
assertEquals(toCompact, policy.needsCompaction(candidates, ImmutableList.of()));
request =
(DateTieredCompactionRequest) policy.selectMinorCompaction(candidates, false, false);
}
List<StoreFile> actual = Lists.newArrayList(request.getFiles());
List<HStoreFile> actual = Lists.newArrayList(request.getFiles());
assertEquals(Arrays.toString(expectedFileSizes), Arrays.toString(getSizes(actual)));
assertEquals(Arrays.toString(expectedBoundaries),
Arrays.toString(request.getBoundaries().toArray()));

View File

@ -592,7 +592,7 @@ public class DataBlockEncodingTool {
Path path = new Path(hfilePath);
CacheConfig cacheConf = new CacheConfig(conf);
FileSystem fs = FileSystem.get(conf);
StoreFile hsf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
HStoreFile hsf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
hsf.initReader();
StoreFileReader reader = hsf.getReader();
reader.loadFileInfo();

View File

@ -59,7 +59,7 @@ public class EncodedSeekPerformanceTest {
List<Cell> allKeyValues = new ArrayList<>();
// read all of the key values
StoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(),
HStoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(),
path, configuration, cacheConf, BloomType.NONE, true);
storeFile.initReader();
StoreFileReader reader = storeFile.getReader();
@ -71,7 +71,7 @@ public class EncodedSeekPerformanceTest {
allKeyValues.add(current);
}
storeFile.closeReader(cacheConf.shouldEvictOnClose());
storeFile.closeStoreFile(cacheConf.shouldEvictOnClose());
// pick seeks by random
List<Cell> seeks = new ArrayList<>();
@ -89,7 +89,7 @@ public class EncodedSeekPerformanceTest {
private void runTest(Path path, DataBlockEncoding blockEncoding,
List<Cell> seeks) throws IOException {
// read all of the key values
StoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(),
HStoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(),
path, configuration, cacheConf, BloomType.NONE, true);
storeFile.initReader();
long totalSize = 0;
@ -132,7 +132,7 @@ public class EncodedSeekPerformanceTest {
double seeksPerSec = (seeks.size() * NANOSEC_IN_SEC) /
(finishSeeksTime - startSeeksTime);
storeFile.closeReader(cacheConf.shouldEvictOnClose());
storeFile.closeStoreFile(cacheConf.shouldEvictOnClose());
clearBlockCache();
System.out.println(blockEncoding);

View File

@ -15,26 +15,30 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.Arrays;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalLong;
import java.util.TreeMap;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.CellBuilderFactory;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.yetus.audience.InterfaceAudience;
/** A mock used so our tests don't deal with actual StoreFiles */
public class MockStoreFile extends HStoreFile {
@InterfaceAudience.Private
public class MockHStoreFile extends HStoreFile {
long length = 0;
boolean isRef = false;
long ageInDisk;
@ -48,7 +52,7 @@ public class MockStoreFile extends HStoreFile {
long modificationTime;
boolean compactedAway;
MockStoreFile(HBaseTestingUtility testUtil, Path testPath,
MockHStoreFile(HBaseTestingUtility testUtil, Path testPath,
long length, long ageInDisk, boolean isRef, long sequenceid) throws IOException {
super(testUtil.getTestFileSystem(), testPath, testUtil.getConfiguration(),
new CacheConfig(testUtil.getConfiguration()), BloomType.NONE, true);
@ -184,29 +188,34 @@ public class MockStoreFile extends HStoreFile {
}
@Override
public Cell getLastKey() {
public Optional<Cell> getLastKey() {
if (splitPoint != null) {
return CellUtil.createCell(Arrays.copyOf(splitPoint, splitPoint.length + 1));
return Optional.of(CellBuilderFactory.create(CellBuilderType.DEEP_COPY)
.setType(KeyValue.Type.Put.getCode())
.setRow(Arrays.copyOf(splitPoint, splitPoint.length + 1)).build());
} else {
return null;
return Optional.empty();
}
}
@Override
public Cell midkey() throws IOException {
public Optional<Cell> midKey() throws IOException {
if (splitPoint != null) {
return CellUtil.createCell(splitPoint);
return Optional.of(CellBuilderFactory.create(CellBuilderType.DEEP_COPY)
.setType(KeyValue.Type.Put.getCode()).setRow(splitPoint).build());
} else {
return null;
return Optional.empty();
}
}
@Override
public Cell getFirstKey() {
public Optional<Cell> getFirstKey() {
if (splitPoint != null) {
return CellUtil.createCell(Arrays.copyOf(splitPoint, splitPoint.length - 1));
return Optional.of(CellBuilderFactory.create(CellBuilderType.DEEP_COPY)
.setType(KeyValue.Type.Put.getCode()).setRow(splitPoint, 0, splitPoint.length - 1)
.build());
} else {
return null;
return Optional.empty();
}
}
};

View File

@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.RegionObserver;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
/**
* RegionObserver that just reimplements the default behavior,
@ -51,7 +50,7 @@ public class NoOpScanPolicyObserver implements RegionObserver {
ScanInfo oldSI = store.getScanInfo();
ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(),
oldSI.getTtl(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator());
return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners,
return new StoreScanner((HStore) store, scanInfo, OptionalInt.empty(), scanners,
ScanType.COMPACT_RETAIN_DELETES, store.getSmallestReadPoint(), HConstants.OLDEST_TIMESTAMP);
}
@ -67,7 +66,7 @@ public class NoOpScanPolicyObserver implements RegionObserver {
ScanInfo oldSI = store.getScanInfo();
ScanInfo scanInfo = new ScanInfo(oldSI.getConfiguration(), store.getColumnFamilyDescriptor(),
oldSI.getTtl(), oldSI.getTimeToPurgeDeletes(), oldSI.getComparator());
return new StoreScanner(store, scanInfo, OptionalInt.empty(), scanners, scanType,
return new StoreScanner((HStore) store, scanInfo, OptionalInt.empty(), scanners, scanType,
store.getSmallestReadPoint(), earliestPutTs);
}
@ -76,10 +75,10 @@ public class NoOpScanPolicyObserver implements RegionObserver {
Store store, Scan scan, NavigableSet<byte[]> targetCols, KeyValueScanner s, long readPoint)
throws IOException {
Region r = c.getEnvironment().getRegion();
return scan.isReversed() ? new ReversedStoreScanner(store,
store.getScanInfo(), scan, targetCols, r.getReadPoint(scan
.getIsolationLevel())) : new StoreScanner(store,
store.getScanInfo(), scan, targetCols, r.getReadPoint(scan
.getIsolationLevel()));
return scan.isReversed()
? new ReversedStoreScanner((HStore) store, store.getScanInfo(), scan, targetCols,
r.getReadPoint(scan.getIsolationLevel()))
: new StoreScanner((HStore) store, store.getScanInfo(), scan, targetCols,
r.getReadPoint(scan.getIsolationLevel()));
}
}

View File

@ -218,7 +218,7 @@ public class TestCacheOnWriteInSchema {
private void readStoreFile(Path path) throws IOException {
CacheConfig cacheConf = store.getCacheConfig();
BlockCache cache = cacheConf.getBlockCache();
StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.ROWCOL, true);
HStoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.ROWCOL, true);
sf.initReader();
HFile.Reader reader = sf.getReader().getHFileReader();
try {

View File

@ -22,6 +22,7 @@ import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY_BYTES;
import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.mockito.Matchers.any;
@ -226,7 +227,7 @@ public class TestCompaction {
private int count() throws IOException {
int count = 0;
for (StoreFile f: this.r.stores.
for (HStoreFile f: this.r.stores.
get(COLUMN_FAMILY_TEXT).getStorefiles()) {
HFileScanner scanner = f.getReader().getScanner(false, false);
if (!scanner.seekTo()) {
@ -255,9 +256,9 @@ public class TestCompaction {
for (int i = 0; i < nfiles; i++) {
createStoreFile(r);
}
HStore store = (HStore) r.getStore(COLUMN_FAMILY);
HStore store = r.getStore(COLUMN_FAMILY);
Collection<StoreFile> storeFiles = store.getStorefiles();
Collection<HStoreFile> storeFiles = store.getStorefiles();
DefaultCompactor tool = (DefaultCompactor)store.storeEngine.getCompactor();
tool.compactForTesting(storeFiles, false);
@ -276,8 +277,8 @@ public class TestCompaction {
} catch (Exception e) {
// The complete compaction should fail and the corrupt file should remain
// in the 'tmp' directory;
assert (fs.exists(origPath));
assert (!fs.exists(dstPath));
assertTrue(fs.exists(origPath));
assertFalse(fs.exists(dstPath));
System.out.println("testCompactionWithCorruptResult Passed");
return;
}
@ -389,8 +390,8 @@ public class TestCompaction {
}
private class StoreMockMaker extends StatefulStoreMockMaker {
public ArrayList<StoreFile> compacting = new ArrayList<>();
public ArrayList<StoreFile> notCompacting = new ArrayList<>();
public ArrayList<HStoreFile> compacting = new ArrayList<>();
public ArrayList<HStoreFile> notCompacting = new ArrayList<>();
private ArrayList<Integer> results;
public StoreMockMaker(ArrayList<Integer> results) {
@ -398,19 +399,21 @@ public class TestCompaction {
}
public class TestCompactionContext extends CompactionContext {
private List<StoreFile> selectedFiles;
public TestCompactionContext(List<StoreFile> selectedFiles) {
private List<HStoreFile> selectedFiles;
public TestCompactionContext(List<HStoreFile> selectedFiles) {
super();
this.selectedFiles = selectedFiles;
}
@Override
public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) {
return new ArrayList<>();
}
@Override
public boolean select(List<StoreFile> filesCompacting, boolean isUserCompaction,
public boolean select(List<HStoreFile> filesCompacting, boolean isUserCompaction,
boolean mayUseOffPeak, boolean forceMajor) throws IOException {
this.request = new CompactionRequest(selectedFiles);
this.request.setPriority(getPriority());
@ -445,7 +448,7 @@ public class TestCompaction {
notCompacting.addAll(ctx.selectedFiles);
}
public synchronized void finishCompaction(List<StoreFile> sfs) {
public synchronized void finishCompaction(List<HStoreFile> sfs) {
if (sfs.isEmpty()) return;
synchronized (results) {
results.add(sfs.size());
@ -466,7 +469,9 @@ public class TestCompaction {
public volatile boolean isInCompact = false;
public void unblock() {
synchronized (this) { this.notifyAll(); }
synchronized (this) {
this.notifyAll();
}
}
@Override
@ -484,12 +489,12 @@ public class TestCompaction {
}
@Override
public List<StoreFile> preSelect(List<StoreFile> filesCompacting) {
public List<HStoreFile> preSelect(List<HStoreFile> filesCompacting) {
return new ArrayList<>();
}
@Override
public boolean select(List<StoreFile> f, boolean i, boolean m, boolean e)
public boolean select(List<HStoreFile> f, boolean i, boolean m, boolean e)
throws IOException {
this.request = new CompactionRequest(new ArrayList<>());
return true;
@ -673,14 +678,14 @@ public class TestCompaction {
}
public static class DummyCompactor extends DefaultCompactor {
public DummyCompactor(Configuration conf, Store store) {
public DummyCompactor(Configuration conf, HStore store) {
super(conf, store);
this.keepSeqIdPeriod = 0;
}
}
private static StoreFile createFile() throws Exception {
StoreFile sf = mock(StoreFile.class);
private static HStoreFile createFile() throws Exception {
HStoreFile sf = mock(HStoreFile.class);
when(sf.getPath()).thenReturn(new Path("file"));
StoreFileReader r = mock(StoreFileReader.class);
when(r.length()).thenReturn(10L);

View File

@ -1,4 +1,4 @@
/*
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.assertEquals;
@ -89,7 +88,7 @@ public class TestCompactionArchiveConcurrentClose {
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(new HColumnDescriptor(fam));
HRegionInfo info = new HRegionInfo(tableName, null, null, false);
Region region = initHRegion(htd, info);
HRegion region = initHRegion(htd, info);
RegionServerServices rss = mock(RegionServerServices.class);
List<Region> regions = new ArrayList<>();
regions.add(region);
@ -112,12 +111,12 @@ public class TestCompactionArchiveConcurrentClose {
region.flush(true);
}
Store store = region.getStore(fam);
HStore store = region.getStore(fam);
assertEquals(fileCount, store.getStorefilesCount());
Collection<StoreFile> storefiles = store.getStorefiles();
Collection<HStoreFile> storefiles = store.getStorefiles();
// None of the files should be in compacted state.
for (StoreFile file : storefiles) {
for (HStoreFile file : storefiles) {
assertFalse(file.isCompactedAway());
}
// Do compaction
@ -157,7 +156,7 @@ public class TestCompactionArchiveConcurrentClose {
}
}
private Region initHRegion(HTableDescriptor htd, HRegionInfo info)
private HRegion initHRegion(HTableDescriptor htd, HRegionInfo info)
throws IOException {
Configuration conf = testUtil.getConfiguration();
Path tableDir = FSUtils.getTableDir(testDir, htd.getTableName());
@ -185,7 +184,7 @@ public class TestCompactionArchiveConcurrentClose {
}
@Override
public void removeStoreFiles(String familyName, Collection<StoreFile> storeFiles)
public void removeStoreFiles(String familyName, Collection<HStoreFile> storeFiles)
throws IOException {
super.removeStoreFiles(familyName, storeFiles);
archived.set(true);

View File

@ -120,24 +120,24 @@ public class TestCompactionArchiveIOException {
region.flush(true);
}
HStore store = (HStore) region.getStore(fam);
HStore store = region.getStore(fam);
assertEquals(fileCount, store.getStorefilesCount());
Collection<StoreFile> storefiles = store.getStorefiles();
Collection<HStoreFile> storefiles = store.getStorefiles();
// None of the files should be in compacted state.
for (StoreFile file : storefiles) {
for (HStoreFile file : storefiles) {
assertFalse(file.isCompactedAway());
}
StoreFileManager fileManager = store.getStoreEngine().getStoreFileManager();
Collection<StoreFile> initialCompactedFiles = fileManager.getCompactedfiles();
Collection<HStoreFile> initialCompactedFiles = fileManager.getCompactedfiles();
assertTrue(initialCompactedFiles == null || initialCompactedFiles.isEmpty());
// Do compaction
region.compact(true);
// all prior store files should now be compacted
Collection<StoreFile> compactedFilesPreClean = fileManager.getCompactedfiles();
Collection<HStoreFile> compactedFilesPreClean = fileManager.getCompactedfiles();
assertNotNull(compactedFilesPreClean);
assertTrue(compactedFilesPreClean.size() > 0);
@ -148,17 +148,17 @@ public class TestCompactionArchiveIOException {
out.writeInt(1);
out.close();
StoreFile errStoreFile = new MockStoreFile(testUtil, errFile, 1, 0, false, 1);
HStoreFile errStoreFile = new MockHStoreFile(testUtil, errFile, 1, 0, false, 1);
fileManager.addCompactionResults(
ImmutableList.of(errStoreFile), ImmutableList.<StoreFile>of());
ImmutableList.of(errStoreFile), ImmutableList.of());
// cleanup compacted files
cleaner.chore();
// make sure the compacted files are cleared
Collection<StoreFile> compactedFilesPostClean = fileManager.getCompactedfiles();
Collection<HStoreFile> compactedFilesPostClean = fileManager.getCompactedfiles();
assertEquals(1, compactedFilesPostClean.size());
for (StoreFile origFile : compactedFilesPreClean) {
for (HStoreFile origFile : compactedFilesPreClean) {
assertFalse(compactedFilesPostClean.contains(origFile));
}

View File

@ -18,8 +18,6 @@
package org.apache.hadoop.hbase.regionserver;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
@ -46,6 +44,8 @@ import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.apache.hadoop.hbase.shaded.com.google.common.collect.Lists;
public class TestCompactionPolicy {
private final static Log LOG = LogFactory.getLog(TestCompactionPolicy.class);
protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@ -141,7 +141,7 @@ public class TestCompactionPolicy {
return result;
}
List<StoreFile> sfCreate(long... sizes) throws IOException {
List<HStoreFile> sfCreate(long... sizes) throws IOException {
ArrayList<Long> ageInDisk = new ArrayList<>();
for (int i = 0; i < sizes.length; i++) {
ageInDisk.add(0L);
@ -149,11 +149,11 @@ public class TestCompactionPolicy {
return sfCreate(toArrayList(sizes), ageInDisk);
}
List<StoreFile> sfCreate(ArrayList<Long> sizes, ArrayList<Long> ageInDisk) throws IOException {
List<HStoreFile> sfCreate(ArrayList<Long> sizes, ArrayList<Long> ageInDisk) throws IOException {
return sfCreate(false, sizes, ageInDisk);
}
List<StoreFile> sfCreate(boolean isReference, long... sizes) throws IOException {
List<HStoreFile> sfCreate(boolean isReference, long... sizes) throws IOException {
ArrayList<Long> ageInDisk = new ArrayList<>(sizes.length);
for (int i = 0; i < sizes.length; i++) {
ageInDisk.add(0L);
@ -161,17 +161,17 @@ public class TestCompactionPolicy {
return sfCreate(isReference, toArrayList(sizes), ageInDisk);
}
List<StoreFile> sfCreate(boolean isReference, ArrayList<Long> sizes, ArrayList<Long> ageInDisk)
List<HStoreFile> sfCreate(boolean isReference, ArrayList<Long> sizes, ArrayList<Long> ageInDisk)
throws IOException {
List<StoreFile> ret = Lists.newArrayList();
List<HStoreFile> ret = Lists.newArrayList();
for (int i = 0; i < sizes.size(); i++) {
ret.add(new MockStoreFile(TEST_UTIL, TEST_FILE, sizes.get(i), ageInDisk.get(i), isReference,
ret.add(new MockHStoreFile(TEST_UTIL, TEST_FILE, sizes.get(i), ageInDisk.get(i), isReference,
i));
}
return ret;
}
long[] getSizes(List<StoreFile> sfList) {
long[] getSizes(List<HStoreFile> sfList) {
long[] aNums = new long[sfList.size()];
for (int i = 0; i < sfList.size(); ++i) {
aNums[i] = sfList.get(i).getReader().length();
@ -179,23 +179,23 @@ public class TestCompactionPolicy {
return aNums;
}
void compactEquals(List<StoreFile> candidates, long... expected) throws IOException {
void compactEquals(List<HStoreFile> candidates, long... expected) throws IOException {
compactEquals(candidates, false, false, expected);
}
void compactEquals(List<StoreFile> candidates, boolean forcemajor, long... expected)
void compactEquals(List<HStoreFile> candidates, boolean forcemajor, long... expected)
throws IOException {
compactEquals(candidates, forcemajor, false, expected);
}
void compactEquals(List<StoreFile> candidates, boolean forcemajor, boolean isOffPeak,
void compactEquals(List<HStoreFile> candidates, boolean forcemajor, boolean isOffPeak,
long... expected) throws IOException {
store.forceMajor = forcemajor;
// Test Default compactions
CompactionRequest result =
((RatioBasedCompactionPolicy) store.storeEngine.getCompactionPolicy()).selectCompaction(
candidates, new ArrayList<>(), false, isOffPeak, forcemajor);
List<StoreFile> actual = new ArrayList<>(result.getFiles());
List<HStoreFile> actual = new ArrayList<>(result.getFiles());
if (isOffPeak && !forcemajor) {
Assert.assertTrue(result.isOffPeak());
}

View File

@ -200,7 +200,7 @@ public class TestCompoundBloomFilter {
private void readStoreFile(int t, BloomType bt, List<KeyValue> kvs,
Path sfPath) throws IOException {
StoreFile sf = new HStoreFile(fs, sfPath, conf, cacheConf, bt, true);
HStoreFile sf = new HStoreFile(fs, sfPath, conf, cacheConf, bt, true);
sf.initReader();
StoreFileReader r = sf.getReader();
final boolean pread = true; // does not really matter

Some files were not shown because too many files have changed in this diff Show More