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:
Sean Busbey 2018-02-23 16:50:15 -06:00
parent ad5cd50dfc
commit 216d2d4648
12 changed files with 67 additions and 57 deletions

View File

@ -47,7 +47,7 @@ import org.slf4j.LoggerFactory;
@InterfaceAudience.Public
public class Append extends Mutation {
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();
/**

View File

@ -66,7 +66,7 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
HeapSize {
public static final long MUTATION_OVERHEAD = ClassSize.align(
// This
(long)ClassSize.OBJECT +
ClassSize.OBJECT +
// row + OperationWithAttributes.attributes
2 * ClassSize.REFERENCE +
// Timestamp
@ -791,10 +791,10 @@ public abstract class Mutation extends OperationWithAttributes implements Row, C
private static final class CellWrapper implements ExtendedCell {
private static final long FIXED_OVERHEAD = ClassSize.align(
(long)ClassSize.OBJECT // object header
ClassSize.OBJECT // object header
+ KeyValue.TIMESTAMP_SIZE // timestamp
+ Bytes.SIZEOF_LONG // sequence id
+ 1L * ClassSize.REFERENCE); // references to cell
+ 1 * ClassSize.REFERENCE); // references to cell
private final Cell cell;
private long sequenceId;
private long timestamp;

View File

@ -27,7 +27,7 @@ import org.apache.yetus.audience.InterfaceAudience;
public class IndividualBytesFieldCell implements ExtendedCell {
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
+ Bytes.SIZEOF_LONG // sequence id
+ 5 * ClassSize.REFERENCE); // references to all byte arrays: row, family, qualifier, value, tags

View File

@ -1783,7 +1783,7 @@ public final class PrivateCellUtil {
FirstOnRowCell.FIXED_HEAPSIZE
+ Bytes.SIZEOF_BYTE // flength
+ Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
+ (long)ClassSize.REFERENCE * 2; // fArray, qArray
+ ClassSize.REFERENCE * 2; // fArray, qArray
private final byte[] fArray;
private final int foffset;
private final byte flength;
@ -1944,7 +1944,7 @@ public final class PrivateCellUtil {
}
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
+ Bytes.SIZEOF_INT * 3 // foffset, qoffset, qlength
+ Bytes.SIZEOF_BYTE; // flength

View File

@ -57,7 +57,7 @@ public class CompactionPipeline {
private static final Logger LOG = LoggerFactory.getLogger(CompactionPipeline.class);
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);
private final RegionServicesForStores region;

View File

@ -447,7 +447,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
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(
(long)ClassSize.OBJECT +
ClassSize.OBJECT +
ClassSize.ARRAY +
50 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
(14 * Bytes.SIZEOF_LONG) +

View File

@ -42,6 +42,7 @@ import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.Future;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Predicate;
@ -149,8 +150,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
volatile boolean forceMajor = false;
/* how many bytes to write between status checks */
static int closeCheckInterval = 0;
private volatile long storeSize = 0L;
private volatile long totalUncompressedBytes = 0L;
private AtomicLong storeSize = new AtomicLong();
private AtomicLong totalUncompressedBytes = new AtomicLong();
/**
* RWLock for store operations.
@ -209,13 +210,13 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
private int compactionCheckMultiplier;
protected Encryption.Context cryptoContext = Encryption.Context.NONE;
private volatile long flushedCellsCount = 0;
private volatile long compactedCellsCount = 0;
private volatile long majorCompactedCellsCount = 0;
private volatile long flushedCellsSize = 0;
private volatile long flushedOutputFileSize = 0;
private volatile long compactedCellsSize = 0;
private volatile long majorCompactedCellsSize = 0;
private AtomicLong flushedCellsCount = new AtomicLong();
private AtomicLong compactedCellsCount = new AtomicLong();
private AtomicLong majorCompactedCellsCount = new AtomicLong();
private AtomicLong flushedCellsSize = new AtomicLong();
private AtomicLong flushedOutputFileSize = new AtomicLong();
private AtomicLong compactedCellsSize = new AtomicLong();
private AtomicLong majorCompactedCellsSize = new AtomicLong();
/**
* Constructor
@ -544,8 +545,9 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
HStoreFile storeFile = completionService.take().get();
if (storeFile != null) {
long length = storeFile.getReader().length();
this.storeSize += length;
this.totalUncompressedBytes += storeFile.getReader().getTotalUncompressedBytes();
this.storeSize.addAndGet(length);
this.totalUncompressedBytes
.addAndGet(storeFile.getReader().getTotalUncompressedBytes());
LOG.debug("loaded {}", storeFile);
results.add(storeFile);
}
@ -844,8 +846,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
private void bulkLoadHFile(HStoreFile sf) throws IOException {
StoreFileReader r = sf.getReader();
this.storeSize += r.length();
this.totalUncompressedBytes += r.getTotalUncompressedBytes();
this.storeSize.addAndGet(r.length());
this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());
// Append the new storefile into the list
this.lock.writeLock().lock();
@ -1021,8 +1023,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
HStoreFile sf = createStoreFileAndReader(dstPath);
StoreFileReader r = sf.getReader();
this.storeSize += r.length();
this.totalUncompressedBytes += r.getTotalUncompressedBytes();
this.storeSize.addAndGet(r.length());
this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());
if (LOG.isInfoEnabled()) {
LOG.info("Added " + sf + ", entries=" + r.getEntries() +
@ -1373,11 +1375,11 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
writeCompactionWalRecord(filesToCompact, sfs);
replaceStoreFiles(filesToCompact, sfs);
if (cr.isMajor()) {
majorCompactedCellsCount += getCompactionProgress().getTotalCompactingKVs();
majorCompactedCellsSize += getCompactionProgress().totalCompactedSize;
majorCompactedCellsCount.addAndGet(getCompactionProgress().getTotalCompactingKVs());
majorCompactedCellsSize.addAndGet(getCompactionProgress().totalCompactedSize);
} else {
compactedCellsCount += getCompactionProgress().getTotalCompactingKVs();
compactedCellsSize += getCompactionProgress().totalCompactedSize;
compactedCellsCount.addAndGet(getCompactionProgress().getTotalCompactingKVs());
compactedCellsSize.addAndGet(getCompactionProgress().totalCompactedSize);
}
long outputBytes = getTotalSize(sfs);
@ -1449,7 +1451,9 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
this.lock.writeLock().lock();
try {
this.storeEngine.getStoreFileManager().addCompactionResults(compactedFiles, result);
filesCompacting.removeAll(compactedFiles); // safe bc: lock.writeLock();
synchronized (filesCompacting) {
filesCompacting.removeAll(compactedFiles);
}
} finally {
this.lock.writeLock().unlock();
}
@ -1478,7 +1482,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
}
}
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(StringUtils.formatTimeDiff(compactionStartTime, cr.getSelectionTime()))
.append(", and took ").append(StringUtils.formatTimeDiff(now, compactionStartTime))
@ -1772,7 +1776,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
completeCompaction(delSfs);
LOG.info("Completed removal of " + delSfs.size() + " unnecessary (expired) file(s) in "
+ 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) {
@ -1826,16 +1831,16 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
@VisibleForTesting
protected void completeCompaction(Collection<HStoreFile> compactedFiles)
throws IOException {
this.storeSize = 0L;
this.totalUncompressedBytes = 0L;
this.storeSize.set(0L);
this.totalUncompressedBytes.set(0L);
for (HStoreFile hsf : this.storeEngine.getStoreFileManager().getStorefiles()) {
StoreFileReader r = hsf.getReader();
if (r == null) {
LOG.warn("StoreFile {} has a null Reader", hsf);
continue;
}
this.storeSize += r.length();
this.totalUncompressedBytes += r.getTotalUncompressedBytes();
this.storeSize.addAndGet(r.length());
this.totalUncompressedBytes.addAndGet(r.getTotalUncompressedBytes());
}
}
@ -1896,7 +1901,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
@Override
public long getSize() {
return storeSize;
return storeSize.get();
}
public void triggerMajorCompaction() {
@ -2043,7 +2048,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
@Override
public long getStoreSizeUncompressed() {
return this.totalUncompressedBytes;
return this.totalUncompressedBytes.get();
}
@Override
@ -2235,9 +2240,9 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
committedFiles.add(sf.getPath());
}
HStore.this.flushedCellsCount += cacheFlushCount;
HStore.this.flushedCellsSize += cacheFlushSize;
HStore.this.flushedOutputFileSize += outputFileSize;
HStore.this.flushedCellsCount.addAndGet(cacheFlushCount);
HStore.this.flushedCellsSize.addAndGet(cacheFlushSize);
HStore.this.flushedOutputFileSize.addAndGet(outputFileSize);
// Add new file to store files. Clear snapshot too while we have the Store write lock.
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);
HStoreFile storeFile = createStoreFileAndReader(storeFileInfo);
storeFiles.add(storeFile);
HStore.this.storeSize += storeFile.getReader().length();
HStore.this.totalUncompressedBytes += storeFile.getReader().getTotalUncompressedBytes();
HStore.this.storeSize.addAndGet(storeFile.getReader().length());
HStore.this.totalUncompressedBytes
.addAndGet(storeFile.getReader().getTotalUncompressedBytes());
if (LOG.isInfoEnabled()) {
LOG.info("Region: " + HStore.this.getRegionInfo().getEncodedName() +
" added " + storeFile + ", entries=" + storeFile.getReader().getEntries() +
@ -2302,7 +2308,11 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
@Override
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 =
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));
public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD
@ -2354,37 +2364,37 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
@Override
public long getFlushedCellsCount() {
return flushedCellsCount;
return flushedCellsCount.get();
}
@Override
public long getFlushedCellsSize() {
return flushedCellsSize;
return flushedCellsSize.get();
}
@Override
public long getFlushedOutputFileSize() {
return flushedOutputFileSize;
return flushedOutputFileSize.get();
}
@Override
public long getCompactedCellsCount() {
return compactedCellsCount;
return compactedCellsCount.get();
}
@Override
public long getCompactedCellsSize() {
return compactedCellsSize;
return compactedCellsSize.get();
}
@Override
public long getMajorCompactedCellsCount() {
return majorCompactedCellsCount;
return majorCompactedCellsCount.get();
}
@Override
public long getMajorCompactedCellsSize() {
return majorCompactedCellsSize;
return majorCompactedCellsSize.get();
}
/**

View File

@ -48,7 +48,7 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
public class MemStoreCompactor {
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)
// "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

View File

@ -291,7 +291,7 @@ public class MultiVersionConcurrencyControl {
}
public static final long FIXED_SIZE = ClassSize.align(
(long)ClassSize.OBJECT +
ClassSize.OBJECT +
2 * Bytes.SIZEOF_LONG +
2 * ClassSize.REFERENCE);
}

View File

@ -51,7 +51,7 @@ public class ScanInfo {
private final long preadMaxBytes;
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)
+ (4 * Bytes.SIZEOF_LONG) + (4 * Bytes.SIZEOF_BOOLEAN));

View File

@ -751,7 +751,7 @@ public class FSHLog extends AbstractFSWAL<Writer> {
}
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));
/**

View File

@ -980,7 +980,7 @@ public class WALSplitter {
internify(entry);
entryBuffer.add(entry);
long incrHeap = entry.getEdit().heapSize() +
ClassSize.align(2L * ClassSize.REFERENCE) + // WALKey pointers
ClassSize.align(2 * ClassSize.REFERENCE) + // WALKey pointers
0; // TODO linkedlist entry
heapInBuffer += incrHeap;
return incrHeap;