Skip to content

Commit

Permalink
Revert "HBASE-19835 Use explicit casting to avoid side effects"
Browse files Browse the repository at this point in the history
This reverts commit f1a8161.

 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.
  • Loading branch information
busbey committed Feb 23, 2018
1 parent ad5cd50 commit 216d2d4
Show file tree
Hide file tree
Showing 12 changed files with 67 additions and 57 deletions.
Expand Up @@ -47,7 +47,7 @@
@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();


/** /**
Expand Down
Expand Up @@ -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
Expand Down Expand Up @@ -791,10 +791,10 @@ Mutation add(Cell cell) throws IOException {


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;
Expand Down
Expand Up @@ -27,7 +27,7 @@
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
Expand Down
Expand Up @@ -1783,7 +1783,7 @@ private static class FirstOnRowColCell extends FirstOnRowCell {
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;
Expand Down Expand Up @@ -1944,7 +1944,7 @@ public Type getType() {
} }


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
Expand Down
Expand Up @@ -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;
Expand Down
Expand Up @@ -447,7 +447,7 @@ void setReadsEnabled(boolean readsEnabled) {
} }


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);
} }


/** /**
Expand Down Expand Up @@ -7890,7 +7890,7 @@ void checkFamily(final byte [] family)
} }


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) +
Expand Down
Expand Up @@ -42,6 +42,7 @@
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;
Expand Down Expand Up @@ -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.
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -544,8 +545,9 @@ private List<HStoreFile> openStoreFiles(Collection<StoreFileInfo> files) throws
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);
} }
Expand Down Expand Up @@ -844,8 +846,8 @@ public void bulkLoadHFile(StoreFileInfo fileInfo) throws IOException {


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();
Expand Down Expand Up @@ -1021,8 +1023,8 @@ private HStoreFile commitFile(Path path, long logCacheFlushId, MonitoredTask sta
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() +
Expand Down Expand Up @@ -1373,11 +1375,11 @@ protected List<HStoreFile> doCompaction(CompactionRequestImpl cr,
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);


Expand Down Expand Up @@ -1449,7 +1451,9 @@ void replaceStoreFiles(Collection<HStoreFile> compactedFiles, Collection<HStoreF
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();
} }
Expand Down Expand Up @@ -1478,7 +1482,7 @@ private void logCompactionEndMessage(
} }
} }
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))
Expand Down Expand Up @@ -1772,7 +1776,8 @@ private void removeUnneededFiles() throws IOException {
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) {
Expand Down Expand Up @@ -1826,16 +1831,16 @@ private void validateStoreFile(Path path) throws IOException {
@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());
} }
} }


Expand Down Expand Up @@ -1896,7 +1901,7 @@ public long getLastCompactSize() {


@Override @Override
public long getSize() { public long getSize() {
return storeSize; return storeSize.get();
} }


public void triggerMajorCompaction() { public void triggerMajorCompaction() {
Expand Down Expand Up @@ -2043,7 +2048,7 @@ public long getNumHFiles() {


@Override @Override
public long getStoreSizeUncompressed() { public long getStoreSizeUncompressed() {
return this.totalUncompressedBytes; return this.totalUncompressedBytes.get();
} }


@Override @Override
Expand Down Expand Up @@ -2235,9 +2240,9 @@ public boolean commit(MonitoredTask status) throws IOException {
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());
Expand Down Expand Up @@ -2270,8 +2275,9 @@ public void replayFlush(List<String> fileNames, boolean dropMemstoreSnapshot)
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() +
Expand Down Expand Up @@ -2302,7 +2308,11 @@ public void abort() throws IOException {


@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);
} }


/** /**
Expand All @@ -2315,7 +2325,7 @@ public CacheConfig getCacheConfig() {
} }


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
Expand Down Expand Up @@ -2354,37 +2364,37 @@ public boolean hasTooManyStoreFiles() {


@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();
} }


/** /**
Expand Down
Expand Up @@ -48,7 +48,7 @@
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
Expand Down
Expand Up @@ -291,7 +291,7 @@ public String toString() {
} }


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);
} }
Expand Up @@ -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));


Expand Down
Expand Up @@ -751,7 +751,7 @@ boolean isLowReplicationRollEnabled() {
} }


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));


/** /**
Expand Down
Expand Up @@ -980,7 +980,7 @@ long appendEntry(Entry entry) {
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;
Expand Down

0 comments on commit 216d2d4

Please sign in to comment.