Revert "HBASE-19835 Use explicit casting to avoid side effects"
This reverts commit f1a81618fd
.
Conflicts:
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java
patch reverted changes that happened in parallel without explanation. see jira.
This commit is contained in:
parent
ad5cd50dfc
commit
216d2d4648
|
@ -47,7 +47,7 @@ import org.slf4j.LoggerFactory;
|
||||||
@InterfaceAudience.Public
|
@InterfaceAudience.Public
|
||||||
public class Append extends Mutation {
|
public class Append extends Mutation {
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(Append.class);
|
private static final Logger LOG = LoggerFactory.getLogger(Append.class);
|
||||||
private static final long HEAP_OVERHEAD = (long)ClassSize.REFERENCE + ClassSize.TIMERANGE;
|
private static final long HEAP_OVERHEAD = ClassSize.REFERENCE + ClassSize.TIMERANGE;
|
||||||
private TimeRange tr = new TimeRange();
|
private TimeRange tr = new TimeRange();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -66,7 +66,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
|
||||||
HeapSize {
|
HeapSize {
|
||||||
public static final long MUTATION_OVERHEAD = ClassSize.align(
|
public static final long MUTATION_OVERHEAD = ClassSize.align(
|
||||||
// This
|
// This
|
||||||
(long)ClassSize.OBJECT +
|
ClassSize.OBJECT +
|
||||||
// row + OperationWithAttributes.attributes
|
// row + OperationWithAttributes.attributes
|
||||||
2 * ClassSize.REFERENCE +
|
2 * ClassSize.REFERENCE +
|
||||||
// Timestamp
|
// Timestamp
|
||||||
|
@ -791,10 +791,10 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
|
||||||
|
|
||||||
private static final class CellWrapper implements ExtendedCell {
|
private static final class CellWrapper implements ExtendedCell {
|
||||||
private static final long FIXED_OVERHEAD = ClassSize.align(
|
private static final long FIXED_OVERHEAD = ClassSize.align(
|
||||||
(long)ClassSize.OBJECT // object header
|
ClassSize.OBJECT // object header
|
||||||
+ KeyValue.TIMESTAMP_SIZE // timestamp
|
+ KeyValue.TIMESTAMP_SIZE // timestamp
|
||||||
+ Bytes.SIZEOF_LONG // sequence id
|
+ Bytes.SIZEOF_LONG // sequence id
|
||||||
+ 1L * ClassSize.REFERENCE); // references to cell
|
+ 1 * ClassSize.REFERENCE); // references to cell
|
||||||
private final Cell cell;
|
private final Cell cell;
|
||||||
private long sequenceId;
|
private long sequenceId;
|
||||||
private long timestamp;
|
private long timestamp;
|
||||||
|
|
|
@ -27,7 +27,7 @@ import org.apache.yetus.audience.InterfaceAudience;
|
||||||
public class IndividualBytesFieldCell implements ExtendedCell {
|
public class IndividualBytesFieldCell implements ExtendedCell {
|
||||||
|
|
||||||
private static final long FIXED_OVERHEAD = ClassSize.align( // do alignment(padding gap)
|
private static final long FIXED_OVERHEAD = ClassSize.align( // do alignment(padding gap)
|
||||||
(long)ClassSize.OBJECT // object header
|
ClassSize.OBJECT // object header
|
||||||
+ KeyValue.TIMESTAMP_TYPE_SIZE // timestamp and type
|
+ KeyValue.TIMESTAMP_TYPE_SIZE // timestamp and type
|
||||||
+ Bytes.SIZEOF_LONG // sequence id
|
+ Bytes.SIZEOF_LONG // sequence id
|
||||||
+ 5 * ClassSize.REFERENCE); // references to all byte arrays: row, family, qualifier, value, tags
|
+ 5 * ClassSize.REFERENCE); // references to all byte arrays: row, family, qualifier, value, tags
|
||||||
|
|
|
@ -1783,7 +1783,7 @@ public final class PrivateCellUtil {
|
||||||
FirstOnRowCell.FIXED_HEAPSIZE
|
FirstOnRowCell.FIXED_HEAPSIZE
|
||||||
+ Bytes.SIZEOF_BYTE // flength
|
+ Bytes.SIZEOF_BYTE // flength
|
||||||
+ Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
|
+ Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
|
||||||
+ (long)ClassSize.REFERENCE * 2; // fArray, qArray
|
+ ClassSize.REFERENCE * 2; // fArray, qArray
|
||||||
private final byte[] fArray;
|
private final byte[] fArray;
|
||||||
private final int foffset;
|
private final int foffset;
|
||||||
private final byte flength;
|
private final byte flength;
|
||||||
|
@ -1944,7 +1944,7 @@ public final class PrivateCellUtil {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class LastOnRowColCell extends LastOnRowCell {
|
private static class LastOnRowColCell extends LastOnRowCell {
|
||||||
private static final long FIXED_OVERHEAD = (long)LastOnRowCell.FIXED_OVERHEAD
|
private static final long FIXED_OVERHEAD = LastOnRowCell.FIXED_OVERHEAD
|
||||||
+ ClassSize.REFERENCE * 2 // fArray and qArray
|
+ ClassSize.REFERENCE * 2 // fArray and qArray
|
||||||
+ Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
|
+ Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
|
||||||
+ Bytes.SIZEOF_BYTE; // flength
|
+ Bytes.SIZEOF_BYTE; // flength
|
||||||
|
|
|
@ -57,7 +57,7 @@ public class CompactionPipeline {
|
||||||
private static final Logger LOG = LoggerFactory.getLogger(CompactionPipeline.class);
|
private static final Logger LOG = LoggerFactory.getLogger(CompactionPipeline.class);
|
||||||
|
|
||||||
public final static long FIXED_OVERHEAD = ClassSize
|
public final static long FIXED_OVERHEAD = ClassSize
|
||||||
.align((long)ClassSize.OBJECT + (3 * ClassSize.REFERENCE) + Bytes.SIZEOF_LONG);
|
.align(ClassSize.OBJECT + (3 * ClassSize.REFERENCE) + Bytes.SIZEOF_LONG);
|
||||||
public final static long DEEP_OVERHEAD = FIXED_OVERHEAD + (2 * ClassSize.LINKEDLIST);
|
public final static long DEEP_OVERHEAD = FIXED_OVERHEAD + (2 * ClassSize.LINKEDLIST);
|
||||||
|
|
||||||
private final RegionServicesForStores region;
|
private final RegionServicesForStores region;
|
||||||
|
|
|
@ -447,7 +447,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
}
|
}
|
||||||
|
|
||||||
static final long HEAP_SIZE = ClassSize.align(
|
static final long HEAP_SIZE = ClassSize.align(
|
||||||
(long)ClassSize.OBJECT + 5 * Bytes.SIZEOF_BOOLEAN);
|
ClassSize.OBJECT + 5 * Bytes.SIZEOF_BOOLEAN);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -7890,7 +7890,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
||||||
}
|
}
|
||||||
|
|
||||||
public static final long FIXED_OVERHEAD = ClassSize.align(
|
public static final long FIXED_OVERHEAD = ClassSize.align(
|
||||||
(long)ClassSize.OBJECT +
|
ClassSize.OBJECT +
|
||||||
ClassSize.ARRAY +
|
ClassSize.ARRAY +
|
||||||
50 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
|
50 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
|
||||||
(14 * Bytes.SIZEOF_LONG) +
|
(14 * Bytes.SIZEOF_LONG) +
|
||||||
|
|
|
@ -42,6 +42,7 @@ import java.util.concurrent.ExecutorCompletionService;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
import java.util.concurrent.ThreadPoolExecutor;
|
import java.util.concurrent.ThreadPoolExecutor;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
import java.util.concurrent.locks.ReentrantLock;
|
import java.util.concurrent.locks.ReentrantLock;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
import java.util.function.Predicate;
|
import java.util.function.Predicate;
|
||||||
|
@ -149,8 +150,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
volatile boolean forceMajor = false;
|
volatile boolean forceMajor = false;
|
||||||
/* how many bytes to write between status checks */
|
/* how many bytes to write between status checks */
|
||||||
static int closeCheckInterval = 0;
|
static int closeCheckInterval = 0;
|
||||||
private volatile long storeSize = 0L;
|
private AtomicLong storeSize = new AtomicLong();
|
||||||
private volatile long totalUncompressedBytes = 0L;
|
private AtomicLong totalUncompressedBytes = new AtomicLong();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* RWLock for store operations.
|
* RWLock for store operations.
|
||||||
|
@ -209,13 +210,13 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
private int compactionCheckMultiplier;
|
private int compactionCheckMultiplier;
|
||||||
protected Encryption.Context cryptoContext = Encryption.Context.NONE;
|
protected Encryption.Context cryptoContext = Encryption.Context.NONE;
|
||||||
|
|
||||||
private volatile long flushedCellsCount = 0;
|
private AtomicLong flushedCellsCount = new AtomicLong();
|
||||||
private volatile long compactedCellsCount = 0;
|
private AtomicLong compactedCellsCount = new AtomicLong();
|
||||||
private volatile long majorCompactedCellsCount = 0;
|
private AtomicLong majorCompactedCellsCount = new AtomicLong();
|
||||||
private volatile long flushedCellsSize = 0;
|
private AtomicLong flushedCellsSize = new AtomicLong();
|
||||||
private volatile long flushedOutputFileSize = 0;
|
private AtomicLong flushedOutputFileSize = new AtomicLong();
|
||||||
private volatile long compactedCellsSize = 0;
|
private AtomicLong compactedCellsSize = new AtomicLong();
|
||||||
private volatile long majorCompactedCellsSize = 0;
|
private AtomicLong majorCompactedCellsSize = new AtomicLong();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Constructor
|
* Constructor
|
||||||
|
@ -544,8 +545,9 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
HStoreFile storeFile = completionService.take().get();
|
HStoreFile storeFile = completionService.take().get();
|
||||||
if (storeFile != null) {
|
if (storeFile != null) {
|
||||||
long length = storeFile.getReader().length();
|
long length = storeFile.getReader().length();
|
||||||
this.storeSize += length;
|
this.storeSize.addAndGet(length);
|
||||||
this.totalUncompressedBytes += storeFile.getReader().getTotalUncompressedBytes();
|
this.totalUncompressedBytes
|
||||||
|
.addAndGet(storeFile.getReader().getTotalUncompressedBytes());
|
||||||
LOG.debug("loaded {}", storeFile);
|
LOG.debug("loaded {}", storeFile);
|
||||||
results.add(storeFile);
|
results.add(storeFile);
|
||||||
}
|
}
|
||||||
|
@ -844,8 +846,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
|
|
||||||
private void bulkLoadHFile(HStoreFile sf) throws IOException {
|
private void bulkLoadHFile(HStoreFile sf) throws IOException {
|
||||||
StoreFileReader r = sf.getReader();
|
StoreFileReader r = sf.getReader();
|
||||||
this.storeSize += r.length();
|
this.storeSize.addAndGet(r.length());
|
||||||
this.totalUncompressedBytes += r.getTotalUncompressedBytes();
|
this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());
|
||||||
|
|
||||||
// Append the new storefile into the list
|
// Append the new storefile into the list
|
||||||
this.lock.writeLock().lock();
|
this.lock.writeLock().lock();
|
||||||
|
@ -1021,8 +1023,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
HStoreFile sf = createStoreFileAndReader(dstPath);
|
HStoreFile sf = createStoreFileAndReader(dstPath);
|
||||||
|
|
||||||
StoreFileReader r = sf.getReader();
|
StoreFileReader r = sf.getReader();
|
||||||
this.storeSize += r.length();
|
this.storeSize.addAndGet(r.length());
|
||||||
this.totalUncompressedBytes += r.getTotalUncompressedBytes();
|
this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());
|
||||||
|
|
||||||
if (LOG.isInfoEnabled()) {
|
if (LOG.isInfoEnabled()) {
|
||||||
LOG.info("Added " + sf + ", entries=" + r.getEntries() +
|
LOG.info("Added " + sf + ", entries=" + r.getEntries() +
|
||||||
|
@ -1373,11 +1375,11 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
writeCompactionWalRecord(filesToCompact, sfs);
|
writeCompactionWalRecord(filesToCompact, sfs);
|
||||||
replaceStoreFiles(filesToCompact, sfs);
|
replaceStoreFiles(filesToCompact, sfs);
|
||||||
if (cr.isMajor()) {
|
if (cr.isMajor()) {
|
||||||
majorCompactedCellsCount += getCompactionProgress().getTotalCompactingKVs();
|
majorCompactedCellsCount.addAndGet(getCompactionProgress().getTotalCompactingKVs());
|
||||||
majorCompactedCellsSize += getCompactionProgress().totalCompactedSize;
|
majorCompactedCellsSize.addAndGet(getCompactionProgress().totalCompactedSize);
|
||||||
} else {
|
} else {
|
||||||
compactedCellsCount += getCompactionProgress().getTotalCompactingKVs();
|
compactedCellsCount.addAndGet(getCompactionProgress().getTotalCompactingKVs());
|
||||||
compactedCellsSize += getCompactionProgress().totalCompactedSize;
|
compactedCellsSize.addAndGet(getCompactionProgress().totalCompactedSize);
|
||||||
}
|
}
|
||||||
long outputBytes = getTotalSize(sfs);
|
long outputBytes = getTotalSize(sfs);
|
||||||
|
|
||||||
|
@ -1449,7 +1451,9 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
this.lock.writeLock().lock();
|
this.lock.writeLock().lock();
|
||||||
try {
|
try {
|
||||||
this.storeEngine.getStoreFileManager().addCompactionResults(compactedFiles, result);
|
this.storeEngine.getStoreFileManager().addCompactionResults(compactedFiles, result);
|
||||||
filesCompacting.removeAll(compactedFiles); // safe bc: lock.writeLock();
|
synchronized (filesCompacting) {
|
||||||
|
filesCompacting.removeAll(compactedFiles);
|
||||||
|
}
|
||||||
} finally {
|
} finally {
|
||||||
this.lock.writeLock().unlock();
|
this.lock.writeLock().unlock();
|
||||||
}
|
}
|
||||||
|
@ -1478,7 +1482,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
message.append("total size for store is ")
|
message.append("total size for store is ")
|
||||||
.append(StringUtils.TraditionalBinaryPrefix.long2String(storeSize, "", 1))
|
.append(StringUtils.TraditionalBinaryPrefix.long2String(storeSize.get(), "", 1))
|
||||||
.append(". This selection was in queue for ")
|
.append(". This selection was in queue for ")
|
||||||
.append(StringUtils.formatTimeDiff(compactionStartTime, cr.getSelectionTime()))
|
.append(StringUtils.formatTimeDiff(compactionStartTime, cr.getSelectionTime()))
|
||||||
.append(", and took ").append(StringUtils.formatTimeDiff(now, compactionStartTime))
|
.append(", and took ").append(StringUtils.formatTimeDiff(now, compactionStartTime))
|
||||||
|
@ -1772,7 +1776,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
completeCompaction(delSfs);
|
completeCompaction(delSfs);
|
||||||
LOG.info("Completed removal of " + delSfs.size() + " unnecessary (expired) file(s) in "
|
LOG.info("Completed removal of " + delSfs.size() + " unnecessary (expired) file(s) in "
|
||||||
+ this + " of " + this.getRegionInfo().getRegionNameAsString()
|
+ this + " of " + this.getRegionInfo().getRegionNameAsString()
|
||||||
+ "; total size for store is " + TraditionalBinaryPrefix.long2String(storeSize, "", 1));
|
+ "; total size for store is "
|
||||||
|
+ TraditionalBinaryPrefix.long2String(storeSize.get(), "", 1));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void cancelRequestedCompaction(CompactionContext compaction) {
|
public void cancelRequestedCompaction(CompactionContext compaction) {
|
||||||
|
@ -1826,16 +1831,16 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
protected void completeCompaction(Collection<HStoreFile> compactedFiles)
|
protected void completeCompaction(Collection<HStoreFile> compactedFiles)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
this.storeSize = 0L;
|
this.storeSize.set(0L);
|
||||||
this.totalUncompressedBytes = 0L;
|
this.totalUncompressedBytes.set(0L);
|
||||||
for (HStoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) {
|
for (HStoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) {
|
||||||
StoreFileReader r = hsf.getReader();
|
StoreFileReader r = hsf.getReader();
|
||||||
if (r == null) {
|
if (r == null) {
|
||||||
LOG.warn("StoreFile {} has a null Reader", hsf);
|
LOG.warn("StoreFile {} has a null Reader", hsf);
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
this.storeSize += r.length();
|
this.storeSize.addAndGet(r.length());
|
||||||
this.totalUncompressedBytes += r.getTotalUncompressedBytes();
|
this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1896,7 +1901,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getSize() {
|
public long getSize() {
|
||||||
return storeSize;
|
return storeSize.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
public void triggerMajorCompaction() {
|
public void triggerMajorCompaction() {
|
||||||
|
@ -2043,7 +2048,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getStoreSizeUncompressed() {
|
public long getStoreSizeUncompressed() {
|
||||||
return this.totalUncompressedBytes;
|
return this.totalUncompressedBytes.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -2235,9 +2240,9 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
committedFiles.add(sf.getPath());
|
committedFiles.add(sf.getPath());
|
||||||
}
|
}
|
||||||
|
|
||||||
HStore.this.flushedCellsCount += cacheFlushCount;
|
HStore.this.flushedCellsCount.addAndGet(cacheFlushCount);
|
||||||
HStore.this.flushedCellsSize += cacheFlushSize;
|
HStore.this.flushedCellsSize.addAndGet(cacheFlushSize);
|
||||||
HStore.this.flushedOutputFileSize += outputFileSize;
|
HStore.this.flushedOutputFileSize.addAndGet(outputFileSize);
|
||||||
|
|
||||||
// Add new file to store files. Clear snapshot too while we have the Store write lock.
|
// Add new file to store files. Clear snapshot too while we have the Store write lock.
|
||||||
return HStore.this.updateStorefiles(storeFiles, snapshot.getId());
|
return HStore.this.updateStorefiles(storeFiles, snapshot.getId());
|
||||||
|
@ -2270,8 +2275,9 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
StoreFileInfo storeFileInfo = fs.getStoreFileInfo(getColumnFamilyName(), file);
|
StoreFileInfo storeFileInfo = fs.getStoreFileInfo(getColumnFamilyName(), file);
|
||||||
HStoreFile storeFile = createStoreFileAndReader(storeFileInfo);
|
HStoreFile storeFile = createStoreFileAndReader(storeFileInfo);
|
||||||
storeFiles.add(storeFile);
|
storeFiles.add(storeFile);
|
||||||
HStore.this.storeSize += storeFile.getReader().length();
|
HStore.this.storeSize.addAndGet(storeFile.getReader().length());
|
||||||
HStore.this.totalUncompressedBytes += storeFile.getReader().getTotalUncompressedBytes();
|
HStore.this.totalUncompressedBytes
|
||||||
|
.addAndGet(storeFile.getReader().getTotalUncompressedBytes());
|
||||||
if (LOG.isInfoEnabled()) {
|
if (LOG.isInfoEnabled()) {
|
||||||
LOG.info("Region: " + HStore.this.getRegionInfo().getEncodedName() +
|
LOG.info("Region: " + HStore.this.getRegionInfo().getEncodedName() +
|
||||||
" added " + storeFile + ", entries=" + storeFile.getReader().getEntries() +
|
" added " + storeFile + ", entries=" + storeFile.getReader().getEntries() +
|
||||||
|
@ -2302,7 +2308,11 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean needsCompaction() {
|
public boolean needsCompaction() {
|
||||||
return this.storeEngine.needsCompaction(this.filesCompacting);
|
List<HStoreFile> filesCompactingClone = null;
|
||||||
|
synchronized (filesCompacting) {
|
||||||
|
filesCompactingClone = Lists.newArrayList(filesCompacting);
|
||||||
|
}
|
||||||
|
return this.storeEngine.needsCompaction(filesCompactingClone);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -2315,7 +2325,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
}
|
}
|
||||||
|
|
||||||
public static final long FIXED_OVERHEAD =
|
public static final long FIXED_OVERHEAD =
|
||||||
ClassSize.align((long)ClassSize.OBJECT + (17 * ClassSize.REFERENCE) + (11 * Bytes.SIZEOF_LONG)
|
ClassSize.align(ClassSize.OBJECT + (26 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_LONG)
|
||||||
+ (5 * Bytes.SIZEOF_INT) + (2 * Bytes.SIZEOF_BOOLEAN));
|
+ (5 * Bytes.SIZEOF_INT) + (2 * Bytes.SIZEOF_BOOLEAN));
|
||||||
|
|
||||||
public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD
|
public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD
|
||||||
|
@ -2354,37 +2364,37 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getFlushedCellsCount() {
|
public long getFlushedCellsCount() {
|
||||||
return flushedCellsCount;
|
return flushedCellsCount.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getFlushedCellsSize() {
|
public long getFlushedCellsSize() {
|
||||||
return flushedCellsSize;
|
return flushedCellsSize.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getFlushedOutputFileSize() {
|
public long getFlushedOutputFileSize() {
|
||||||
return flushedOutputFileSize;
|
return flushedOutputFileSize.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getCompactedCellsCount() {
|
public long getCompactedCellsCount() {
|
||||||
return compactedCellsCount;
|
return compactedCellsCount.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getCompactedCellsSize() {
|
public long getCompactedCellsSize() {
|
||||||
return compactedCellsSize;
|
return compactedCellsSize.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getMajorCompactedCellsCount() {
|
public long getMajorCompactedCellsCount() {
|
||||||
return majorCompactedCellsCount;
|
return majorCompactedCellsCount.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getMajorCompactedCellsSize() {
|
public long getMajorCompactedCellsSize() {
|
||||||
return majorCompactedCellsSize;
|
return majorCompactedCellsSize.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -48,7 +48,7 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
|
||||||
public class MemStoreCompactor {
|
public class MemStoreCompactor {
|
||||||
|
|
||||||
public static final long DEEP_OVERHEAD = ClassSize
|
public static final long DEEP_OVERHEAD = ClassSize
|
||||||
.align((long)ClassSize.OBJECT + 4 * ClassSize.REFERENCE
|
.align(ClassSize.OBJECT + 4 * ClassSize.REFERENCE
|
||||||
// compactingMemStore, versionedList, isInterrupted, strategy (the reference)
|
// compactingMemStore, versionedList, isInterrupted, strategy (the reference)
|
||||||
// "action" is an enum and thus it is a class with static final constants,
|
// "action" is an enum and thus it is a class with static final constants,
|
||||||
// so counting only the size of the reference to it and not the size of the internals
|
// so counting only the size of the reference to it and not the size of the internals
|
||||||
|
|
|
@ -291,7 +291,7 @@ public class MultiVersionConcurrencyControl {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static final long FIXED_SIZE = ClassSize.align(
|
public static final long FIXED_SIZE = ClassSize.align(
|
||||||
(long)ClassSize.OBJECT +
|
ClassSize.OBJECT +
|
||||||
2 * Bytes.SIZEOF_LONG +
|
2 * Bytes.SIZEOF_LONG +
|
||||||
2 * ClassSize.REFERENCE);
|
2 * ClassSize.REFERENCE);
|
||||||
}
|
}
|
||||||
|
|
|
@ -51,7 +51,7 @@ public class ScanInfo {
|
||||||
private final long preadMaxBytes;
|
private final long preadMaxBytes;
|
||||||
private final boolean newVersionBehavior;
|
private final boolean newVersionBehavior;
|
||||||
|
|
||||||
public static final long FIXED_OVERHEAD = ClassSize.align((long)ClassSize.OBJECT
|
public static final long FIXED_OVERHEAD = ClassSize.align(ClassSize.OBJECT
|
||||||
+ (2 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_INT)
|
+ (2 * ClassSize.REFERENCE) + (2 * Bytes.SIZEOF_INT)
|
||||||
+ (4 * Bytes.SIZEOF_LONG) + (4 * Bytes.SIZEOF_BOOLEAN));
|
+ (4 * Bytes.SIZEOF_LONG) + (4 * Bytes.SIZEOF_BOOLEAN));
|
||||||
|
|
||||||
|
|
|
@ -751,7 +751,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
|
||||||
}
|
}
|
||||||
|
|
||||||
public static final long FIXED_OVERHEAD = ClassSize
|
public static final long FIXED_OVERHEAD = ClassSize
|
||||||
.align((long)ClassSize.OBJECT + (5 * ClassSize.REFERENCE) + ClassSize.ATOMIC_INTEGER
|
.align(ClassSize.OBJECT + (5 * ClassSize.REFERENCE) + ClassSize.ATOMIC_INTEGER
|
||||||
+ Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
|
+ Bytes.SIZEOF_INT + (3 * Bytes.SIZEOF_LONG));
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -980,7 +980,7 @@ public class WALSplitter {
|
||||||
internify(entry);
|
internify(entry);
|
||||||
entryBuffer.add(entry);
|
entryBuffer.add(entry);
|
||||||
long incrHeap = entry.getEdit().heapSize() +
|
long incrHeap = entry.getEdit().heapSize() +
|
||||||
ClassSize.align(2L * ClassSize.REFERENCE) + // WALKey pointers
|
ClassSize.align(2 * ClassSize.REFERENCE) + // WALKey pointers
|
||||||
0; // TODO linkedlist entry
|
0; // TODO linkedlist entry
|
||||||
heapInBuffer += incrHeap;
|
heapInBuffer += incrHeap;
|
||||||
return incrHeap;
|
return incrHeap;
|
||||||
|
|
Loading…
Reference in New Issue