HBASE-18825 Use HStoreFile instead of StoreFile in our own code base and remove unnecessary methods in StoreFile interface
This commit is contained in:
parent
1540483135
commit
a5f84430a3
|
@ -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));
|
||||
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 + ")",
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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>
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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()));
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
};
|
|
@ -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()));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue