HBASE-18038 Rename StoreFile to HStoreFile and add a StoreFile interface for CP
This commit is contained in:
parent
8bfa8aaaca
commit
ee0f148c73
|
@ -56,7 +56,7 @@ import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
|
|||
import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.HStoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
|
||||
|
@ -605,11 +605,8 @@ public class MergeTableRegionsProcedure
|
|||
final CacheConfig cacheConf = new CacheConfig(conf, hcd);
|
||||
for (StoreFileInfo storeFileInfo: storeFiles) {
|
||||
// Create reference file(s) of the region in mergedDir
|
||||
regionFs.mergeStoreFile(
|
||||
mergedRegion,
|
||||
family,
|
||||
new StoreFile(
|
||||
mfs.getFileSystem(), storeFileInfo, conf, cacheConf, hcd.getBloomFilterType()),
|
||||
regionFs.mergeStoreFile(mergedRegion, family, new HStoreFile(mfs.getFileSystem(),
|
||||
storeFileInfo, conf, cacheConf, hcd.getBloomFilterType(), true),
|
||||
mergedDir);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,6 +18,8 @@
|
|||
|
||||
package org.apache.hadoop.hbase.master.assignment;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.InterruptedIOException;
|
||||
|
@ -62,6 +64,7 @@ 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;
|
||||
|
@ -71,8 +74,6 @@ import org.apache.hadoop.hbase.util.FSUtils;
|
|||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* The procedure to split a region in a table.
|
||||
* Takes lock on the parent region.
|
||||
|
@ -525,11 +526,9 @@ public class SplitTableRegionProcedure
|
|||
if (storeFiles != null && storeFiles.size() > 0) {
|
||||
final CacheConfig cacheConf = new CacheConfig(conf, hcd);
|
||||
for (StoreFileInfo storeFileInfo: storeFiles) {
|
||||
StoreFileSplitter sfs = new StoreFileSplitter(
|
||||
regionFs,
|
||||
family.getBytes(),
|
||||
new StoreFile(
|
||||
mfs.getFileSystem(), storeFileInfo, conf, cacheConf, hcd.getBloomFilterType()));
|
||||
StoreFileSplitter sfs =
|
||||
new StoreFileSplitter(regionFs, family.getBytes(), new HStoreFile(mfs.getFileSystem(),
|
||||
storeFileInfo, conf, cacheConf, hcd.getBloomFilterType(), true));
|
||||
futures.add(threadPool.submit(sfs));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,7 +28,6 @@ import java.util.Map;
|
|||
import java.util.Map.Entry;
|
||||
import java.util.Random;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -51,6 +50,8 @@ import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer.Cluster.SwapRegi
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* <p>This is a best effort load balancer. Given a Cost function F(C) => x It will
|
||||
* randomly try and mutate the cluster to Cprime. If F(Cprime) < F(C) then the
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hbase.classification.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;
|
||||
|
||||
/**
|
||||
|
@ -46,7 +47,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 StoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
|
||||
StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
|
||||
return new CachedMobFile(sf);
|
||||
}
|
||||
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.classification.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;
|
||||
|
||||
|
@ -146,7 +147,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 StoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
|
||||
StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
|
||||
return new MobFile(sf);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -69,6 +69,7 @@ import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.C
|
|||
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;
|
||||
|
@ -334,7 +335,7 @@ public final class MobUtils {
|
|||
LOG.debug(fileName + " is an expired file");
|
||||
}
|
||||
filesToClean
|
||||
.add(new StoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true));
|
||||
.add(new HStoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("Cannot parse the fileName " + fileName, e);
|
||||
|
@ -722,7 +723,7 @@ public final class MobUtils {
|
|||
CacheConfig cacheConfig, boolean primaryReplica) throws IOException {
|
||||
StoreFile storeFile = null;
|
||||
try {
|
||||
storeFile = new StoreFile(fs, path, conf, cacheConfig, BloomType.NONE, primaryReplica);
|
||||
storeFile = new HStoreFile(fs, path, conf, cacheConfig, BloomType.NONE, primaryReplica);
|
||||
storeFile.initReader();
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to open mob file[" + path + "], keep it in temp directory.", e);
|
||||
|
|
|
@ -36,7 +36,6 @@ import java.util.concurrent.Callable;
|
|||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Future;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -74,6 +73,7 @@ import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.C
|
|||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartitionId;
|
||||
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.ScanInfo;
|
||||
import org.apache.hadoop.hbase.regionserver.ScanType;
|
||||
import org.apache.hadoop.hbase.regionserver.ScannerContext;
|
||||
|
@ -87,6 +87,8 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* An implementation of {@link MobCompactor} that compacts the mob files in partitions.
|
||||
*/
|
||||
|
@ -335,7 +337,7 @@ public class PartitionedMobCompactor extends MobCompactor {
|
|||
for (CompactionDelPartition delPartition : request.getDelPartitions()) {
|
||||
for (Path newDelPath : delPartition.listDelFiles()) {
|
||||
StoreFile sf =
|
||||
new StoreFile(fs, newDelPath, conf, compactionCacheConfig, BloomType.NONE, true);
|
||||
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.
|
||||
sf.initReader();
|
||||
|
@ -551,7 +553,7 @@ public class PartitionedMobCompactor extends MobCompactor {
|
|||
// add the selected mob files and del files into filesToCompact
|
||||
List<StoreFile> filesToCompact = new ArrayList<>();
|
||||
for (int i = offset; i < batch + offset; i++) {
|
||||
StoreFile sf = new StoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig,
|
||||
StoreFile sf = new HStoreFile(fs, files.get(i).getPath(), conf, compactionCacheConfig,
|
||||
BloomType.NONE, true);
|
||||
filesToCompact.add(sf);
|
||||
}
|
||||
|
@ -733,7 +735,7 @@ public class PartitionedMobCompactor extends MobCompactor {
|
|||
continue;
|
||||
}
|
||||
for (int i = offset; i < batch + offset; i++) {
|
||||
batchedDelFiles.add(new StoreFile(fs, delFilePaths.get(i), conf, compactionCacheConfig,
|
||||
batchedDelFiles.add(new HStoreFile(fs, delFilePaths.get(i), conf, compactionCacheConfig,
|
||||
BloomType.NONE, true));
|
||||
}
|
||||
// compact the del files in a batch.
|
||||
|
|
|
@ -27,19 +27,29 @@ import java.util.Set;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
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.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.mapreduce.JobUtil;
|
||||
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
||||
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
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.io.LongWritable;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.util.LineReader;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
import org.apache.hadoop.mapreduce.InputSplit;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.apache.hadoop.mapreduce.JobContext;
|
||||
|
@ -47,21 +57,9 @@ import org.apache.hadoop.mapreduce.Mapper;
|
|||
import org.apache.hadoop.mapreduce.lib.input.FileSplit;
|
||||
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
|
||||
import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
||||
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
|
||||
import org.apache.hadoop.hbase.mapreduce.JobUtil;
|
||||
import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
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.util.LineReader;
|
||||
import org.apache.hadoop.util.Tool;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
|
||||
/*
|
||||
* The CompactionTool allows to execute a compaction specifying a:
|
||||
|
|
|
@ -58,7 +58,7 @@ public class DateTieredStoreEngine extends StoreEngine<DefaultStoreFlusher,
|
|||
throws IOException {
|
||||
this.compactionPolicy = new DateTieredCompactionPolicy(conf, store);
|
||||
this.storeFileManager =
|
||||
new DefaultStoreFileManager(kvComparator, StoreFile.Comparators.SEQ_ID_MAX_TIMESTAMP, conf,
|
||||
new DefaultStoreFileManager(kvComparator, StoreFileComparators.SEQ_ID_MAX_TIMESTAMP, conf,
|
||||
compactionPolicy.getConf());
|
||||
this.storeFlusher = new DefaultStoreFlusher(conf, store);
|
||||
this.compactor = new DateTieredCompactor(conf, store);
|
||||
|
|
|
@ -69,7 +69,7 @@ public class DefaultStoreEngine extends StoreEngine<
|
|||
createCompactionPolicy(conf, store);
|
||||
createStoreFlusher(conf, store);
|
||||
storeFileManager =
|
||||
new DefaultStoreFileManager(kvComparator, StoreFile.Comparators.SEQ_ID, conf,
|
||||
new DefaultStoreFileManager(kvComparator, StoreFileComparators.SEQ_ID, conf,
|
||||
compactionPolicy.getConf());
|
||||
}
|
||||
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.Collections;
|
|||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
|
||||
import com.google.common.collect.ImmutableCollection;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
|
@ -172,10 +173,13 @@ class DefaultStoreFileManager implements StoreFileManager {
|
|||
|
||||
@Override
|
||||
public final byte[] getSplitPoint() throws IOException {
|
||||
if (this.storefiles.isEmpty()) {
|
||||
List<StoreFile> storefiles = this.storefiles;
|
||||
if (storefiles.isEmpty()) {
|
||||
return null;
|
||||
}
|
||||
return StoreUtils.getLargestFile(this.storefiles).getFileSplitPoint(this.kvComparator);
|
||||
Optional<StoreFile> largestFile = StoreUtils.getLargestFile(storefiles);
|
||||
return largestFile.isPresent()
|
||||
? StoreUtils.getFileSplitPoint(largestFile.get(), kvComparator).orElse(null) : null;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -293,7 +293,7 @@ public class HMobStore extends HStore {
|
|||
private void validateMobFile(Path path) throws IOException {
|
||||
StoreFile storeFile = null;
|
||||
try {
|
||||
storeFile = new StoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig,
|
||||
storeFile = new HStoreFile(region.getFilesystem(), path, conf, this.mobCacheConfig,
|
||||
BloomType.NONE, isPrimaryReplicaStore());
|
||||
storeFile.initReader();
|
||||
} catch (IOException e) {
|
||||
|
|
|
@ -1456,7 +1456,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* time-sensitive thread.
|
||||
*
|
||||
* @return Vector of all the storage files that the HRegion's component
|
||||
* HStores make use of. It's a list of all HStoreFile objects. Returns empty
|
||||
* HStores make use of. It's a list of all StoreFile objects. Returns empty
|
||||
* vector if already closed and null if judged that it should not close.
|
||||
*
|
||||
* @throws IOException e
|
||||
|
@ -1497,7 +1497,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
*
|
||||
* @param abort true if server is aborting (only during testing)
|
||||
* @return Vector of all the storage files that the HRegion's component
|
||||
* HStores make use of. It's a list of HStoreFile objects. Can be null if
|
||||
* HStores make use of. It's a list of StoreFile objects. Can be null if
|
||||
* we are not to close at this time or we are already closed.
|
||||
*
|
||||
* @throws IOException e
|
||||
|
@ -4204,7 +4204,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
Set<StoreFile> fakeStoreFiles = new HashSet<>(files.size());
|
||||
for (Path file: files) {
|
||||
fakeStoreFiles.add(
|
||||
new StoreFile(getRegionFileSystem().getFileSystem(), file, this.conf, null, null, true));
|
||||
new HStoreFile(getRegionFileSystem().getFileSystem(), file, this.conf, null, null, true));
|
||||
}
|
||||
getRegionFileSystem().removeStoreFiles(fakeFamilyName, fakeStoreFiles);
|
||||
} else {
|
||||
|
|
|
@ -18,13 +18,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableCollection;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.net.InetSocketAddress;
|
||||
|
@ -53,7 +46,15 @@ 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.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CompoundConfiguration;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.MemoryCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.backup.FailedArchiveException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
|
@ -90,6 +91,13 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
|
|||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableCollection;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
/**
|
||||
* A Store holds a column family in a Region. Its a memstore and a set of zero
|
||||
* or more StoreFiles, which stretch backwards over time.
|
||||
|
@ -455,12 +463,12 @@ public class HStore implements Store {
|
|||
*/
|
||||
@Override
|
||||
public long getMaxSequenceId() {
|
||||
return StoreFile.getMaxSequenceIdInList(this.getStorefiles());
|
||||
return StoreUtils.getMaxSequenceIdInList(this.getStorefiles());
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMaxMemstoreTS() {
|
||||
return StoreFile.getMaxMemstoreTSInList(this.getStorefiles());
|
||||
return StoreUtils.getMaxMemstoreTSInList(this.getStorefiles());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -655,7 +663,7 @@ public class HStore implements Store {
|
|||
|
||||
private StoreFile createStoreFileAndReader(final StoreFileInfo info) throws IOException {
|
||||
info.setRegionCoprocessorHost(this.region.getCoprocessorHost());
|
||||
StoreFile storeFile = new StoreFile(this.getFileSystem(), info, this.conf, this.cacheConf,
|
||||
StoreFile storeFile = new HStoreFile(this.getFileSystem(), info, this.conf, this.cacheConf,
|
||||
this.family.getBloomFilterType(), isPrimaryReplicaStore());
|
||||
storeFile.initReader();
|
||||
return storeFile;
|
||||
|
|
|
@ -0,0 +1,560 @@
|
|||
/**
|
||||
*
|
||||
* 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
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* 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.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Map;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
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.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.classification.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 com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* A Store data file. Stores usually have one or more of these files. They
|
||||
* are produced by flushing the memstore to disk. To
|
||||
* create, instantiate a writer using {@link StoreFileWriter.Builder}
|
||||
* and append data. Be sure to add any metadata before calling close on the
|
||||
* Writer (Use the appendMetadata convenience methods). On close, a StoreFile
|
||||
* is sitting in the Filesystem. To refer to it, create a StoreFile instance
|
||||
* passing filesystem and path. To read, call {@link #initReader()}
|
||||
* <p>StoreFiles may also reference store files in another Store.
|
||||
*
|
||||
* The reason for this weird pattern where you use a different instance for the
|
||||
* writer and a reader is that we write once but read a lot more.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class HStoreFile implements StoreFile {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(HStoreFile.class.getName());
|
||||
|
||||
private static final boolean DEFAULT_STORE_FILE_READER_NO_READAHEAD = false;
|
||||
|
||||
private final StoreFileInfo fileInfo;
|
||||
private final FileSystem fs;
|
||||
|
||||
// Block cache configuration and reference.
|
||||
private final CacheConfig cacheConf;
|
||||
|
||||
// Counter that is incremented every time a scanner is created on the
|
||||
// store file. It is decremented when the scan on the store file is
|
||||
// done.
|
||||
private final AtomicInteger refCount = new AtomicInteger(0);
|
||||
|
||||
private final boolean noReadahead;
|
||||
|
||||
private final boolean primaryReplica;
|
||||
|
||||
// Indicates if the file got compacted
|
||||
private volatile boolean compactedAway = false;
|
||||
|
||||
// Keys for metadata stored in backing HFile.
|
||||
// Set when we obtain a Reader.
|
||||
private long sequenceid = -1;
|
||||
|
||||
// max of the MemstoreTS in the KV's in this store
|
||||
// Set when we obtain a Reader.
|
||||
private long maxMemstoreTS = -1;
|
||||
|
||||
// firstKey, lastkey and cellComparator will be set when openReader.
|
||||
private Cell firstKey;
|
||||
|
||||
private Cell lastKey;
|
||||
|
||||
private Comparator<Cell> comparator;
|
||||
|
||||
@Override
|
||||
public CacheConfig getCacheConf() {
|
||||
return cacheConf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Cell getFirstKey() {
|
||||
return firstKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Cell getLastKey() {
|
||||
return lastKey;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<Cell> getComparator() {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMaxMemstoreTS() {
|
||||
return maxMemstoreTS;
|
||||
}
|
||||
|
||||
// If true, this file was product of a major compaction. Its then set
|
||||
// whenever you get a Reader.
|
||||
private AtomicBoolean majorCompaction = null;
|
||||
|
||||
// If true, this file should not be included in minor compactions.
|
||||
// It's set whenever you get a Reader.
|
||||
private boolean excludeFromMinorCompaction = false;
|
||||
|
||||
/**
|
||||
* Map of the metadata entries in the corresponding HFile. Populated when Reader is opened
|
||||
* after which it is not modified again.
|
||||
*/
|
||||
private Map<byte[], byte[]> metadataMap;
|
||||
|
||||
// StoreFile.Reader
|
||||
private volatile StoreFileReader reader;
|
||||
|
||||
/**
|
||||
* Bloom filter type specified in column family configuration. Does not
|
||||
* necessarily correspond to the Bloom filter type present in the HFile.
|
||||
*/
|
||||
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?).
|
||||
* @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.
|
||||
* @param primaryReplica true if this is a store file for primary replica, otherwise false.
|
||||
* @throws IOException
|
||||
*/
|
||||
public HStoreFile(FileSystem fs, Path p, Configuration conf, CacheConfig cacheConf,
|
||||
BloomType cfBloomType, boolean primaryReplica) throws IOException {
|
||||
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?).
|
||||
* @param fs 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.
|
||||
* @param primaryReplica true if this is a store file for primary replica, otherwise false.
|
||||
*/
|
||||
public HStoreFile(FileSystem fs, StoreFileInfo fileInfo, Configuration conf, CacheConfig cacheConf,
|
||||
BloomType cfBloomType, boolean primaryReplica) {
|
||||
this.fs = fs;
|
||||
this.fileInfo = fileInfo;
|
||||
this.cacheConf = cacheConf;
|
||||
this.noReadahead =
|
||||
conf.getBoolean(STORE_FILE_READER_NO_READAHEAD, DEFAULT_STORE_FILE_READER_NO_READAHEAD);
|
||||
if (BloomFilterFactory.isGeneralBloomEnabled(conf)) {
|
||||
this.cfBloomType = cfBloomType;
|
||||
} else {
|
||||
LOG.info("Ignoring bloom filter check for file " + this.getPath() + ": " + "cfBloomType=" +
|
||||
cfBloomType + " (disabled in config)");
|
||||
this.cfBloomType = BloomType.NONE;
|
||||
}
|
||||
this.primaryReplica = primaryReplica;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StoreFileInfo getFileInfo() {
|
||||
return this.fileInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getPath() {
|
||||
return this.fileInfo.getPath();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getQualifiedPath() {
|
||||
return this.fileInfo.getPath().makeQualified(fs.getUri(), fs.getWorkingDirectory());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isReference() {
|
||||
return this.fileInfo.isReference();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isHFile() {
|
||||
return StoreFileInfo.isHFile(this.fileInfo.getPath());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isMajorCompactionResult() {
|
||||
if (this.majorCompaction == null) {
|
||||
throw new NullPointerException("This has not been set yet");
|
||||
}
|
||||
return this.majorCompaction.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean excludeFromMinorCompaction() {
|
||||
return this.excludeFromMinorCompaction;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMaxSequenceId() {
|
||||
return this.sequenceid;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getModificationTimeStamp() throws IOException {
|
||||
return fileInfo.getModificationTime();
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getMetadataValue(byte[] key) {
|
||||
return metadataMap.get(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isBulkLoadResult() {
|
||||
boolean bulkLoadedHFile = false;
|
||||
String fileName = this.getPath().getName();
|
||||
int startPos = fileName.indexOf("SeqId_");
|
||||
if (startPos != -1) {
|
||||
bulkLoadedHFile = true;
|
||||
}
|
||||
return bulkLoadedHFile || (metadataMap != null && metadataMap.containsKey(BULKLOAD_TIME_KEY));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isCompactedAway() {
|
||||
return compactedAway;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public int getRefCount() {
|
||||
return refCount.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isReferencedInReads() {
|
||||
int rc = refCount.get();
|
||||
assert rc >= 0; // we should not go negative.
|
||||
return rc > 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OptionalLong getBulkLoadTimestamp() {
|
||||
byte[] bulkLoadTimestamp = metadataMap.get(BULKLOAD_TIME_KEY);
|
||||
return bulkLoadTimestamp == null ? OptionalLong.empty()
|
||||
: OptionalLong.of(Bytes.toLong(bulkLoadTimestamp));
|
||||
}
|
||||
|
||||
@Override
|
||||
public HDFSBlocksDistribution getHDFSBlockDistribution() {
|
||||
return this.fileInfo.getHDFSBlockDistribution();
|
||||
}
|
||||
|
||||
/**
|
||||
* Opens reader on this store file. Called by Constructor.
|
||||
* @throws IOException
|
||||
* @see #closeReader(boolean)
|
||||
*/
|
||||
private void open() throws IOException {
|
||||
if (this.reader != null) {
|
||||
throw new IllegalAccessError("Already open");
|
||||
}
|
||||
|
||||
// Open the StoreFile.Reader
|
||||
this.reader = fileInfo.open(this.fs, this.cacheConf, false, noReadahead ? 0L : -1L,
|
||||
primaryReplica, refCount, true);
|
||||
|
||||
// Load up indices and fileinfo. This also loads Bloom filter type.
|
||||
metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
|
||||
|
||||
// Read in our metadata.
|
||||
byte [] b = metadataMap.get(MAX_SEQ_ID_KEY);
|
||||
if (b != null) {
|
||||
// By convention, if halfhfile, top half has a sequence number > bottom
|
||||
// half. Thats why we add one in below. Its done for case the two halves
|
||||
// are ever merged back together --rare. Without it, on open of store,
|
||||
// since store files are distinguished by sequence id, the one half would
|
||||
// subsume the other.
|
||||
this.sequenceid = Bytes.toLong(b);
|
||||
if (fileInfo.isTopReference()) {
|
||||
this.sequenceid += 1;
|
||||
}
|
||||
}
|
||||
|
||||
if (isBulkLoadResult()){
|
||||
// generate the sequenceId from the fileName
|
||||
// fileName is of the form <randomName>_SeqId_<id-when-loaded>_
|
||||
String fileName = this.getPath().getName();
|
||||
// Use lastIndexOf() to get the last, most recent bulk load seqId.
|
||||
int startPos = fileName.lastIndexOf("SeqId_");
|
||||
if (startPos != -1) {
|
||||
this.sequenceid = Long.parseLong(fileName.substring(startPos + 6,
|
||||
fileName.indexOf('_', startPos + 6)));
|
||||
// Handle reference files as done above.
|
||||
if (fileInfo.isTopReference()) {
|
||||
this.sequenceid += 1;
|
||||
}
|
||||
}
|
||||
// SKIP_RESET_SEQ_ID only works in bulk loaded file.
|
||||
// In mob compaction, the hfile where the cells contain the path of a new mob file is bulk
|
||||
// loaded to hbase, these cells have the same seqIds with the old ones. We do not want
|
||||
// to reset new seqIds for them since this might make a mess of the visibility of cells that
|
||||
// have the same row key but different seqIds.
|
||||
boolean skipResetSeqId = isSkipResetSeqId(metadataMap.get(SKIP_RESET_SEQ_ID));
|
||||
if (skipResetSeqId) {
|
||||
// increase the seqId when it is a bulk loaded file from mob compaction.
|
||||
this.sequenceid += 1;
|
||||
}
|
||||
this.reader.setSkipResetSeqId(skipResetSeqId);
|
||||
this.reader.setBulkLoaded(true);
|
||||
}
|
||||
this.reader.setSequenceID(this.sequenceid);
|
||||
|
||||
b = metadataMap.get(HFile.Writer.MAX_MEMSTORE_TS_KEY);
|
||||
if (b != null) {
|
||||
this.maxMemstoreTS = Bytes.toLong(b);
|
||||
}
|
||||
|
||||
b = metadataMap.get(MAJOR_COMPACTION_KEY);
|
||||
if (b != null) {
|
||||
boolean mc = Bytes.toBoolean(b);
|
||||
if (this.majorCompaction == null) {
|
||||
this.majorCompaction = new AtomicBoolean(mc);
|
||||
} else {
|
||||
this.majorCompaction.set(mc);
|
||||
}
|
||||
} else {
|
||||
// Presume it is not major compacted if it doesn't explicity say so
|
||||
// HFileOutputFormat explicitly sets the major compacted key.
|
||||
this.majorCompaction = new AtomicBoolean(false);
|
||||
}
|
||||
|
||||
b = metadataMap.get(EXCLUDE_FROM_MINOR_COMPACTION_KEY);
|
||||
this.excludeFromMinorCompaction = (b != null && Bytes.toBoolean(b));
|
||||
|
||||
BloomType hfileBloomType = reader.getBloomFilterType();
|
||||
if (cfBloomType != BloomType.NONE) {
|
||||
reader.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
|
||||
if (hfileBloomType != cfBloomType) {
|
||||
LOG.info("HFile Bloom filter type for "
|
||||
+ reader.getHFileReader().getName() + ": " + hfileBloomType
|
||||
+ ", but " + cfBloomType + " specified in column family "
|
||||
+ "configuration");
|
||||
}
|
||||
} else if (hfileBloomType != BloomType.NONE) {
|
||||
LOG.info("Bloom filter turned off by CF config for "
|
||||
+ reader.getHFileReader().getName());
|
||||
}
|
||||
|
||||
// load delete family bloom filter
|
||||
reader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
|
||||
|
||||
try {
|
||||
this.reader.timeRange = TimeRangeTracker.getTimeRange(metadataMap.get(TIMERANGE_KEY));
|
||||
} catch (IllegalArgumentException e) {
|
||||
LOG.error("Error reading timestamp range data from meta -- " +
|
||||
"proceeding without", e);
|
||||
this.reader.timeRange = null;
|
||||
}
|
||||
// initialize so we can reuse them after reader closed.
|
||||
firstKey = reader.getFirstKey();
|
||||
lastKey = reader.getLastKey();
|
||||
comparator = reader.getComparator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initReader() throws IOException {
|
||||
if (reader == null) {
|
||||
try {
|
||||
open();
|
||||
} catch (Exception e) {
|
||||
try {
|
||||
boolean evictOnClose = cacheConf != null ? cacheConf.shouldEvictOnClose() : true;
|
||||
this.closeReader(evictOnClose);
|
||||
} catch (IOException ee) {
|
||||
LOG.warn("failed to close reader", ee);
|
||||
}
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private StoreFileReader createStreamReader(boolean canUseDropBehind) throws IOException {
|
||||
initReader();
|
||||
StoreFileReader reader = fileInfo.open(this.fs, this.cacheConf, canUseDropBehind, -1L,
|
||||
primaryReplica, refCount, false);
|
||||
reader.copyFields(this.reader);
|
||||
return reader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public StoreFileScanner getPreadScanner(boolean cacheBlocks, long readPt, long scannerOrder,
|
||||
boolean canOptimizeForNonNullColumn) {
|
||||
return getReader().getStoreFileScanner(cacheBlocks, true, false, readPt, scannerOrder,
|
||||
canOptimizeForNonNullColumn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks,
|
||||
boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn)
|
||||
throws IOException {
|
||||
return createStreamReader(canUseDropBehind).getStoreFileScanner(cacheBlocks, false,
|
||||
isCompaction, readPt, scannerOrder, canOptimizeForNonNullColumn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public StoreFileReader getReader() {
|
||||
return this.reader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void closeReader(boolean evictOnClose)
|
||||
throws IOException {
|
||||
if (this.reader != null) {
|
||||
this.reader.close(evictOnClose);
|
||||
this.reader = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void markCompactedAway() {
|
||||
this.compactedAway = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deleteReader() throws IOException {
|
||||
boolean evictOnClose =
|
||||
cacheConf != null? cacheConf.shouldEvictOnClose(): true;
|
||||
closeReader(evictOnClose);
|
||||
this.fs.delete(getPath(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return this.fileInfo.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toStringDetailed() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append(this.getPath().toString());
|
||||
sb.append(", isReference=").append(isReference());
|
||||
sb.append(", isBulkLoadResult=").append(isBulkLoadResult());
|
||||
if (isBulkLoadResult()) {
|
||||
sb.append(", bulkLoadTS=");
|
||||
OptionalLong bulkLoadTS = getBulkLoadTimestamp();
|
||||
if (bulkLoadTS.isPresent()) {
|
||||
sb.append(bulkLoadTS.getAsLong());
|
||||
} else {
|
||||
sb.append("NotPresent");
|
||||
}
|
||||
} else {
|
||||
sb.append(", seqid=").append(getMaxSequenceId());
|
||||
}
|
||||
sb.append(", majorCompaction=").append(isMajorCompactionResult());
|
||||
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets whether to skip resetting the sequence id for cells.
|
||||
* @param skipResetSeqId The byte array of boolean.
|
||||
* @return Whether to skip resetting the sequence id.
|
||||
*/
|
||||
private boolean isSkipResetSeqId(byte[] skipResetSeqId) {
|
||||
if (skipResetSeqId != null && skipResetSeqId.length == 1) {
|
||||
return Bytes.toBoolean(skipResetSeqId);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OptionalLong getMinimumTimestamp() {
|
||||
TimeRange tr = getReader().timeRange;
|
||||
return tr != null ? OptionalLong.of(tr.getMin()) : OptionalLong.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public OptionalLong getMaximumTimestamp() {
|
||||
TimeRange tr = getReader().timeRange;
|
||||
return tr != null ? OptionalLong.of(tr.getMax()) : OptionalLong.empty();
|
||||
}
|
||||
}
|
|
@ -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,789 +17,200 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Ordering;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.OptionalLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
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.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockType;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
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;
|
||||
|
||||
/**
|
||||
* A Store data file. Stores usually have one or more of these files. They
|
||||
* are produced by flushing the memstore to disk. To
|
||||
* create, instantiate a writer using {@link StoreFileWriter.Builder}
|
||||
* and append data. Be sure to add any metadata before calling close on the
|
||||
* Writer (Use the appendMetadata convenience methods). On close, a StoreFile
|
||||
* is sitting in the Filesystem. To refer to it, create a StoreFile instance
|
||||
* passing filesystem and path. To read, call {@link #initReader()}
|
||||
* <p>StoreFiles may also reference store files in another Store.
|
||||
*
|
||||
* The reason for this weird pattern where you use a different instance for the
|
||||
* writer and a reader is that we write once but read a lot more.
|
||||
* An interface to describe a store data file.
|
||||
*/
|
||||
@InterfaceAudience.LimitedPrivate("Coprocessor")
|
||||
public class StoreFile {
|
||||
private static final Log LOG = LogFactory.getLog(StoreFile.class.getName());
|
||||
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
|
||||
@InterfaceStability.Evolving
|
||||
public interface StoreFile {
|
||||
|
||||
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;
|
||||
static final String STORE_FILE_READER_NO_READAHEAD = "hbase.store.reader.no-readahead";
|
||||
|
||||
// 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");
|
||||
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");
|
||||
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 =
|
||||
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");
|
||||
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");
|
||||
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");
|
||||
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 Timerange information in metadata */
|
||||
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 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*/
|
||||
public static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT");
|
||||
|
||||
private final StoreFileInfo fileInfo;
|
||||
private final FileSystem fs;
|
||||
|
||||
// Block cache configuration and reference.
|
||||
private final CacheConfig cacheConf;
|
||||
|
||||
// Counter that is incremented every time a scanner is created on the
|
||||
// store file. It is decremented when the scan on the store file is
|
||||
// done.
|
||||
private final AtomicInteger refCount = new AtomicInteger(0);
|
||||
|
||||
private final boolean noReadahead;
|
||||
|
||||
private final boolean primaryReplica;
|
||||
|
||||
// Indicates if the file got compacted
|
||||
private volatile boolean compactedAway = false;
|
||||
|
||||
// Keys for metadata stored in backing HFile.
|
||||
// Set when we obtain a Reader.
|
||||
private long sequenceid = -1;
|
||||
|
||||
// max of the MemstoreTS in the KV's in this store
|
||||
// Set when we obtain a Reader.
|
||||
private long maxMemstoreTS = -1;
|
||||
|
||||
// firstKey, lastkey and cellComparator will be set when openReader.
|
||||
private Cell firstKey;
|
||||
|
||||
private Cell lastKey;
|
||||
|
||||
private Comparator<Cell> comparator;
|
||||
|
||||
CacheConfig getCacheConf() {
|
||||
return cacheConf;
|
||||
}
|
||||
|
||||
public Cell getFirstKey() {
|
||||
return firstKey;
|
||||
}
|
||||
|
||||
public Cell getLastKey() {
|
||||
return lastKey;
|
||||
}
|
||||
|
||||
public Comparator<Cell> getComparator() {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
public long getMaxMemstoreTS() {
|
||||
return maxMemstoreTS;
|
||||
}
|
||||
|
||||
public void setMaxMemstoreTS(long maxMemstoreTS) {
|
||||
this.maxMemstoreTS = maxMemstoreTS;
|
||||
}
|
||||
|
||||
// If true, this file was product of a major compaction. Its then set
|
||||
// whenever you get a Reader.
|
||||
private AtomicBoolean majorCompaction = null;
|
||||
|
||||
// If true, this file should not be included in minor compactions.
|
||||
// It's set whenever you get a Reader.
|
||||
private boolean excludeFromMinorCompaction = false;
|
||||
/** 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 */
|
||||
public static final byte[] BULKLOAD_TASK_KEY =
|
||||
Bytes.toBytes("BULKLOAD_SOURCE_TASK");
|
||||
public static final byte[] BULKLOAD_TIME_KEY =
|
||||
Bytes.toBytes("BULKLOAD_TIMESTAMP");
|
||||
static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK");
|
||||
static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP");
|
||||
|
||||
/**
|
||||
* Map of the metadata entries in the corresponding HFile. Populated when Reader is opened
|
||||
* after which it is not modified again.
|
||||
* 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.
|
||||
*/
|
||||
private Map<byte[], byte[]> metadataMap;
|
||||
static final byte[] SKIP_RESET_SEQ_ID = Bytes.toBytes("SKIP_RESET_SEQ_ID");
|
||||
|
||||
// StoreFile.Reader
|
||||
private volatile StoreFileReader reader;
|
||||
CacheConfig getCacheConf();
|
||||
|
||||
Cell getFirstKey();
|
||||
|
||||
Cell getLastKey();
|
||||
|
||||
Comparator<Cell> getComparator();
|
||||
|
||||
long getMaxMemstoreTS();
|
||||
|
||||
/**
|
||||
* Bloom filter type specified in column family configuration. Does not
|
||||
* necessarily correspond to the Bloom filter type present in the HFile.
|
||||
* @return the StoreFile object associated to this StoreFile. null if the StoreFile is not a
|
||||
* reference.
|
||||
*/
|
||||
private final BloomType cfBloomType;
|
||||
|
||||
/**
|
||||
* 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");
|
||||
|
||||
/**
|
||||
* 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 #StoreFile(FileSystem, Path, Configuration, CacheConfig, BloomType, boolean)}
|
||||
* directly.
|
||||
*/
|
||||
@Deprecated
|
||||
public StoreFile(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?).
|
||||
* @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.
|
||||
* @param primaryReplica true if this is a store file for primary replica, otherwise false.
|
||||
* @throws IOException
|
||||
*/
|
||||
public StoreFile(FileSystem fs, Path p, Configuration conf, CacheConfig cacheConf,
|
||||
BloomType cfBloomType, boolean primaryReplica) throws IOException {
|
||||
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 #StoreFile(FileSystem, StoreFileInfo, Configuration, CacheConfig, BloomType, boolean)}
|
||||
* directly.
|
||||
*/
|
||||
@Deprecated
|
||||
public StoreFile(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?).
|
||||
* @param fs 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 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.
|
||||
* @param primaryReplica true if this is a store file for primary replica, otherwise false.
|
||||
*/
|
||||
public StoreFile(FileSystem fs, StoreFileInfo fileInfo, Configuration conf, CacheConfig cacheConf,
|
||||
BloomType cfBloomType, boolean primaryReplica) {
|
||||
this.fs = fs;
|
||||
this.fileInfo = fileInfo;
|
||||
this.cacheConf = cacheConf;
|
||||
this.noReadahead =
|
||||
conf.getBoolean(STORE_FILE_READER_NO_READAHEAD, DEFAULT_STORE_FILE_READER_NO_READAHEAD);
|
||||
if (BloomFilterFactory.isGeneralBloomEnabled(conf)) {
|
||||
this.cfBloomType = cfBloomType;
|
||||
} else {
|
||||
LOG.info("Ignoring bloom filter check for file " + this.getPath() + ": " + "cfBloomType=" +
|
||||
cfBloomType + " (disabled in config)");
|
||||
this.cfBloomType = BloomType.NONE;
|
||||
}
|
||||
this.primaryReplica = primaryReplica;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the StoreFile object associated to this StoreFile.
|
||||
* null if the StoreFile is not a reference.
|
||||
*/
|
||||
public StoreFileInfo getFileInfo() {
|
||||
return this.fileInfo;
|
||||
}
|
||||
StoreFileInfo getFileInfo();
|
||||
|
||||
/**
|
||||
* @return Path or null if this StoreFile was made with a Stream.
|
||||
*/
|
||||
public Path getPath() {
|
||||
return this.fileInfo.getPath();
|
||||
}
|
||||
Path getPath();
|
||||
|
||||
/**
|
||||
* @return Returns the qualified path of this StoreFile
|
||||
*/
|
||||
public Path getQualifiedPath() {
|
||||
return this.fileInfo.getPath().makeQualified(fs.getUri(), fs.getWorkingDirectory());
|
||||
}
|
||||
Path getQualifiedPath();
|
||||
|
||||
/**
|
||||
* @return True if this is a StoreFile Reference; call
|
||||
* after {@link #open()} else may get wrong answer.
|
||||
* @return True if this is a StoreFile Reference.
|
||||
*/
|
||||
public boolean isReference() {
|
||||
return this.fileInfo.isReference();
|
||||
}
|
||||
boolean isReference();
|
||||
|
||||
/**
|
||||
* @return True if this is HFile.
|
||||
*/
|
||||
public boolean isHFile() {
|
||||
return StoreFileInfo.isHFile(this.fileInfo.getPath());
|
||||
}
|
||||
boolean isHFile();
|
||||
|
||||
/**
|
||||
* @return True if this file was made by a major compaction.
|
||||
*/
|
||||
public boolean isMajorCompaction() {
|
||||
if (this.majorCompaction == null) {
|
||||
throw new NullPointerException("This has not been set yet");
|
||||
}
|
||||
return this.majorCompaction.get();
|
||||
}
|
||||
boolean isMajorCompactionResult();
|
||||
|
||||
/**
|
||||
* @return True if this file should not be part of a minor compaction.
|
||||
*/
|
||||
public boolean excludeFromMinorCompaction() {
|
||||
return this.excludeFromMinorCompaction;
|
||||
}
|
||||
boolean excludeFromMinorCompaction();
|
||||
|
||||
/**
|
||||
* @return This files maximum edit sequence id.
|
||||
*/
|
||||
public long getMaxSequenceId() {
|
||||
return this.sequenceid;
|
||||
}
|
||||
long getMaxSequenceId();
|
||||
|
||||
public long getModificationTimeStamp() throws IOException {
|
||||
return (fileInfo == null) ? 0 : fileInfo.getModificationTime();
|
||||
}
|
||||
long getModificationTimeStamp() throws IOException;
|
||||
|
||||
/**
|
||||
* 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);
|
||||
}
|
||||
byte[] getMetadataValue(byte[] key);
|
||||
|
||||
/**
|
||||
* Return the largest memstoreTS found across all storefiles in
|
||||
* the given list. Store files that were created by a mapreduce
|
||||
* bulk load are ignored, as they do not correspond to any specific
|
||||
* put operation, and thus do not have a memstoreTS associated with them.
|
||||
* @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;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
public static long getMaxSequenceIdInList(Collection<StoreFile> sfs) {
|
||||
long max = 0;
|
||||
for (StoreFile sf : sfs) {
|
||||
max = Math.max(max, sf.getMaxSequenceId());
|
||||
}
|
||||
return max;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if this storefile was created by bulk load.
|
||||
* When a hfile is bulk loaded into HBase, we append
|
||||
* {@code '_SeqId_<id-when-loaded>'} to the hfile name, unless
|
||||
* "hbase.mapreduce.bulkload.assign.sequenceNumbers" is
|
||||
* explicitly turned off.
|
||||
* If "hbase.mapreduce.bulkload.assign.sequenceNumbers"
|
||||
* is turned off, fall back to BULKLOAD_TIME_KEY.
|
||||
* Check if this storefile was created by bulk load. When a hfile is bulk loaded into HBase, we
|
||||
* append {@code '_SeqId_<id-when-loaded>'} to the hfile name, unless
|
||||
* "hbase.mapreduce.bulkload.assign.sequenceNumbers" is explicitly turned off. If
|
||||
* "hbase.mapreduce.bulkload.assign.sequenceNumbers" is turned off, fall back to
|
||||
* BULKLOAD_TIME_KEY.
|
||||
* @return true if this storefile was created by bulk load.
|
||||
*/
|
||||
public boolean isBulkLoadResult() {
|
||||
boolean bulkLoadedHFile = false;
|
||||
String fileName = this.getPath().getName();
|
||||
int startPos = fileName.indexOf("SeqId_");
|
||||
if (startPos != -1) {
|
||||
bulkLoadedHFile = true;
|
||||
}
|
||||
return bulkLoadedHFile || (metadataMap != null && metadataMap.containsKey(BULKLOAD_TIME_KEY));
|
||||
}
|
||||
boolean isBulkLoadResult();
|
||||
|
||||
@VisibleForTesting
|
||||
public boolean isCompactedAway() {
|
||||
return compactedAway;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public int getRefCount() {
|
||||
return refCount.get();
|
||||
}
|
||||
boolean isCompactedAway();
|
||||
|
||||
/**
|
||||
* @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.
|
||||
return rc > 0;
|
||||
}
|
||||
boolean isReferencedInReads();
|
||||
|
||||
/**
|
||||
* Return the timestamp at which this bulk load file was generated.
|
||||
*/
|
||||
public long getBulkLoadTimestamp() {
|
||||
byte[] bulkLoadTimestamp = metadataMap.get(BULKLOAD_TIME_KEY);
|
||||
return (bulkLoadTimestamp == null) ? 0 : Bytes.toLong(bulkLoadTimestamp);
|
||||
}
|
||||
OptionalLong getBulkLoadTimestamp();
|
||||
|
||||
/**
|
||||
* @return the cached value of HDFS blocks distribution. The cached value is
|
||||
* calculated when store file is opened.
|
||||
* @return the cached value of HDFS blocks distribution. The cached value is calculated when store
|
||||
* file is opened.
|
||||
*/
|
||||
public HDFSBlocksDistribution getHDFSBlockDistribution() {
|
||||
return this.fileInfo.getHDFSBlockDistribution();
|
||||
}
|
||||
|
||||
/**
|
||||
* Opens reader on this store file. Called by Constructor.
|
||||
* @throws IOException
|
||||
* @see #closeReader(boolean)
|
||||
*/
|
||||
private void open() throws IOException {
|
||||
if (this.reader != null) {
|
||||
throw new IllegalAccessError("Already open");
|
||||
}
|
||||
|
||||
// Open the StoreFile.Reader
|
||||
this.reader = fileInfo.open(this.fs, this.cacheConf, false, noReadahead ? 0L : -1L,
|
||||
primaryReplica, refCount, true);
|
||||
|
||||
// Load up indices and fileinfo. This also loads Bloom filter type.
|
||||
metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
|
||||
|
||||
// Read in our metadata.
|
||||
byte [] b = metadataMap.get(MAX_SEQ_ID_KEY);
|
||||
if (b != null) {
|
||||
// By convention, if halfhfile, top half has a sequence number > bottom
|
||||
// half. Thats why we add one in below. Its done for case the two halves
|
||||
// are ever merged back together --rare. Without it, on open of store,
|
||||
// since store files are distinguished by sequence id, the one half would
|
||||
// subsume the other.
|
||||
this.sequenceid = Bytes.toLong(b);
|
||||
if (fileInfo.isTopReference()) {
|
||||
this.sequenceid += 1;
|
||||
}
|
||||
}
|
||||
|
||||
if (isBulkLoadResult()){
|
||||
// generate the sequenceId from the fileName
|
||||
// fileName is of the form <randomName>_SeqId_<id-when-loaded>_
|
||||
String fileName = this.getPath().getName();
|
||||
// Use lastIndexOf() to get the last, most recent bulk load seqId.
|
||||
int startPos = fileName.lastIndexOf("SeqId_");
|
||||
if (startPos != -1) {
|
||||
this.sequenceid = Long.parseLong(fileName.substring(startPos + 6,
|
||||
fileName.indexOf('_', startPos + 6)));
|
||||
// Handle reference files as done above.
|
||||
if (fileInfo.isTopReference()) {
|
||||
this.sequenceid += 1;
|
||||
}
|
||||
}
|
||||
// SKIP_RESET_SEQ_ID only works in bulk loaded file.
|
||||
// In mob compaction, the hfile where the cells contain the path of a new mob file is bulk
|
||||
// loaded to hbase, these cells have the same seqIds with the old ones. We do not want
|
||||
// to reset new seqIds for them since this might make a mess of the visibility of cells that
|
||||
// have the same row key but different seqIds.
|
||||
boolean skipResetSeqId = isSkipResetSeqId(metadataMap.get(SKIP_RESET_SEQ_ID));
|
||||
if (skipResetSeqId) {
|
||||
// increase the seqId when it is a bulk loaded file from mob compaction.
|
||||
this.sequenceid += 1;
|
||||
}
|
||||
this.reader.setSkipResetSeqId(skipResetSeqId);
|
||||
this.reader.setBulkLoaded(true);
|
||||
}
|
||||
this.reader.setSequenceID(this.sequenceid);
|
||||
|
||||
b = metadataMap.get(HFile.Writer.MAX_MEMSTORE_TS_KEY);
|
||||
if (b != null) {
|
||||
this.maxMemstoreTS = Bytes.toLong(b);
|
||||
}
|
||||
|
||||
b = metadataMap.get(MAJOR_COMPACTION_KEY);
|
||||
if (b != null) {
|
||||
boolean mc = Bytes.toBoolean(b);
|
||||
if (this.majorCompaction == null) {
|
||||
this.majorCompaction = new AtomicBoolean(mc);
|
||||
} else {
|
||||
this.majorCompaction.set(mc);
|
||||
}
|
||||
} else {
|
||||
// Presume it is not major compacted if it doesn't explicity say so
|
||||
// HFileOutputFormat explicitly sets the major compacted key.
|
||||
this.majorCompaction = new AtomicBoolean(false);
|
||||
}
|
||||
|
||||
b = metadataMap.get(EXCLUDE_FROM_MINOR_COMPACTION_KEY);
|
||||
this.excludeFromMinorCompaction = (b != null && Bytes.toBoolean(b));
|
||||
|
||||
BloomType hfileBloomType = reader.getBloomFilterType();
|
||||
if (cfBloomType != BloomType.NONE) {
|
||||
reader.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
|
||||
if (hfileBloomType != cfBloomType) {
|
||||
LOG.info("HFile Bloom filter type for "
|
||||
+ reader.getHFileReader().getName() + ": " + hfileBloomType
|
||||
+ ", but " + cfBloomType + " specified in column family "
|
||||
+ "configuration");
|
||||
}
|
||||
} else if (hfileBloomType != BloomType.NONE) {
|
||||
LOG.info("Bloom filter turned off by CF config for "
|
||||
+ reader.getHFileReader().getName());
|
||||
}
|
||||
|
||||
// load delete family bloom filter
|
||||
reader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
|
||||
|
||||
try {
|
||||
this.reader.timeRange = TimeRangeTracker.getTimeRange(metadataMap.get(TIMERANGE_KEY));
|
||||
} catch (IllegalArgumentException e) {
|
||||
LOG.error("Error reading timestamp range data from meta -- " +
|
||||
"proceeding without", e);
|
||||
this.reader.timeRange = null;
|
||||
}
|
||||
// initialize so we can reuse them after reader closed.
|
||||
firstKey = reader.getFirstKey();
|
||||
lastKey = reader.getLastKey();
|
||||
comparator = reader.getComparator();
|
||||
}
|
||||
HDFSBlocksDistribution getHDFSBlockDistribution();
|
||||
|
||||
/**
|
||||
* Initialize the reader used for pread.
|
||||
*/
|
||||
public void initReader() throws IOException {
|
||||
if (reader == null) {
|
||||
try {
|
||||
open();
|
||||
} catch (Exception e) {
|
||||
try {
|
||||
boolean evictOnClose = cacheConf != null ? cacheConf.shouldEvictOnClose() : true;
|
||||
this.closeReader(evictOnClose);
|
||||
} catch (IOException ee) {
|
||||
LOG.warn("failed to close reader", ee);
|
||||
}
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private StoreFileReader createStreamReader(boolean canUseDropBehind) throws IOException {
|
||||
initReader();
|
||||
StoreFileReader reader = fileInfo.open(this.fs, this.cacheConf, canUseDropBehind, -1L,
|
||||
primaryReplica, refCount, false);
|
||||
reader.copyFields(this.reader);
|
||||
return reader;
|
||||
}
|
||||
|
||||
public StoreFileScanner getPreadScanner(boolean cacheBlocks, long readPt, long scannerOrder,
|
||||
boolean canOptimizeForNonNullColumn) {
|
||||
return getReader().getStoreFileScanner(cacheBlocks, true, false, readPt, scannerOrder,
|
||||
canOptimizeForNonNullColumn);
|
||||
}
|
||||
|
||||
public StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks,
|
||||
boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn)
|
||||
throws IOException {
|
||||
return createStreamReader(canUseDropBehind).getStoreFileScanner(cacheBlocks, false,
|
||||
isCompaction, readPt, scannerOrder, canOptimizeForNonNullColumn);
|
||||
}
|
||||
void initReader() throws IOException;
|
||||
|
||||
/**
|
||||
* @return Current reader. Must call initReader first else returns null.
|
||||
* 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()
|
||||
*/
|
||||
public StoreFileReader getReader() {
|
||||
return this.reader;
|
||||
}
|
||||
StoreFileReader getReader();
|
||||
|
||||
/**
|
||||
* @param evictOnClose whether to evict blocks belonging to this file
|
||||
* @throws IOException
|
||||
*/
|
||||
public synchronized void closeReader(boolean evictOnClose)
|
||||
throws IOException {
|
||||
if (this.reader != null) {
|
||||
this.reader.close(evictOnClose);
|
||||
this.reader = null;
|
||||
}
|
||||
}
|
||||
void closeReader(boolean evictOnClose) throws IOException;
|
||||
|
||||
/**
|
||||
* Marks the status of the file as compactedAway.
|
||||
*/
|
||||
public void markCompactedAway() {
|
||||
this.compactedAway = true;
|
||||
}
|
||||
void markCompactedAway();
|
||||
|
||||
/**
|
||||
* Delete this file
|
||||
* @throws IOException
|
||||
*/
|
||||
public void deleteReader() throws IOException {
|
||||
boolean evictOnClose =
|
||||
cacheConf != null? cacheConf.shouldEvictOnClose(): true;
|
||||
closeReader(evictOnClose);
|
||||
this.fs.delete(getPath(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return this.fileInfo.toString();
|
||||
}
|
||||
void deleteReader() throws IOException;
|
||||
|
||||
/**
|
||||
* @return a length description of this StoreFile, suitable for debug output
|
||||
*/
|
||||
public String toStringDetailed() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append(this.getPath().toString());
|
||||
sb.append(", isReference=").append(isReference());
|
||||
sb.append(", isBulkLoadResult=").append(isBulkLoadResult());
|
||||
if (isBulkLoadResult()) {
|
||||
sb.append(", bulkLoadTS=").append(getBulkLoadTimestamp());
|
||||
} else {
|
||||
sb.append(", seqid=").append(getMaxSequenceId());
|
||||
}
|
||||
sb.append(", majorCompaction=").append(isMajorCompaction());
|
||||
String toStringDetailed();
|
||||
|
||||
return sb.toString();
|
||||
}
|
||||
OptionalLong getMinimumTimestamp();
|
||||
|
||||
/**
|
||||
* Gets whether to skip resetting the sequence id for cells.
|
||||
* @param skipResetSeqId The byte array of boolean.
|
||||
* @return Whether to skip resetting the sequence id.
|
||||
*/
|
||||
private boolean isSkipResetSeqId(byte[] skipResetSeqId) {
|
||||
if (skipResetSeqId != null && skipResetSeqId.length == 1) {
|
||||
return Bytes.toBoolean(skipResetSeqId);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param fs
|
||||
* @param dir Directory to create file in.
|
||||
* @return random filename inside passed <code>dir</code>
|
||||
*/
|
||||
public static Path getUniqueFile(final FileSystem fs, final Path dir)
|
||||
throws IOException {
|
||||
if (!fs.getFileStatus(dir).isDirectory()) {
|
||||
throw new IOException("Expecting " + dir.toString() +
|
||||
" to be a directory");
|
||||
}
|
||||
return new Path(dir, UUID.randomUUID().toString().replaceAll("-", ""));
|
||||
}
|
||||
|
||||
public Long getMinimumTimestamp() {
|
||||
return getReader().timeRange == null? null: getReader().timeRange.getMin();
|
||||
}
|
||||
|
||||
public Long getMaximumTimestamp() {
|
||||
return getReader().timeRange == null? null: getReader().timeRange.getMax();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Gets the approximate mid-point of this file that is optimal for use in splitting it.
|
||||
* @param comparator Comparator used to compare KVs.
|
||||
* @return The split point row, or null if splitting is not possible, or reader is null.
|
||||
*/
|
||||
byte[] getFileSplitPoint(CellComparator comparator) throws IOException {
|
||||
if (this.reader == null) {
|
||||
LOG.warn("Storefile " + this + " Reader is null; cannot get split point");
|
||||
return null;
|
||||
}
|
||||
// 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 = this.reader.midkey();
|
||||
if (midkey != null) {
|
||||
Cell firstKey = this.reader.getFirstKey();
|
||||
Cell lastKey = this.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 null;
|
||||
}
|
||||
return CellUtil.cloneRow(midkey);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Useful comparators for comparing StoreFiles.
|
||||
*/
|
||||
public abstract static class Comparators {
|
||||
/**
|
||||
* Comparator that compares based on the Sequence Ids of the
|
||||
* the StoreFiles. 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 =
|
||||
Ordering.compound(ImmutableList.of(
|
||||
Ordering.natural().onResultOf(new GetSeqId()),
|
||||
Ordering.natural().onResultOf(new GetFileSize()).reverse(),
|
||||
Ordering.natural().onResultOf(new GetBulkTime()),
|
||||
Ordering.natural().onResultOf(new GetPathName())
|
||||
));
|
||||
|
||||
/**
|
||||
* Comparator for time-aware compaction. SeqId is still the first
|
||||
* ordering criterion to maintain MVCC.
|
||||
*/
|
||||
public static final Comparator<StoreFile> 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> {
|
||||
@Override
|
||||
public Long apply(StoreFile sf) {
|
||||
return sf.getMaxSequenceId();
|
||||
}
|
||||
}
|
||||
|
||||
private static class GetFileSize implements Function<StoreFile, Long> {
|
||||
@Override
|
||||
public Long apply(StoreFile sf) {
|
||||
if (sf.getReader() != null) {
|
||||
return sf.getReader().length();
|
||||
} else {
|
||||
// the reader may be null for the compacted files and if the archiving
|
||||
// had failed.
|
||||
return -1L;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static class GetBulkTime implements Function<StoreFile, Long> {
|
||||
@Override
|
||||
public Long apply(StoreFile sf) {
|
||||
if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
|
||||
return sf.getBulkLoadTimestamp();
|
||||
}
|
||||
}
|
||||
|
||||
private static class GetPathName implements Function<StoreFile, String> {
|
||||
@Override
|
||||
public String apply(StoreFile sf) {
|
||||
return sf.getPath().getName();
|
||||
}
|
||||
}
|
||||
|
||||
private static class GetMaxTimestamp implements Function<StoreFile, Long> {
|
||||
@Override
|
||||
public Long apply(StoreFile sf) {
|
||||
return sf.getMaximumTimestamp() == null? (Long)Long.MAX_VALUE : sf.getMaximumTimestamp();
|
||||
}
|
||||
}
|
||||
}
|
||||
OptionalLong getMaximumTimestamp();
|
||||
}
|
||||
|
|
|
@ -0,0 +1,96 @@
|
|||
/**
|
||||
* 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
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Ordering;
|
||||
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
* Useful comparators for comparing StoreFiles.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
final class StoreFileComparators {
|
||||
/**
|
||||
* Comparator that compares based on the Sequence Ids of the the StoreFiles. 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 =
|
||||
Ordering.compound(ImmutableList.of(Ordering.natural().onResultOf(new GetSeqId()),
|
||||
Ordering.natural().onResultOf(new GetFileSize()).reverse(),
|
||||
Ordering.natural().onResultOf(new GetBulkTime()),
|
||||
Ordering.natural().onResultOf(new GetPathName())));
|
||||
|
||||
/**
|
||||
* Comparator for time-aware compaction. SeqId is still the first ordering criterion to maintain
|
||||
* MVCC.
|
||||
*/
|
||||
public static final Comparator<StoreFile> 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> {
|
||||
@Override
|
||||
public Long apply(StoreFile sf) {
|
||||
return sf.getMaxSequenceId();
|
||||
}
|
||||
}
|
||||
|
||||
private static class GetFileSize implements Function<StoreFile, Long> {
|
||||
@Override
|
||||
public Long apply(StoreFile sf) {
|
||||
if (sf.getReader() != null) {
|
||||
return sf.getReader().length();
|
||||
} else {
|
||||
// the reader may be null for the compacted files and if the archiving
|
||||
// had failed.
|
||||
return -1L;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static class GetBulkTime implements Function<StoreFile, Long> {
|
||||
@Override
|
||||
public Long apply(StoreFile sf) {
|
||||
return sf.getBulkLoadTimestamp().orElse(Long.MAX_VALUE);
|
||||
}
|
||||
}
|
||||
|
||||
private static class GetPathName implements Function<StoreFile, String> {
|
||||
@Override
|
||||
public String apply(StoreFile sf) {
|
||||
return sf.getPath().getName();
|
||||
}
|
||||
}
|
||||
|
||||
private static class GetMaxTimestamp implements Function<StoreFile, Long> {
|
||||
@Override
|
||||
public Long apply(StoreFile sf) {
|
||||
return sf.getMaximumTimestamp().orElse(Long.MAX_VALUE);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -283,7 +283,6 @@ public class StoreFileInfo {
|
|||
*/
|
||||
public HDFSBlocksDistribution computeHDFSBlocksDistribution(final FileSystem fs)
|
||||
throws IOException {
|
||||
|
||||
// guard against the case where we get the FileStatus from link, but by the time we
|
||||
// call compute the file is moved again
|
||||
if (this.link != null) {
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.ArrayList;
|
|||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.PriorityQueue;
|
||||
import java.util.concurrent.atomic.LongAdder;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -120,13 +121,20 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
public static List<StoreFileScanner> getScannersForStoreFiles(Collection<StoreFile> files,
|
||||
boolean cacheBlocks, boolean usePread, boolean isCompaction, boolean canUseDrop,
|
||||
ScanQueryMatcher matcher, long readPt) throws IOException {
|
||||
if (files.isEmpty()) {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
List<StoreFileScanner> scanners = new ArrayList<>(files.size());
|
||||
List<StoreFile> sortedFiles = new ArrayList<>(files);
|
||||
Collections.sort(sortedFiles, StoreFile.Comparators.SEQ_ID);
|
||||
boolean canOptimizeForNonNullColumn = matcher != null ? !matcher.hasNullColumnInQuery() : false;
|
||||
for (int i = 0, n = sortedFiles.size(); i < n; i++) {
|
||||
StoreFile sf = sortedFiles.get(i);
|
||||
sf.initReader();
|
||||
PriorityQueue<StoreFile> sortedFiles =
|
||||
new PriorityQueue<>(files.size(), StoreFileComparators.SEQ_ID);
|
||||
for (StoreFile 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();
|
||||
StoreFileScanner scanner;
|
||||
if (usePread) {
|
||||
scanner = sf.getPreadScanner(cacheBlocks, readPt, i, canOptimizeForNonNullColumn);
|
||||
|
@ -147,7 +155,7 @@ public class StoreFileScanner implements KeyValueScanner {
|
|||
boolean canUseDropBehind, long readPt) throws IOException {
|
||||
List<StoreFileScanner> scanners = new ArrayList<>(files.size());
|
||||
List<StoreFile> sortedFiles = new ArrayList<>(files);
|
||||
Collections.sort(sortedFiles, StoreFile.Comparators.SEQ_ID);
|
||||
Collections.sort(sortedFiles, StoreFileComparators.SEQ_ID);
|
||||
boolean succ = false;
|
||||
try {
|
||||
for (int i = 0, n = sortedFiles.size(); i < n; i++) {
|
||||
|
|
|
@ -18,8 +18,11 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -33,7 +36,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.fs.HFileSystem;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
|
@ -42,13 +44,10 @@ import org.apache.hadoop.hbase.util.BloomFilterFactory;
|
|||
import org.apache.hadoop.hbase.util.BloomFilterWriter;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
import org.apache.hadoop.hbase.util.RowBloomContext;
|
||||
import org.apache.hadoop.hbase.util.RowColBloomContext;
|
||||
import org.apache.hadoop.io.WritableUtils;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* A StoreFile writer. Use this to read/write HBase Store Files. It is package
|
||||
* local because it is an implementation detail of the HBase regionserver.
|
||||
|
@ -359,6 +358,18 @@ public class StoreFileWriter implements CellSink, ShipperListener {
|
|||
return writer;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param fs
|
||||
* @param dir Directory to create file in.
|
||||
* @return random filename inside passed <code>dir</code>
|
||||
*/
|
||||
static Path getUniqueFile(final FileSystem fs, final Path dir) throws IOException {
|
||||
if (!fs.getFileStatus(dir).isDirectory()) {
|
||||
throw new IOException("Expecting " + dir.toString() + " to be a directory");
|
||||
}
|
||||
return new Path(dir, UUID.randomUUID().toString().replaceAll("-", ""));
|
||||
}
|
||||
|
||||
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="ICAST_INTEGER_MULTIPLY_CAST_TO_LONG",
|
||||
justification="Will not overflow")
|
||||
public static class Builder {
|
||||
|
@ -496,7 +507,7 @@ public class StoreFileWriter implements CellSink, ShipperListener {
|
|||
FSUtils.setStoragePolicy(this.fs, dir, policyName);
|
||||
|
||||
if (filePath == null) {
|
||||
filePath = StoreFile.getUniqueFile(fs, dir);
|
||||
filePath = getUniqueFile(fs, dir);
|
||||
if (!BloomFilterFactory.isGeneralBloomEnabled(conf)) {
|
||||
bloomType = BloomType.NONE;
|
||||
}
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -55,6 +53,8 @@ import org.apache.hadoop.hbase.regionserver.querymatcher.UserScanQueryMatcher;
|
|||
import org.apache.hadoop.hbase.util.CollectionUtils;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
* Scanner scans both the memstore and the Store. Coalesce KeyValue stream into List<KeyValue>
|
||||
* for a single row.
|
||||
|
|
|
@ -20,7 +20,14 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Optional;
|
||||
import java.util.OptionalInt;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
|
||||
/**
|
||||
|
@ -28,14 +35,14 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class StoreUtils {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(StoreUtils.class);
|
||||
|
||||
/**
|
||||
* Creates a deterministic hash code for store file collection.
|
||||
*/
|
||||
public static Integer getDeterministicRandomSeed(final Collection<StoreFile> files) {
|
||||
if (files != null && !files.isEmpty()) {
|
||||
return files.iterator().next().getPath().getName().hashCode();
|
||||
}
|
||||
return null;
|
||||
public static OptionalInt getDeterministicRandomSeed(Collection<StoreFile> files) {
|
||||
return files.stream().mapToInt(f -> f.getPath().getName().hashCode()).findFirst();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -70,18 +77,73 @@ public class StoreUtils {
|
|||
* @param candidates The files to choose from.
|
||||
* @return The largest file; null if no file has a reader.
|
||||
*/
|
||||
static StoreFile getLargestFile(final Collection<StoreFile> candidates) {
|
||||
long maxSize = -1L;
|
||||
StoreFile largestSf = null;
|
||||
for (StoreFile sf : candidates) {
|
||||
StoreFileReader r = sf.getReader();
|
||||
if (r == null) continue;
|
||||
long size = r.length();
|
||||
if (size > maxSize) {
|
||||
maxSize = size;
|
||||
largestSf = sf;
|
||||
static Optional<StoreFile> getLargestFile(Collection<StoreFile> candidates) {
|
||||
return candidates.stream().filter(f -> f.getReader() != null)
|
||||
.max((f1, f2) -> Long.compare(f1.getReader().length(), f2.getReader().length()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the largest memstoreTS found across all storefiles in the given list. Store files that
|
||||
* were created by a mapreduce bulk load are ignored, as they do not correspond to any specific
|
||||
* put operation, and thus do not have a memstoreTS associated with them.
|
||||
* @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 largestSf;
|
||||
return max;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
public static long getMaxSequenceIdInList(Collection<StoreFile> sfs) {
|
||||
long max = 0;
|
||||
for (StoreFile sf : sfs) {
|
||||
max = Math.max(max, sf.getMaxSequenceId());
|
||||
}
|
||||
return max;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the approximate mid-point of the given file that is optimal for use in splitting it.
|
||||
* @param file the store file
|
||||
* @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)
|
||||
throws IOException {
|
||||
StoreFileReader reader = file.getReader();
|
||||
if (reader == null) {
|
||||
LOG.warn("Storefile " + file + " Reader is null; cannot get split point");
|
||||
return Optional.empty();
|
||||
}
|
||||
// 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));
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import java.util.HashMap;
|
|||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -38,7 +39,6 @@ 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.hadoop.hbase.KeyValue.KeyOnlyKeyValue;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.StripeCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -216,7 +216,12 @@ public class StripeStoreFileManager
|
|||
return original;
|
||||
}
|
||||
|
||||
@Override
|
||||
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
|
||||
|
@ -224,6 +229,7 @@ public class StripeStoreFileManager
|
|||
* minimize imbalance.
|
||||
* @return The split point, or null if no split is possible.
|
||||
*/
|
||||
@Override
|
||||
public byte[] getSplitPoint() throws IOException {
|
||||
if (this.getStorefileCount() == 0) return null;
|
||||
if (state.stripeFiles.size() <= 1) {
|
||||
|
@ -271,16 +277,14 @@ public class StripeStoreFileManager
|
|||
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 StoreUtils.getLargestFile(state.stripeFiles.get(
|
||||
isRightLarger ? rightIndex : leftIndex)).getFileSplitPoint(this.cellComparator);
|
||||
return getSplitPoint(state.stripeFiles.get(isRightLarger ? rightIndex : leftIndex));
|
||||
}
|
||||
|
||||
private byte[] getSplitPointFromAllFiles() throws IOException {
|
||||
ConcatenatedLists<StoreFile> sfs = new ConcatenatedLists<>();
|
||||
sfs.addSublist(state.level0Files);
|
||||
sfs.addAllSublists(state.stripeFiles);
|
||||
if (sfs.isEmpty()) return null;
|
||||
return StoreUtils.getLargestFile(sfs).getFileSplitPoint(this.cellComparator);
|
||||
return getSplitPoint(sfs);
|
||||
}
|
||||
|
||||
private double getMidStripeSplitRatio(long smallerSize, long largerSize, long lastLargerSize) {
|
||||
|
@ -639,7 +643,7 @@ public class StripeStoreFileManager
|
|||
// we will store the file in reverse order by seqNum from the outset.
|
||||
for (int insertBefore = 0; ; ++insertBefore) {
|
||||
if (insertBefore == stripe.size()
|
||||
|| (StoreFile.Comparators.SEQ_ID.compare(sf, stripe.get(insertBefore)) >= 0)) {
|
||||
|| (StoreFileComparators.SEQ_ID.compare(sf, stripe.get(insertBefore)) >= 0)) {
|
||||
stripe.add(insertBefore, sf);
|
||||
break;
|
||||
}
|
||||
|
@ -1071,6 +1075,6 @@ public class StripeStoreFileManager
|
|||
|
||||
@Override
|
||||
public Comparator<StoreFile> getStoreFileComparator() {
|
||||
return StoreFile.Comparators.SEQ_ID;
|
||||
return StoreFileComparators.SEQ_ID;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,24 +18,19 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver.compactions;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.Collections2;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
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.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* This class holds all logical details necessary to run a compaction.
|
||||
|
@ -43,7 +38,7 @@ import org.apache.hadoop.util.StringUtils;
|
|||
@InterfaceAudience.LimitedPrivate({ "coprocessor" })
|
||||
@InterfaceStability.Evolving
|
||||
public class CompactionRequest implements Comparable<CompactionRequest> {
|
||||
private static final Log LOG = LogFactory.getLog(CompactionRequest.class);
|
||||
|
||||
// was this compaction promoted to an off-peak
|
||||
private boolean isOffPeak = false;
|
||||
private enum DisplayCompactionType { MINOR, ALL_FILES, MAJOR }
|
||||
|
@ -207,27 +202,15 @@ public class CompactionRequest implements Comparable<CompactionRequest> {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
String fsList = Joiner.on(", ").join(
|
||||
Collections2.transform(Collections2.filter(
|
||||
this.getFiles(),
|
||||
new Predicate<StoreFile>() {
|
||||
@Override
|
||||
public boolean apply(StoreFile sf) {
|
||||
return sf.getReader() != null;
|
||||
}
|
||||
}), new Function<StoreFile, String>() {
|
||||
@Override
|
||||
public String apply(StoreFile sf) {
|
||||
return StringUtils.humanReadableInt(
|
||||
(sf.getReader() == null) ? 0 : sf.getReader().length());
|
||||
}
|
||||
}));
|
||||
String fsList = filesToCompact.stream().filter(f -> f.getReader() != null)
|
||||
.map(f -> TraditionalBinaryPrefix.long2String(f.getReader().length(), "", 1))
|
||||
.collect(Collectors.joining(", "));
|
||||
|
||||
return "regionName=" + regionName + ", storeName=" + storeName +
|
||||
", fileCount=" + this.getFiles().size() +
|
||||
", fileSize=" + StringUtils.humanReadableInt(totalSize) +
|
||||
((fsList.isEmpty()) ? "" : " (" + fsList + ")") +
|
||||
", priority=" + priority + ", time=" + timeInNanos;
|
||||
return "regionName=" + regionName + ", storeName=" + storeName + ", fileCount=" +
|
||||
this.getFiles().size() + ", fileSize=" +
|
||||
TraditionalBinaryPrefix.long2String(totalSize, "", 1) +
|
||||
((fsList.isEmpty()) ? "" : " (" + fsList + ")") + ", priority=" + priority + ", time=" +
|
||||
timeInNanos;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -17,8 +17,6 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver.compactions;
|
||||
|
||||
import com.google.common.io.Closeables;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -60,6 +58,8 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
|
||||
|
||||
import com.google.common.io.Closeables;
|
||||
|
||||
/**
|
||||
* A compactor is a compaction algorithm associated a given policy. Base class also contains
|
||||
* reusable parts for implementing compactors (what is common and what isn't is evolving).
|
||||
|
|
|
@ -18,17 +18,12 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver.compactions;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.PeekingIterator;
|
||||
import com.google.common.math.LongMath;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.OptionalLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -44,6 +39,12 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
|||
import org.apache.hadoop.hbase.util.Pair;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.PeekingIterator;
|
||||
import com.google.common.math.LongMath;
|
||||
|
||||
/**
|
||||
* HBASE-15181 This is a simple implementation of date-based tiered compaction similar to
|
||||
* Cassandra's for the following benefits:
|
||||
|
@ -135,24 +136,24 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
|
|||
boolean[] filesInWindow = new boolean[boundaries.size()];
|
||||
|
||||
for (StoreFile file: filesToCompact) {
|
||||
Long minTimestamp = file.getMinimumTimestamp();
|
||||
long oldest = (minTimestamp == null) ? Long.MIN_VALUE : now - minTimestamp.longValue();
|
||||
OptionalLong minTimestamp = file.getMinimumTimestamp();
|
||||
long oldest = minTimestamp.isPresent() ? now - minTimestamp.getAsLong() : Long.MIN_VALUE;
|
||||
if (cfTTL != Long.MAX_VALUE && oldest >= cfTTL) {
|
||||
LOG.debug("Major compaction triggered on store " + this
|
||||
+ "; for TTL maintenance");
|
||||
return true;
|
||||
}
|
||||
if (!file.isMajorCompaction() || file.isBulkLoadResult()) {
|
||||
if (!file.isMajorCompactionResult() || file.isBulkLoadResult()) {
|
||||
LOG.debug("Major compaction triggered on store " + this
|
||||
+ ", because there are new files and time since last major compaction "
|
||||
+ (now - lowTimestamp) + "ms");
|
||||
return true;
|
||||
}
|
||||
|
||||
int lowerWindowIndex = Collections.binarySearch(boundaries,
|
||||
minTimestamp == null ? (Long)Long.MAX_VALUE : minTimestamp);
|
||||
int upperWindowIndex = Collections.binarySearch(boundaries,
|
||||
file.getMaximumTimestamp() == null ? (Long)Long.MAX_VALUE : file.getMaximumTimestamp());
|
||||
int lowerWindowIndex =
|
||||
Collections.binarySearch(boundaries, minTimestamp.orElse(Long.MAX_VALUE));
|
||||
int upperWindowIndex =
|
||||
Collections.binarySearch(boundaries, file.getMaximumTimestamp().orElse(Long.MAX_VALUE));
|
||||
// Handle boundary conditions and negative values of binarySearch
|
||||
lowerWindowIndex = (lowerWindowIndex < 0) ? Math.abs(lowerWindowIndex + 2) : lowerWindowIndex;
|
||||
upperWindowIndex = (upperWindowIndex < 0) ? Math.abs(upperWindowIndex + 2) : upperWindowIndex;
|
||||
|
@ -220,8 +221,8 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
|
|||
for (StoreFile storeFile : candidateSelection) {
|
||||
// if there is out-of-order data,
|
||||
// we put them in the same window as the last file in increasing order
|
||||
maxTimestampSeen = Math.max(maxTimestampSeen,
|
||||
storeFile.getMaximumTimestamp() == null? Long.MIN_VALUE : storeFile.getMaximumTimestamp());
|
||||
maxTimestampSeen =
|
||||
Math.max(maxTimestampSeen, storeFile.getMaximumTimestamp().orElse(Long.MIN_VALUE));
|
||||
storefileMaxTimestampPairs.add(new Pair<>(storeFile, maxTimestampSeen));
|
||||
}
|
||||
Collections.reverse(storefileMaxTimestampPairs);
|
||||
|
@ -288,23 +289,18 @@ public class DateTieredCompactionPolicy extends SortedCompactionPolicy {
|
|||
}
|
||||
|
||||
/**
|
||||
* Return a list of boundaries for multiple compaction output
|
||||
* in ascending order.
|
||||
* Return a list of boundaries for multiple compaction output in ascending order.
|
||||
*/
|
||||
private List<Long> getCompactBoundariesForMajor(Collection<StoreFile> filesToCompact, long now) {
|
||||
long minTimestamp = Long.MAX_VALUE;
|
||||
for (StoreFile file : filesToCompact) {
|
||||
minTimestamp =
|
||||
Math.min(minTimestamp,
|
||||
file.getMinimumTimestamp() == null ? Long.MAX_VALUE : file.getMinimumTimestamp());
|
||||
}
|
||||
long minTimestamp =
|
||||
filesToCompact.stream().mapToLong(f -> f.getMinimumTimestamp().orElse(Long.MAX_VALUE)).min()
|
||||
.orElse(Long.MAX_VALUE);
|
||||
|
||||
List<Long> boundaries = new ArrayList<>();
|
||||
|
||||
// Add startMillis of all windows between now and min timestamp
|
||||
for (CompactionWindow window = getIncomingWindow(now);
|
||||
window.compareToTimestamp(minTimestamp) > 0;
|
||||
window = window.nextEarlierWindow()) {
|
||||
for (CompactionWindow window = getIncomingWindow(now); window
|
||||
.compareToTimestamp(minTimestamp) > 0; window = window.nextEarlierWindow()) {
|
||||
boundaries.add(window.startMillis());
|
||||
}
|
||||
boundaries.add(Long.MIN_VALUE);
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.regionserver.DateTieredMultiFileWriter;
|
||||
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.StoreUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
|
||||
|
@ -47,7 +47,7 @@ public class DateTieredCompactor extends AbstractMultiOutputCompactor<DateTiered
|
|||
private boolean needEmptyFile(CompactionRequest request) {
|
||||
// if we are going to compact the last N files, then we need to emit an empty file to retain the
|
||||
// maxSeqId if we haven't written out anything.
|
||||
return StoreFile.getMaxSequenceIdInList(request.getFiles()) == store.getMaxSequenceId();
|
||||
return StoreUtils.getMaxSequenceIdInList(request.getFiles()) == store.getMaxSequenceId();
|
||||
}
|
||||
|
||||
public List<Path> compact(final CompactionRequest request, final List<Long> lowerBoundaries,
|
||||
|
|
|
@ -22,6 +22,7 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.OptionalLong;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -79,9 +80,9 @@ public class RatioBasedCompactionPolicy extends SortedCompactionPolicy {
|
|||
if (filesToCompact.size() == 1) {
|
||||
// Single file
|
||||
StoreFile sf = filesToCompact.iterator().next();
|
||||
Long minTimestamp = sf.getMinimumTimestamp();
|
||||
long oldest = (minTimestamp == null) ? Long.MIN_VALUE : now - minTimestamp.longValue();
|
||||
if (sf.isMajorCompaction() && (cfTTL == Long.MAX_VALUE || oldest < cfTTL)) {
|
||||
OptionalLong minTimestamp = sf.getMinimumTimestamp();
|
||||
long oldest = minTimestamp.isPresent() ? now - minTimestamp.getAsLong() : Long.MIN_VALUE;
|
||||
if (sf.isMajorCompactionResult() && (cfTTL == Long.MAX_VALUE || oldest < cfTTL)) {
|
||||
float blockLocalityIndex =
|
||||
sf.getHDFSBlockDistribution().getBlockLocalityIndex(
|
||||
RSRpcServices.getHostname(comConf.conf, false));
|
||||
|
|
|
@ -10,15 +10,11 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver.compactions;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.Collections2;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.OptionalInt;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -29,6 +25,11 @@ import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
|
|||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreUtils;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.Collections2;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* An abstract compaction policy that select files on seq id order.
|
||||
*/
|
||||
|
@ -118,30 +119,31 @@ public abstract class SortedCompactionPolicy extends CompactionPolicy {
|
|||
* @param filesToCompact
|
||||
* @return When to run next major compaction
|
||||
*/
|
||||
public long getNextMajorCompactTime(final Collection<StoreFile> filesToCompact) {
|
||||
public long getNextMajorCompactTime(Collection<StoreFile> filesToCompact) {
|
||||
// default = 24hrs
|
||||
long ret = comConf.getMajorCompactionPeriod();
|
||||
if (ret > 0) {
|
||||
// default = 20% = +/- 4.8 hrs
|
||||
double jitterPct = comConf.getMajorCompactionJitter();
|
||||
if (jitterPct > 0) {
|
||||
long jitter = Math.round(ret * jitterPct);
|
||||
// deterministic jitter avoids a major compaction storm on restart
|
||||
Integer seed = StoreUtils.getDeterministicRandomSeed(filesToCompact);
|
||||
if (seed != null) {
|
||||
// Synchronized to ensure one user of random instance at a time.
|
||||
double rnd = -1;
|
||||
synchronized (this) {
|
||||
this.random.setSeed(seed);
|
||||
rnd = this.random.nextDouble();
|
||||
}
|
||||
ret += jitter - Math.round(2L * jitter * rnd);
|
||||
} else {
|
||||
ret = 0; // If seed is null, then no storefiles == no major compaction
|
||||
}
|
||||
}
|
||||
long period = comConf.getMajorCompactionPeriod();
|
||||
if (period <= 0) {
|
||||
return period;
|
||||
}
|
||||
// default = 20% = +/- 4.8 hrs
|
||||
double jitterPct = comConf.getMajorCompactionJitter();
|
||||
if (jitterPct <= 0) {
|
||||
return period;
|
||||
}
|
||||
// deterministic jitter avoids a major compaction storm on restart
|
||||
OptionalInt seed = StoreUtils.getDeterministicRandomSeed(filesToCompact);
|
||||
if (seed.isPresent()) {
|
||||
// Synchronized to ensure one user of random instance at a time.
|
||||
double rnd;
|
||||
synchronized (this) {
|
||||
this.random.setSeed(seed.getAsInt());
|
||||
rnd = this.random.nextDouble();
|
||||
}
|
||||
long jitter = Math.round(period * jitterPct);
|
||||
return period + jitter - Math.round(2L * jitter * rnd);
|
||||
} else {
|
||||
return 0L;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -18,9 +18,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.snapshot;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -45,15 +42,17 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
|
||||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
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.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.StoreFileInfo;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.CodedInputStream;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.InvalidProtocolBufferException;
|
||||
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;
|
||||
|
|
|
@ -24,9 +24,9 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.classification.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;
|
||||
|
|
|
@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.Cell;
|
|||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.CoprocessorEnvironment;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.client.Append;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
|
@ -54,6 +53,7 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
|||
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
|
||||
import org.apache.hadoop.hbase.io.Reference;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
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.Leases;
|
||||
|
|
|
@ -22,15 +22,14 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertNotSame;
|
||||
import static org.junit.Assert.assertNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.UnsupportedEncodingException;
|
||||
import java.lang.reflect.Field;
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -81,7 +80,6 @@ import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
|
|||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile.Reader;
|
||||
import org.apache.hadoop.hbase.mapreduce.TestImportTSVWithTTLs.TTLCheckingObserver;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
|
@ -93,7 +91,6 @@ import org.apache.hadoop.hbase.testclassification.VerySlowMapReduceTests;
|
|||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.FSUtils;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
import org.apache.hadoop.hbase.util.Threads;
|
||||
import org.apache.hadoop.hbase.util.Writables;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
|
||||
|
|
|
@ -33,7 +33,7 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
|||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.HStoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
|
@ -61,7 +61,7 @@ public class TestMobFile extends TestCase {
|
|||
MobTestUtil.writeStoreFile(writer, caseName);
|
||||
|
||||
MobFile mobFile =
|
||||
new MobFile(new StoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true));
|
||||
new MobFile(new HStoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true));
|
||||
byte[] family = Bytes.toBytes(caseName);
|
||||
byte[] qualify = Bytes.toBytes(caseName);
|
||||
|
||||
|
@ -113,7 +113,7 @@ public class TestMobFile extends TestCase {
|
|||
MobTestUtil.writeStoreFile(writer, getName());
|
||||
|
||||
MobFile mobFile =
|
||||
new MobFile(new StoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true));
|
||||
new MobFile(new HStoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true));
|
||||
assertNotNull(mobFile.getScanner());
|
||||
assertTrue(mobFile.getScanner() instanceof StoreFileScanner);
|
||||
}
|
||||
|
|
|
@ -82,6 +82,7 @@ import org.apache.hadoop.hbase.mob.MobFileName;
|
|||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.BloomType;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
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.regionserver.StoreFileInfo;
|
||||
|
@ -814,7 +815,7 @@ public class TestMobCompactor {
|
|||
Assert.assertTrue(hasFiles);
|
||||
Path path = files[0].getPath();
|
||||
CacheConfig cacheConf = new CacheConfig(conf);
|
||||
StoreFile sf = new StoreFile(TEST_UTIL.getTestFileSystem(), path, conf, cacheConf,
|
||||
StoreFile sf = new HStoreFile(TEST_UTIL.getTestFileSystem(), path, conf, cacheConf,
|
||||
BloomType.NONE, true);
|
||||
sf.initReader();
|
||||
HFile.Reader reader = sf.getReader().getHFileReader();
|
||||
|
|
|
@ -18,9 +18,20 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.mob.compactions;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.text.ParseException;
|
||||
import java.util.*;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Calendar;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.RejectedExecutionException;
|
||||
import java.util.concurrent.RejectedExecutionHandler;
|
||||
|
@ -34,11 +45,16 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValue.Type;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.MobCompactPartitionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.*;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
|
@ -49,13 +65,23 @@ import org.apache.hadoop.hbase.mob.MobUtils;
|
|||
import org.apache.hadoop.hbase.mob.compactions.MobCompactionRequest.CompactionType;
|
||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionDelPartition;
|
||||
import org.apache.hadoop.hbase.mob.compactions.PartitionedMobCompactionRequest.CompactionPartition;
|
||||
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.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;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
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 static org.junit.Assert.assertTrue;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
|
@ -516,7 +542,7 @@ public class TestPartitionedMobCompactor {
|
|||
for (CompactionDelPartition delPartition : request.getDelPartitions()) {
|
||||
for (Path newDelPath : delPartition.listDelFiles()) {
|
||||
StoreFile sf =
|
||||
new StoreFile(fs, newDelPath, conf, this.cacheConfig, BloomType.NONE, true);
|
||||
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.
|
||||
sf.initReader();
|
||||
|
@ -851,7 +877,7 @@ public class TestPartitionedMobCompactor {
|
|||
List<StoreFile> sfs = new ArrayList<>();
|
||||
int size = 0;
|
||||
for (Path path : paths) {
|
||||
StoreFile sf = new StoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
|
||||
StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
|
||||
sfs.add(sf);
|
||||
}
|
||||
List<KeyValueScanner> scanners = new ArrayList<>(StoreFileScanner.getScannersForStoreFiles(sfs,
|
||||
|
|
|
@ -25,6 +25,8 @@ import static org.junit.Assert.assertNull;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
|
|
@ -17,19 +17,20 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.DateTieredCompactionRequest;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
public class AbstractTestDateTieredCompactionPolicy extends TestCompactionPolicy {
|
||||
|
||||
|
|
|
@ -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 StoreFile(fs, path, conf, cacheConf, BloomType.NONE, true);
|
||||
StoreFile 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 StoreFile(testingUtility.getTestFileSystem(),
|
||||
StoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(),
|
||||
path, configuration, cacheConf, BloomType.NONE, true);
|
||||
storeFile.initReader();
|
||||
StoreFileReader reader = storeFile.getReader();
|
||||
|
@ -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 StoreFile(testingUtility.getTestFileSystem(),
|
||||
StoreFile storeFile = new HStoreFile(testingUtility.getTestFileSystem(),
|
||||
path, configuration, cacheConf, BloomType.NONE, true);
|
||||
storeFile.initReader();
|
||||
long totalSize = 0;
|
||||
|
|
|
@ -19,11 +19,14 @@
|
|||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
|
@ -31,7 +34,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
||||
/** A mock used so our tests don't deal with actual StoreFiles */
|
||||
public class MockStoreFile extends StoreFile {
|
||||
public class MockStoreFile extends HStoreFile {
|
||||
long length = 0;
|
||||
boolean isRef = false;
|
||||
long ageInDisk;
|
||||
|
@ -64,18 +67,13 @@ public class MockStoreFile extends StoreFile {
|
|||
this.length = newLen;
|
||||
}
|
||||
|
||||
@Override
|
||||
byte[] getFileSplitPoint(CellComparator comparator) throws IOException {
|
||||
return this.splitPoint;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMaxSequenceId() {
|
||||
return sequenceid;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isMajorCompaction() {
|
||||
public boolean isMajorCompactionResult() {
|
||||
return isMajor;
|
||||
}
|
||||
|
||||
|
@ -110,14 +108,14 @@ public class MockStoreFile extends StoreFile {
|
|||
this.entryCount = entryCount;
|
||||
}
|
||||
|
||||
public Long getMinimumTimestamp() {
|
||||
return (timeRangeTracker == null) ?
|
||||
null : timeRangeTracker.getMin();
|
||||
public OptionalLong getMinimumTimestamp() {
|
||||
return timeRangeTracker == null ? OptionalLong.empty()
|
||||
: OptionalLong.of(timeRangeTracker.getMin());
|
||||
}
|
||||
|
||||
public Long getMaximumTimestamp() {
|
||||
return (timeRangeTracker == null) ?
|
||||
null : timeRangeTracker.getMax();
|
||||
public OptionalLong getMaximumTimestamp() {
|
||||
return timeRangeTracker == null ? OptionalLong.empty()
|
||||
: OptionalLong.of(timeRangeTracker.getMax());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -184,6 +182,39 @@ public class MockStoreFile extends StoreFile {
|
|||
public void close(boolean evictOnClose) throws IOException {
|
||||
// no-op
|
||||
}
|
||||
|
||||
@Override
|
||||
public Cell getLastKey() {
|
||||
if (splitPoint != null) {
|
||||
return CellUtil.createCell(Arrays.copyOf(splitPoint, splitPoint.length + 1));
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Cell midkey() throws IOException {
|
||||
if (splitPoint != null) {
|
||||
return CellUtil.createCell(splitPoint);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Cell getFirstKey() {
|
||||
if (splitPoint != null) {
|
||||
return CellUtil.createCell(Arrays.copyOf(splitPoint, splitPoint.length - 1));
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public OptionalLong getBulkLoadTimestamp() {
|
||||
// we always return false for isBulkLoadResult so we do not have a bulk load timestamp
|
||||
return OptionalLong.empty();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 StoreFile(fs, path, conf, cacheConf, BloomType.ROWCOL, true);
|
||||
StoreFile sf = new HStoreFile(fs, path, conf, cacheConf, BloomType.ROWCOL, true);
|
||||
sf.initReader();
|
||||
HFile.Reader reader = sf.getReader().getHFileReader();
|
||||
try {
|
||||
|
|
|
@ -23,6 +23,14 @@ import static org.junit.Assert.assertFalse;
|
|||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -45,14 +53,6 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
* Tests a race condition between archiving of compacted files in CompactedHFilesDischarger chore
|
||||
* and HRegion.close();
|
||||
|
|
|
@ -30,6 +30,12 @@ import static org.mockito.Mockito.spy;
|
|||
import static org.mockito.Mockito.when;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -53,11 +59,6 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Tests that archiving compacted files behaves correctly when encountering exceptions.
|
||||
*/
|
||||
|
|
|
@ -43,9 +43,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
|
|||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.io.hfile.RandomKeyValueUtil;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
import org.apache.hadoop.hbase.io.hfile.BlockCache;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
|
@ -54,9 +51,12 @@ import org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter;
|
|||
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.RandomKeyValueUtil;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.BloomFilterFactory;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.BloomFilterUtil;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
@ -200,7 +200,7 @@ public class TestCompoundBloomFilter {
|
|||
|
||||
private void readStoreFile(int t, BloomType bt, List<KeyValue> kvs,
|
||||
Path sfPath) throws IOException {
|
||||
StoreFile sf = new StoreFile(fs, sfPath, conf, cacheConf, bt, true);
|
||||
StoreFile sf = new HStoreFile(fs, sfPath, conf, cacheConf, bt, true);
|
||||
sf.initReader();
|
||||
StoreFileReader r = sf.getReader();
|
||||
final boolean pread = true; // does not really matter
|
||||
|
|
|
@ -17,7 +17,8 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.security.Key;
|
||||
|
@ -36,8 +37,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
|
|||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Waiter.Predicate;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
|
@ -49,8 +48,9 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
|||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.security.EncryptionUtil;
|
||||
import org.apache.hadoop.hbase.security.User;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Rule;
|
||||
|
|
|
@ -17,7 +17,8 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.security.Key;
|
||||
import java.util.ArrayList;
|
||||
|
@ -29,8 +30,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
|
|||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
|
@ -38,8 +37,9 @@ import org.apache.hadoop.hbase.io.crypto.Encryption;
|
|||
import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
|
|
@ -91,10 +91,10 @@ public class TestFSErrorsExposed {
|
|||
.withOutputDir(hfilePath)
|
||||
.withFileContext(meta)
|
||||
.build();
|
||||
TestStoreFile.writeStoreFile(
|
||||
TestHStoreFile.writeStoreFile(
|
||||
writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
|
||||
|
||||
StoreFile sf = new StoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf,
|
||||
StoreFile sf = new HStoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf,
|
||||
BloomType.NONE, true);
|
||||
sf.initReader();
|
||||
StoreFileReader reader = sf.getReader();
|
||||
|
@ -141,10 +141,10 @@ public class TestFSErrorsExposed {
|
|||
.withOutputDir(hfilePath)
|
||||
.withFileContext(meta)
|
||||
.build();
|
||||
TestStoreFile.writeStoreFile(
|
||||
TestHStoreFile.writeStoreFile(
|
||||
writer, Bytes.toBytes("cf"), Bytes.toBytes("qual"));
|
||||
|
||||
StoreFile sf = new StoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf,
|
||||
StoreFile sf = new HStoreFile(fs, writer.getPath(), util.getConfiguration(), cacheConf,
|
||||
BloomType.NONE, true);
|
||||
|
||||
List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(
|
||||
|
|
|
@ -18,11 +18,26 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.security.Key;
|
||||
import java.security.SecureRandom;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.concurrent.ConcurrentSkipListSet;
|
||||
|
||||
import javax.crypto.spec.SecretKeySpec;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
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.hadoop.hbase.ArrayBackedTag;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
|
@ -35,7 +50,6 @@ import org.apache.hadoop.hbase.HTableDescriptor;
|
|||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.Tag;
|
||||
import org.apache.hadoop.hbase.ArrayBackedTag;
|
||||
import org.apache.hadoop.hbase.TagType;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.Scan;
|
||||
|
@ -61,19 +75,6 @@ import org.junit.experimental.categories.Category;
|
|||
import org.junit.rules.TestName;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import javax.crypto.spec.SecretKeySpec;
|
||||
import java.io.IOException;
|
||||
import java.security.Key;
|
||||
import java.security.SecureRandom;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.NavigableSet;
|
||||
import java.util.concurrent.ConcurrentSkipListSet;
|
||||
|
||||
@Category(MediumTests.class)
|
||||
public class TestHMobStore {
|
||||
public static final Log LOG = LogFactory.getLog(TestHMobStore.class);
|
||||
|
|
|
@ -116,7 +116,6 @@ import org.apache.hadoop.hbase.filter.BinaryComparator;
|
|||
import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
|
||||
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
|
||||
import org.apache.hadoop.hbase.filter.Filter;
|
||||
import org.apache.hadoop.hbase.filter.FilterAllFilter;
|
||||
import org.apache.hadoop.hbase.filter.FilterBase;
|
||||
import org.apache.hadoop.hbase.filter.FilterList;
|
||||
import org.apache.hadoop.hbase.filter.NullComparator;
|
||||
|
|
|
@ -35,6 +35,8 @@ import static org.mockito.Mockito.times;
|
|||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -64,6 +66,10 @@ import org.apache.hadoop.hbase.client.Put;
|
|||
import org.apache.hadoop.hbase.executor.ExecutorService;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion.FlushResultImpl;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult;
|
||||
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
|
||||
|
@ -75,10 +81,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.regionserver.HRegion.FlushResultImpl;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion.PrepareFlushResult;
|
||||
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
@ -98,8 +100,6 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* Tests of HRegion methods for replaying flush, compaction, region open, etc events for secondary
|
||||
* region replicas
|
||||
|
@ -1350,7 +1350,7 @@ public class TestHRegionReplayEvents {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testRefreshStoreFiles() throws IOException {
|
||||
public void testRefresStoreFiles() throws IOException {
|
||||
assertEquals(0, primaryRegion.getStoreFileList(families).size());
|
||||
assertEquals(0, secondaryRegion.getStoreFileList(families).size());
|
||||
|
||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
import static org.hamcrest.core.Is.is;
|
||||
import static org.junit.Assert.assertThat;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InterruptedIOException;
|
||||
import java.util.ArrayList;
|
||||
|
@ -63,11 +65,11 @@ 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.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.TestWALActionsListener;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.CompactRegionRequest;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.TestWALActionsListener;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
|
||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
|
@ -81,8 +83,6 @@ import org.junit.runner.RunWith;
|
|||
import org.junit.runners.Parameterized;
|
||||
import org.junit.runners.Parameterized.Parameters;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* Tests bulk loading of HFiles and shows the atomicity or lack of atomicity of
|
||||
* the region server's bullkLoad functionality.
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.util.Collections;
|
|||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
|
@ -78,8 +79,8 @@ import static org.mockito.Mockito.when;
|
|||
* Test HStoreFile
|
||||
*/
|
||||
@Category({RegionServerTests.class, SmallTests.class})
|
||||
public class TestStoreFile extends HBaseTestCase {
|
||||
private static final Log LOG = LogFactory.getLog(TestStoreFile.class);
|
||||
public class TestHStoreFile extends HBaseTestCase {
|
||||
private static final Log LOG = LogFactory.getLog(TestHStoreFile.class);
|
||||
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
|
||||
private CacheConfig cacheConf = new CacheConfig(TEST_UTIL.getConfiguration());
|
||||
private static String ROOT_DIR = TEST_UTIL.getDataTestDir("TestStoreFile").toString();
|
||||
|
@ -117,7 +118,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
writeStoreFile(writer);
|
||||
|
||||
Path sfPath = regionFs.commitStoreFile(TEST_FAMILY, writer.getPath());
|
||||
StoreFile sf = new StoreFile(this.fs, sfPath, conf, cacheConf, BloomType.NONE, true);
|
||||
HStoreFile sf = new HStoreFile(this.fs, sfPath, conf, cacheConf, BloomType.NONE, true);
|
||||
checkHalfHFile(regionFs, sf);
|
||||
}
|
||||
|
||||
|
@ -169,7 +170,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
writeStoreFile(writer);
|
||||
|
||||
Path hsfPath = regionFs.commitStoreFile(TEST_FAMILY, writer.getPath());
|
||||
StoreFile hsf = new StoreFile(this.fs, hsfPath, conf, cacheConf, BloomType.NONE, true);
|
||||
HStoreFile hsf = new HStoreFile(this.fs, hsfPath, conf, cacheConf, BloomType.NONE, true);
|
||||
hsf.initReader();
|
||||
StoreFileReader reader = hsf.getReader();
|
||||
// Split on a row, not in middle of row. Midkey returned by reader
|
||||
|
@ -184,7 +185,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
// Make a reference
|
||||
HRegionInfo splitHri = new HRegionInfo(hri.getTable(), null, midRow);
|
||||
Path refPath = splitStoreFile(regionFs, splitHri, TEST_FAMILY, hsf, midRow, true);
|
||||
StoreFile refHsf = new StoreFile(this.fs, refPath, conf, cacheConf, BloomType.NONE, true);
|
||||
HStoreFile refHsf = new HStoreFile(this.fs, refPath, conf, cacheConf, BloomType.NONE, true);
|
||||
refHsf.initReader();
|
||||
// Now confirm that I can read from the reference and that it only gets
|
||||
// keys from top half of the file.
|
||||
|
@ -242,8 +243,8 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
|
||||
// Try to open store file from link
|
||||
StoreFileInfo storeFileInfo = new StoreFileInfo(testConf, this.fs, linkFilePath);
|
||||
StoreFile hsf =
|
||||
new StoreFile(this.fs, storeFileInfo, testConf, cacheConf, BloomType.NONE, true);
|
||||
HStoreFile hsf =
|
||||
new HStoreFile(this.fs, storeFileInfo, testConf, cacheConf, BloomType.NONE, true);
|
||||
assertTrue(storeFileInfo.isLink());
|
||||
hsf.initReader();
|
||||
|
||||
|
@ -296,7 +297,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
// <root>/clone/splitB/<cf>/<reftohfilelink>
|
||||
HRegionInfo splitHriA = new HRegionInfo(hri.getTable(), null, SPLITKEY);
|
||||
HRegionInfo splitHriB = new HRegionInfo(hri.getTable(), SPLITKEY, null);
|
||||
StoreFile f = new StoreFile(fs, linkFilePath, testConf, cacheConf, BloomType.NONE, true);
|
||||
HStoreFile f = new HStoreFile(fs, linkFilePath, testConf, cacheConf, BloomType.NONE, true);
|
||||
f.initReader();
|
||||
Path pathA = splitStoreFile(cloneRegionFs, splitHriA, TEST_FAMILY, f, SPLITKEY, true); // top
|
||||
Path pathB = splitStoreFile(cloneRegionFs, splitHriB, TEST_FAMILY, f, SPLITKEY, false);// bottom
|
||||
|
@ -308,7 +309,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
// reference to a hfile link. This code in StoreFile that handles this case.
|
||||
|
||||
// Try to open store file from link
|
||||
StoreFile hsfA = new StoreFile(this.fs, pathA, testConf, cacheConf, BloomType.NONE, true);
|
||||
HStoreFile hsfA = new HStoreFile(this.fs, pathA, testConf, cacheConf, BloomType.NONE, true);
|
||||
hsfA.initReader();
|
||||
|
||||
// Now confirm that I can read from the ref to link
|
||||
|
@ -321,7 +322,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
assertTrue(count > 0); // read some rows here
|
||||
|
||||
// Try to open store file from link
|
||||
StoreFile hsfB = new StoreFile(this.fs, pathB, testConf, cacheConf, BloomType.NONE, true);
|
||||
HStoreFile hsfB = new HStoreFile(this.fs, pathB, testConf, cacheConf, BloomType.NONE, true);
|
||||
hsfB.initReader();
|
||||
|
||||
// Now confirm that I can read from the ref to link
|
||||
|
@ -338,7 +339,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
assertEquals((LAST_CHAR - FIRST_CHAR + 1) * (LAST_CHAR - FIRST_CHAR + 1), count);
|
||||
}
|
||||
|
||||
private void checkHalfHFile(final HRegionFileSystem regionFs, final StoreFile f)
|
||||
private void checkHalfHFile(final HRegionFileSystem regionFs, final HStoreFile f)
|
||||
throws IOException {
|
||||
f.initReader();
|
||||
Cell midkey = f.getReader().midkey();
|
||||
|
@ -353,10 +354,10 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
midRow, null);
|
||||
Path bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, midRow, false);
|
||||
// Make readers on top and bottom.
|
||||
StoreFile topF = new StoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE, true);
|
||||
HStoreFile topF = new HStoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE, true);
|
||||
topF.initReader();
|
||||
StoreFileReader top = topF.getReader();
|
||||
StoreFile bottomF = new StoreFile(this.fs, bottomPath, conf, cacheConf, BloomType.NONE, true);
|
||||
HStoreFile bottomF = new HStoreFile(this.fs, bottomPath, conf, cacheConf, BloomType.NONE, true);
|
||||
bottomF.initReader();
|
||||
StoreFileReader bottom = bottomF.getReader();
|
||||
ByteBuffer previous = null;
|
||||
|
@ -416,7 +417,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
|
||||
assertNull(bottomPath);
|
||||
|
||||
topF = new StoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE, true);
|
||||
topF = new HStoreFile(this.fs, topPath, conf, cacheConf, BloomType.NONE, true);
|
||||
topF.initReader();
|
||||
top = topF.getReader();
|
||||
// Now read from the top.
|
||||
|
@ -456,7 +457,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
bottomPath = splitStoreFile(regionFs, bottomHri, TEST_FAMILY, f, badmidkey, false);
|
||||
assertNull(topPath);
|
||||
|
||||
bottomF = new StoreFile(this.fs, bottomPath, conf, cacheConf, BloomType.NONE, true);
|
||||
bottomF = new HStoreFile(this.fs, bottomPath, conf, cacheConf, BloomType.NONE, true);
|
||||
bottomF.initReader();
|
||||
bottom = bottomF.getReader();
|
||||
first = true;
|
||||
|
@ -754,7 +755,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
|
||||
@Test
|
||||
public void testSeqIdComparator() {
|
||||
assertOrdering(StoreFile.Comparators.SEQ_ID, mockStoreFile(true, 100, 1000, -1, "/foo/123"),
|
||||
assertOrdering(StoreFileComparators.SEQ_ID, mockStoreFile(true, 100, 1000, -1, "/foo/123"),
|
||||
mockStoreFile(true, 100, 1000, -1, "/foo/124"),
|
||||
mockStoreFile(true, 99, 1000, -1, "/foo/126"),
|
||||
mockStoreFile(true, 98, 2000, -1, "/foo/126"), mockStoreFile(false, 3453, -1, 1, "/foo/1"),
|
||||
|
@ -766,8 +767,8 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
* Assert that the given comparator orders the given storefiles in the
|
||||
* same way that they're passed.
|
||||
*/
|
||||
private void assertOrdering(Comparator<StoreFile> comparator, StoreFile ... sfs) {
|
||||
ArrayList<StoreFile> sorted = Lists.newArrayList(sfs);
|
||||
private void assertOrdering(Comparator<? super HStoreFile> comparator, HStoreFile ... sfs) {
|
||||
ArrayList<HStoreFile> sorted = Lists.newArrayList(sfs);
|
||||
Collections.shuffle(sorted);
|
||||
Collections.sort(sorted, comparator);
|
||||
LOG.debug("sfs: " + Joiner.on(",").join(sfs));
|
||||
|
@ -778,19 +779,19 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
/**
|
||||
* Create a mock StoreFile with the given attributes.
|
||||
*/
|
||||
private StoreFile mockStoreFile(boolean bulkLoad,
|
||||
private HStoreFile mockStoreFile(boolean bulkLoad,
|
||||
long size,
|
||||
long bulkTimestamp,
|
||||
long seqId,
|
||||
String path) {
|
||||
StoreFile mock = Mockito.mock(StoreFile.class);
|
||||
HStoreFile mock = Mockito.mock(HStoreFile.class);
|
||||
StoreFileReader reader = Mockito.mock(StoreFileReader.class);
|
||||
|
||||
Mockito.doReturn(size).when(reader).length();
|
||||
|
||||
Mockito.doReturn(reader).when(mock).getReader();
|
||||
Mockito.doReturn(bulkLoad).when(mock).isBulkLoadResult();
|
||||
Mockito.doReturn(bulkTimestamp).when(mock).getBulkLoadTimestamp();
|
||||
Mockito.doReturn(OptionalLong.of(bulkTimestamp)).when(mock).getBulkLoadTimestamp();
|
||||
Mockito.doReturn(seqId).when(mock).getMaxSequenceId();
|
||||
Mockito.doReturn(new Path(path)).when(mock).getPath();
|
||||
String name = "mock storefile, bulkLoad=" + bulkLoad +
|
||||
|
@ -855,7 +856,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
writer.appendMetadata(0, false);
|
||||
writer.close();
|
||||
|
||||
StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
|
||||
HStoreFile hsf = new HStoreFile(this.fs, writer.getPath(), conf, cacheConf,
|
||||
BloomType.NONE, true);
|
||||
Store store = mock(Store.class);
|
||||
HColumnDescriptor hcd = mock(HColumnDescriptor.class);
|
||||
|
@ -913,7 +914,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
CacheConfig cacheConf = new CacheConfig(conf);
|
||||
Path pathCowOff = new Path(baseDir, "123456789");
|
||||
StoreFileWriter writer = writeStoreFile(conf, cacheConf, pathCowOff, 3);
|
||||
StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
|
||||
HStoreFile hsf = new HStoreFile(this.fs, writer.getPath(), conf, cacheConf,
|
||||
BloomType.NONE, true);
|
||||
LOG.debug(hsf.getPath().toString());
|
||||
|
||||
|
@ -936,7 +937,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
cacheConf = new CacheConfig(conf);
|
||||
Path pathCowOn = new Path(baseDir, "123456788");
|
||||
writer = writeStoreFile(conf, cacheConf, pathCowOn, 3);
|
||||
hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
|
||||
hsf = new HStoreFile(this.fs, writer.getPath(), conf, cacheConf,
|
||||
BloomType.NONE, true);
|
||||
|
||||
// Read this file, we should see 3 hits
|
||||
|
@ -953,13 +954,13 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
reader.close(cacheConf.shouldEvictOnClose());
|
||||
|
||||
// Let's read back the two files to ensure the blocks exactly match
|
||||
hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf, BloomType.NONE, true);
|
||||
hsf = new HStoreFile(this.fs, pathCowOff, conf, cacheConf, BloomType.NONE, true);
|
||||
hsf.initReader();
|
||||
StoreFileReader readerOne = hsf.getReader();
|
||||
readerOne.loadFileInfo();
|
||||
StoreFileScanner scannerOne = getStoreFileScanner(readerOne, true, true);
|
||||
scannerOne.seek(KeyValue.LOWESTKEY);
|
||||
hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf, BloomType.NONE, true);
|
||||
hsf = new HStoreFile(this.fs, pathCowOn, conf, cacheConf, BloomType.NONE, true);
|
||||
hsf.initReader();
|
||||
StoreFileReader readerTwo = hsf.getReader();
|
||||
readerTwo.loadFileInfo();
|
||||
|
@ -992,7 +993,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
// Let's close the first file with evict on close turned on
|
||||
conf.setBoolean("hbase.rs.evictblocksonclose", true);
|
||||
cacheConf = new CacheConfig(conf);
|
||||
hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf, BloomType.NONE, true);
|
||||
hsf = new HStoreFile(this.fs, pathCowOff, conf, cacheConf, BloomType.NONE, true);
|
||||
hsf.initReader();
|
||||
reader = hsf.getReader();
|
||||
reader.close(cacheConf.shouldEvictOnClose());
|
||||
|
@ -1006,7 +1007,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
// Let's close the second file with evict on close turned off
|
||||
conf.setBoolean("hbase.rs.evictblocksonclose", false);
|
||||
cacheConf = new CacheConfig(conf);
|
||||
hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf, BloomType.NONE, true);
|
||||
hsf = new HStoreFile(this.fs, pathCowOn, conf, cacheConf, BloomType.NONE, true);
|
||||
hsf.initReader();
|
||||
reader = hsf.getReader();
|
||||
reader.close(cacheConf.shouldEvictOnClose());
|
||||
|
@ -1018,7 +1019,7 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
}
|
||||
|
||||
private Path splitStoreFile(final HRegionFileSystem regionFs, final HRegionInfo hri,
|
||||
final String family, final StoreFile sf, final byte[] splitKey, boolean isTopRef)
|
||||
final String family, final HStoreFile sf, final byte[] splitKey, boolean isTopRef)
|
||||
throws IOException {
|
||||
FileSystem fs = regionFs.getFileSystem();
|
||||
Path path = regionFs.splitStoreFile(hri, family, sf, splitKey, isTopRef, null);
|
||||
|
@ -1093,8 +1094,8 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
.build();
|
||||
writer.close();
|
||||
|
||||
StoreFile storeFile =
|
||||
new StoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true);
|
||||
HStoreFile storeFile =
|
||||
new HStoreFile(fs, writer.getPath(), conf, cacheConf, BloomType.NONE, true);
|
||||
storeFile.initReader();
|
||||
StoreFileReader reader = storeFile.getReader();
|
||||
|
||||
|
@ -1103,4 +1104,3 @@ public class TestStoreFile extends HBaseTestCase {
|
|||
assertEquals(dataBlockEncoderAlgo.getNameInBytes(), value);
|
||||
}
|
||||
}
|
||||
|
|
@ -37,7 +37,14 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.*;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Durability;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
|
@ -293,7 +300,7 @@ public class TestMobStoreCompaction {
|
|||
if (fs.exists(mobDirPath)) {
|
||||
FileStatus[] files = UTIL.getTestFileSystem().listStatus(mobDirPath);
|
||||
for (FileStatus file : files) {
|
||||
StoreFile sf = new StoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true);
|
||||
StoreFile sf = new HStoreFile(fs, file.getPath(), conf, cacheConfig, BloomType.NONE, true);
|
||||
sf.initReader();
|
||||
Map<byte[], byte[]> fileInfo = sf.getReader().loadFileInfo();
|
||||
byte[] count = fileInfo.get(StoreFile.MOB_CELLS_COUNT);
|
||||
|
@ -408,14 +415,14 @@ public class TestMobStoreCompaction {
|
|||
int size = 0;
|
||||
if (fs.exists(mobDirPath)) {
|
||||
for (FileStatus f : fs.listStatus(mobDirPath)) {
|
||||
StoreFile sf = new StoreFile(fs, f.getPath(), conf, cacheConfig, BloomType.NONE, true);
|
||||
StoreFile sf = new HStoreFile(fs, f.getPath(), conf, cacheConfig, BloomType.NONE, true);
|
||||
sfs.add(sf);
|
||||
if (StoreFileInfo.isDelFile(sf.getPath())) {
|
||||
numDelfiles++;
|
||||
}
|
||||
}
|
||||
|
||||
List scanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true, false, false,
|
||||
List<StoreFileScanner> scanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true, false, false,
|
||||
HConstants.LATEST_TIMESTAMP);
|
||||
Scan scan = new Scan();
|
||||
scan.setMaxVersions(hcd.getMaxVersions());
|
||||
|
|
|
@ -225,7 +225,7 @@ public class TestRegionReplicas {
|
|||
}
|
||||
|
||||
@Test(timeout = 300000)
|
||||
public void testRefreshStoreFiles() throws Exception {
|
||||
public void testRefresStoreFiles() throws Exception {
|
||||
// enable store file refreshing
|
||||
final int refreshPeriod = 2000; // 2 sec
|
||||
HTU.getConfiguration().setInt("hbase.hstore.compactionThreshold", 100);
|
||||
|
|
|
@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals;
|
|||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
|
@ -66,8 +68,6 @@ import org.junit.Test;
|
|||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* Test cases against ReversibleKeyValueScanner
|
||||
*/
|
||||
|
@ -113,7 +113,7 @@ public class TestReversibleScanners {
|
|||
.withFileContext(hFileContext).build();
|
||||
writeStoreFile(writer);
|
||||
|
||||
StoreFile sf = new StoreFile(fs, writer.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
|
||||
StoreFile sf = new HStoreFile(fs, writer.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
|
||||
BloomType.NONE, true);
|
||||
|
||||
List<StoreFileScanner> scanners = StoreFileScanner
|
||||
|
@ -167,10 +167,10 @@ public class TestReversibleScanners {
|
|||
writeMemstoreAndStoreFiles(memstore, new StoreFileWriter[] { writer1,
|
||||
writer2 });
|
||||
|
||||
StoreFile sf1 = new StoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
|
||||
StoreFile sf1 = new HStoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
|
||||
BloomType.NONE, true);
|
||||
|
||||
StoreFile sf2 = new StoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
|
||||
StoreFile sf2 = new HStoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
|
||||
BloomType.NONE, true);
|
||||
/**
|
||||
* Test without MVCC
|
||||
|
@ -257,10 +257,10 @@ public class TestReversibleScanners {
|
|||
writeMemstoreAndStoreFiles(memstore, new StoreFileWriter[] { writer1,
|
||||
writer2 });
|
||||
|
||||
StoreFile sf1 = new StoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
|
||||
StoreFile sf1 = new HStoreFile(fs, writer1.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
|
||||
BloomType.NONE, true);
|
||||
|
||||
StoreFile sf2 = new StoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
|
||||
StoreFile sf2 = new HStoreFile(fs, writer2.getPath(), TEST_UTIL.getConfiguration(), cacheConf,
|
||||
BloomType.NONE, true);
|
||||
|
||||
ScanType scanType = ScanType.USER_SCAN;
|
||||
|
|
|
@ -72,12 +72,11 @@ import org.apache.hadoop.hbase.coprocessor.ObserverContext;
|
|||
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
|
||||
import org.apache.hadoop.hbase.master.HMaster;
|
||||
import org.apache.hadoop.hbase.master.MasterRpcServices;
|
||||
import org.apache.hadoop.hbase.master.NoSuchProcedureException;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.master.RegionState.State;
|
||||
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
|
||||
import org.apache.hadoop.hbase.master.assignment.RegionStates;
|
||||
import org.apache.hadoop.hbase.master.NoSuchProcedureException;
|
||||
import org.apache.hadoop.hbase.procedure2.Procedure;
|
||||
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
||||
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
|
||||
|
|
|
@ -27,7 +27,6 @@ import static org.mockito.Mockito.spy;
|
|||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import java.io.IOException;
|
||||
import java.lang.ref.SoftReference;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
|
@ -45,7 +44,7 @@ import java.util.concurrent.ExecutorService;
|
|||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -100,6 +99,8 @@ import org.junit.experimental.categories.Category;
|
|||
import org.junit.rules.TestName;
|
||||
import org.mockito.Mockito;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
/**
|
||||
* Test class for the Store
|
||||
*/
|
||||
|
|
|
@ -30,6 +30,7 @@ import static org.mockito.Mockito.when;
|
|||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.OptionalLong;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -108,6 +109,7 @@ public class TestStripeStoreEngine {
|
|||
.thenReturn(StripeStoreFileManager.INVALID_KEY);
|
||||
when(sf.getReader()).thenReturn(mock(StoreFileReader.class));
|
||||
when(sf.getPath()).thenReturn(new Path("moo"));
|
||||
when(sf.getBulkLoadTimestamp()).thenReturn(OptionalLong.empty());
|
||||
return sf;
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import static org.apache.hadoop.hbase.regionserver.StripeStoreFileManager.OPEN_KEY;
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNull;
|
||||
|
@ -179,20 +180,20 @@ public class TestStripeStoreFileManager {
|
|||
|
||||
// If there are no stripes, should pick midpoint from the biggest file in L0.
|
||||
MockStoreFile sf5 = createFile(5, 0);
|
||||
sf5.splitPoint = new byte[1];
|
||||
sf5.splitPoint = new byte[] { 1 };
|
||||
manager.insertNewFiles(al(sf5));
|
||||
manager.insertNewFiles(al(createFile(1, 0)));
|
||||
assertEquals(sf5.splitPoint, manager.getSplitPoint());
|
||||
assertArrayEquals(sf5.splitPoint, manager.getSplitPoint());
|
||||
|
||||
// Same if there's one stripe but the biggest file is still in L0.
|
||||
manager.addCompactionResults(al(), al(createFile(2, 0, OPEN_KEY, OPEN_KEY)));
|
||||
assertEquals(sf5.splitPoint, manager.getSplitPoint());
|
||||
assertArrayEquals(sf5.splitPoint, manager.getSplitPoint());
|
||||
|
||||
// If the biggest file is in the stripe, should get from it.
|
||||
MockStoreFile sf6 = createFile(6, 0, OPEN_KEY, OPEN_KEY);
|
||||
sf6.splitPoint = new byte[1];
|
||||
sf6.splitPoint = new byte[] { 2 };
|
||||
manager.addCompactionResults(al(), al(sf6));
|
||||
assertEquals(sf6.splitPoint, manager.getSplitPoint());
|
||||
assertArrayEquals(sf6.splitPoint, manager.getSplitPoint());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -572,7 +573,7 @@ public class TestStripeStoreFileManager {
|
|||
private static MockStoreFile createFile(
|
||||
long size, long seqNum, byte[] startKey, byte[] endKey) throws Exception {
|
||||
FileSystem fs = TEST_UTIL.getTestFileSystem();
|
||||
Path testFilePath = StoreFile.getUniqueFile(fs, CFDIR);
|
||||
Path testFilePath = StoreFileWriter.getUniqueFile(fs, CFDIR);
|
||||
fs.create(testFilePath).close();
|
||||
MockStoreFile sf = new MockStoreFile(TEST_UTIL, testFilePath, size, 0, false, seqNum);
|
||||
if (startKey != null) {
|
||||
|
|
|
@ -18,20 +18,21 @@
|
|||
|
||||
package org.apache.hadoop.hbase.regionserver.compactions;
|
||||
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import org.apache.commons.lang.RandomStringUtils;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileReader;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
/**
|
||||
* Base class of objects that can create mock store files with a given size.
|
||||
*/
|
||||
|
|
|
@ -18,29 +18,28 @@
|
|||
|
||||
package org.apache.hadoop.hbase.regionserver.compactions;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
import static org.mockito.Mockito.when;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
@Category({RegionServerTests.class, MediumTests.class})
|
||||
@RunWith(Parameterized.class)
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.client.Scan;
|
|||
import org.apache.hadoop.hbase.regionserver.CompactedHFilesDischarger;
|
||||
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.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
|
@ -203,13 +204,13 @@ public class TestCompactedHFilesDischarger {
|
|||
int usedReaderCount = 0;
|
||||
int unusedReaderCount = 0;
|
||||
for (StoreFile file : storefiles) {
|
||||
if (file.getRefCount() == 3) {
|
||||
if (((HStoreFile) file).getRefCount() == 3) {
|
||||
usedReaderCount++;
|
||||
}
|
||||
}
|
||||
compactedfiles = ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles();
|
||||
for(StoreFile file : compactedfiles) {
|
||||
assertEquals("Refcount should be 3", 0, file.getRefCount());
|
||||
assertEquals("Refcount should be 3", 0, ((HStoreFile) file).getRefCount());
|
||||
unusedReaderCount++;
|
||||
}
|
||||
// Though there are files we are not using them for reads
|
||||
|
@ -274,14 +275,14 @@ public class TestCompactedHFilesDischarger {
|
|||
int usedReaderCount = 0;
|
||||
int unusedReaderCount = 0;
|
||||
for (StoreFile file : storefiles) {
|
||||
if (file.getRefCount() == 0) {
|
||||
if (((HStoreFile) file).getRefCount() == 0) {
|
||||
unusedReaderCount++;
|
||||
}
|
||||
}
|
||||
compactedfiles =
|
||||
((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles();
|
||||
for(StoreFile file : compactedfiles) {
|
||||
assertEquals("Refcount should be 3", 3, file.getRefCount());
|
||||
assertEquals("Refcount should be 3", 3, ((HStoreFile) file).getRefCount());
|
||||
usedReaderCount++;
|
||||
}
|
||||
// The newly compacted file will not be used by any scanner
|
||||
|
@ -307,13 +308,13 @@ public class TestCompactedHFilesDischarger {
|
|||
usedReaderCount = 0;
|
||||
unusedReaderCount = 0;
|
||||
for (StoreFile file : storefiles) {
|
||||
if (file.getRefCount() == 3) {
|
||||
if (((HStoreFile) file).getRefCount() == 3) {
|
||||
usedReaderCount++;
|
||||
}
|
||||
}
|
||||
compactedfiles = ((HStore) store).getStoreEngine().getStoreFileManager().getCompactedfiles();
|
||||
for(StoreFile file : compactedfiles) {
|
||||
assertEquals("Refcount should be 0", 0, file.getRefCount());
|
||||
assertEquals("Refcount should be 0", 0, ((HStoreFile) file).getRefCount());
|
||||
unusedReaderCount++;
|
||||
}
|
||||
// Though there are files we are not using them for reads
|
||||
|
|
|
@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.regionserver.ScanType;
|
|||
import org.apache.hadoop.hbase.regionserver.Store;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFileScanner;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.Scanner;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.TestCompactor.StoreFileWritersCapture;
|
||||
import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
|
||||
|
@ -103,7 +104,7 @@ public class TestDateTieredCompactor {
|
|||
when(store.createWriterInTmp(anyLong(), any(Compression.Algorithm.class), anyBoolean(),
|
||||
anyBoolean(), anyBoolean(), anyBoolean())).thenAnswer(writers);
|
||||
when(store.getComparator()).thenReturn(CellComparator.COMPARATOR);
|
||||
long maxSequenceId = StoreFile.getMaxSequenceIdInList(storefiles);
|
||||
long maxSequenceId = StoreUtils.getMaxSequenceIdInList(storefiles);
|
||||
when(store.getMaxSequenceId()).thenReturn(maxSequenceId);
|
||||
|
||||
return new DateTieredCompactor(conf, store) {
|
||||
|
|
|
@ -36,12 +36,16 @@ import static org.mockito.Mockito.times;
|
|||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.OptionalLong;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -84,9 +88,6 @@ import org.junit.runners.Parameterized.Parameter;
|
|||
import org.junit.runners.Parameterized.Parameters;
|
||||
import org.mockito.ArgumentMatcher;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
@Category({RegionServerTests.class, SmallTests.class})
|
||||
public class TestStripeCompactionPolicy {
|
||||
|
@ -753,6 +754,7 @@ public class TestStripeCompactionPolicy {
|
|||
when(r.getStoreFileScanner(anyBoolean(), anyBoolean(), anyBoolean(), anyLong(), anyLong(),
|
||||
anyBoolean())).thenReturn(mock(StoreFileScanner.class));
|
||||
when(sf.getReader()).thenReturn(r);
|
||||
when(sf.getBulkLoadTimestamp()).thenReturn(OptionalLong.empty());
|
||||
return sf;
|
||||
}
|
||||
|
||||
|
|
|
@ -27,6 +27,8 @@ import static org.junit.Assert.assertNull;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.ArrayList;
|
||||
|
@ -72,8 +74,6 @@ import org.junit.Rule;
|
|||
import org.junit.Test;
|
||||
import org.junit.rules.TestName;
|
||||
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
/**
|
||||
* Base test class for visibility labels basic features
|
||||
*/
|
||||
|
|
|
@ -18,6 +18,11 @@
|
|||
*/
|
||||
package org.apache.hadoop.hbase.util;
|
||||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -40,11 +45,6 @@ import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
|||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.junit.Assert.assertArrayEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
/**
|
||||
* Utility class for HFile-related testing.
|
||||
*/
|
||||
|
@ -130,8 +130,7 @@ public class HFileTestUtil {
|
|||
writer.append(kv);
|
||||
}
|
||||
} finally {
|
||||
writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY,
|
||||
Bytes.toBytes(System.currentTimeMillis()));
|
||||
writer.appendFileInfo(StoreFile.BULKLOAD_TIME_KEY, Bytes.toBytes(System.currentTimeMillis()));
|
||||
writer.close();
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue