Skip to content

HBASE-23968 Periodically check whether a system stop is requested in compaction by time. #1274

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 3 commits into from
May 4, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
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.regionserver.compactions.CloseChecker;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil;
Expand Down Expand Up @@ -260,7 +261,6 @@ private void calculateMobLengthMap(List<Path> mobFiles) throws IOException {
protected boolean performCompaction(FileDetails fd, InternalScanner scanner, CellSink writer,
long smallestReadPoint, boolean cleanSeqId, ThroughputController throughputController,
boolean major, int numofFilesToCompact) throws IOException {
long bytesWrittenProgressForCloseCheck = 0;
long bytesWrittenProgressForLog = 0;
long bytesWrittenProgressForShippedCall = 0;
// Clear old mob references
Expand All @@ -284,11 +284,12 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
// we have to use a do/while loop.
List<Cell> cells = new ArrayList<>();
// Limit to "hbase.hstore.compaction.kv.max" (default 10) to avoid OOME
int closeCheckSizeLimit = HStore.getCloseCheckInterval();
long currentTime = EnvironmentEdgeManager.currentTime();
long lastMillis = 0;
if (LOG.isDebugEnabled()) {
lastMillis = EnvironmentEdgeManager.currentTime();
lastMillis = currentTime;
}
CloseChecker closeChecker = new CloseChecker(conf, currentTime);
String compactionName = ThroughputControlUtil.getNameForThrottling(store, "compaction");
long now = 0;
boolean hasMore;
Expand Down Expand Up @@ -317,7 +318,14 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel

do {
hasMore = scanner.next(cells, scannerContext);
now = EnvironmentEdgeManager.currentTime();
currentTime = EnvironmentEdgeManager.currentTime();
if (LOG.isDebugEnabled()) {
now = currentTime;
}
if (closeChecker.isTimeLimit(store, currentTime)) {
progress.cancel();
return false;
}
for (Cell c : cells) {
if (compactMOBs) {
if (MobUtils.isMobReferenceCell(c)) {
Expand Down Expand Up @@ -481,16 +489,9 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
bytesWrittenProgressForLog += len;
}
throughputController.control(compactionName, len);
// check periodically to see if a system stop is requested
if (closeCheckSizeLimit > 0) {
bytesWrittenProgressForCloseCheck += len;
if (bytesWrittenProgressForCloseCheck > closeCheckSizeLimit) {
bytesWrittenProgressForCloseCheck = 0;
if (!store.areWritesEnabled()) {
progress.cancel();
return false;
}
}
if (closeChecker.isSizeLimit(store, len)) {
progress.cancel();
return false;
}
if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) {
((ShipperListener) writer).beforeShipped();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2210,15 +2210,13 @@ private boolean shouldForbidMajorCompaction() {
* }
* Also in compactor.performCompaction():
* check periodically to see if a system stop is requested
* if (closeCheckInterval > 0) {
* bytesWritten += len;
* if (bytesWritten > closeCheckInterval) {
* bytesWritten = 0;
* if (!store.areWritesEnabled()) {
* progress.cancel();
* return false;
* }
* }
* if (closeChecker != null && closeChecker.isTimeLimit(store, now)) {
* progress.cancel();
* return false;
* }
* if (closeChecker != null && closeChecker.isSizeLimit(store, len)) {
* progress.cancel();
* return false;
* }
*/
public boolean compact(CompactionContext compaction, HStore store,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,8 +154,6 @@ public class HStore implements Store, HeapSize, StoreConfigInformation,
protected CacheConfig cacheConf;
private long lastCompactSize = 0;
volatile boolean forceMajor = false;
/* how many bytes to write between status checks */
static int closeCheckInterval = 0;
private AtomicLong storeSize = new AtomicLong();
private AtomicLong totalUncompressedBytes = new AtomicLong();

Expand Down Expand Up @@ -297,11 +295,6 @@ protected HStore(final HRegion region, final ColumnFamilyDescriptor family,
this.compactionCheckMultiplier = DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER;
}

if (HStore.closeCheckInterval == 0) {
HStore.closeCheckInterval = conf.getInt(
"hbase.hstore.close.check.interval", 10*1000*1000 /* 10 MB */);
}

this.storeEngine = createStoreEngine(this, this.conf, this.comparator);
List<HStoreFile> hStoreFiles = loadStoreFiles(warmup);
// Move the storeSize calculation out of loadStoreFiles() method, because the secondary read
Expand Down Expand Up @@ -490,13 +483,6 @@ public static ChecksumType getChecksumType(Configuration conf) {
}
}

/**
* @return how many bytes to write between status checks
*/
public static int getCloseCheckInterval() {
return closeCheckInterval;
}

@Override
public ColumnFamilyDescriptor getColumnFamilyDescriptor() {
return this.family;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
/**
* 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.compactions;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.regionserver.Store;
import org.apache.yetus.audience.InterfaceAudience;

/**
* Check periodically to see if a system stop is requested
*/
@InterfaceAudience.Private
public class CloseChecker {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does it have to be public? Can it be package local? It is only used in this package?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it would be good to use package local.
However, The package of FaultyMobStoreCompactor is not the same as CloseChecker.
The package of FaultyMobStoreCompactor is org.apache.hadoop.hbase.mob.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

dang.

ok. public for now.

public static final String SIZE_LIMIT_KEY = "hbase.hstore.close.check.interval";
public static final String TIME_LIMIT_KEY = "hbase.hstore.close.check.time.interval";

private final int closeCheckSizeLimit;
private final long closeCheckTimeLimit;

private long bytesWrittenProgressForCloseCheck;
private long lastCloseCheckMillis;

public CloseChecker(Configuration conf, long currentTime) {
this.closeCheckSizeLimit = conf.getInt(SIZE_LIMIT_KEY, 10 * 1000 * 1000 /* 10 MB */);
this.closeCheckTimeLimit = conf.getLong(TIME_LIMIT_KEY, 10 * 1000L /* 10 s */);
this.bytesWrittenProgressForCloseCheck = 0;
this.lastCloseCheckMillis = currentTime;
}

/**
* Check periodically to see if a system stop is requested every written bytes reach size limit.
*
* @return if true, system stop.
*/
public boolean isSizeLimit(Store store, long bytesWritten) {
if (closeCheckSizeLimit <= 0) {
return false;
}

bytesWrittenProgressForCloseCheck += bytesWritten;
if (bytesWrittenProgressForCloseCheck <= closeCheckSizeLimit) {
return false;
}

bytesWrittenProgressForCloseCheck = 0;
return !store.areWritesEnabled();
}

/**
* Check periodically to see if a system stop is requested every time.
*
* @return if true, system stop.
*/
public boolean isTimeLimit(Store store, long now) {
if (closeCheckTimeLimit <= 0) {
return false;
}

final long elapsedMillis = now - lastCloseCheckMillis;
if (elapsedMillis <= closeCheckTimeLimit) {
return false;
}

lastCloseCheckMillis = now;
return !store.areWritesEnabled();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -368,17 +368,17 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
long smallestReadPoint, boolean cleanSeqId, ThroughputController throughputController,
boolean major, int numofFilesToCompact) throws IOException {
assert writer instanceof ShipperListener;
long bytesWrittenProgressForCloseCheck = 0;
long bytesWrittenProgressForLog = 0;
long bytesWrittenProgressForShippedCall = 0;
// Since scanner.next() can return 'false' but still be delivering data,
// we have to use a do/while loop.
List<Cell> cells = new ArrayList<>();
long closeCheckSizeLimit = HStore.getCloseCheckInterval();
long currentTime = EnvironmentEdgeManager.currentTime();
long lastMillis = 0;
if (LOG.isDebugEnabled()) {
lastMillis = EnvironmentEdgeManager.currentTime();
lastMillis = currentTime;
}
CloseChecker closeChecker = new CloseChecker(conf, currentTime);
String compactionName = ThroughputControlUtil.getNameForThrottling(store, "compaction");
long now = 0;
boolean hasMore;
Expand All @@ -392,8 +392,13 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
try {
do {
hasMore = scanner.next(cells, scannerContext);
currentTime = EnvironmentEdgeManager.currentTime();
if (LOG.isDebugEnabled()) {
now = EnvironmentEdgeManager.currentTime();
now = currentTime;
}
if (closeChecker.isTimeLimit(store, currentTime)) {
progress.cancel();
return false;
}
// output to writer:
Cell lastCleanCell = null;
Expand All @@ -416,16 +421,9 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
bytesWrittenProgressForLog += len;
}
throughputController.control(compactionName, len);
// check periodically to see if a system stop is requested
if (closeCheckSizeLimit > 0) {
bytesWrittenProgressForCloseCheck += len;
if (bytesWrittenProgressForCloseCheck > closeCheckSizeLimit) {
bytesWrittenProgressForCloseCheck = 0;
if (!store.areWritesEnabled()) {
progress.cancel();
return false;
}
}
if (closeChecker.isSizeLimit(store, len)) {
progress.cancel();
return false;
}
if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) {
if (lastCleanCell != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@
import org.apache.hadoop.hbase.regionserver.ScannerContext;
import org.apache.hadoop.hbase.regionserver.ShipperListener;
import org.apache.hadoop.hbase.regionserver.StoreFileWriter;
import org.apache.hadoop.hbase.regionserver.compactions.CloseChecker;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputControlUtil;
import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
import org.apache.hadoop.hbase.util.Bytes;
Expand Down Expand Up @@ -93,7 +94,6 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
if (major) {
totalMajorCompactions.incrementAndGet();
}
long bytesWrittenProgressForCloseCheck = 0;
long bytesWrittenProgressForLog = 0;
long bytesWrittenProgressForShippedCall = 0;
// Clear old mob references
Expand All @@ -119,11 +119,12 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
// we have to use a do/while loop.
List<Cell> cells = new ArrayList<>();
// Limit to "hbase.hstore.compaction.kv.max" (default 10) to avoid OOME
int closeCheckSizeLimit = HStore.getCloseCheckInterval();
long currentTime = EnvironmentEdgeManager.currentTime();
long lastMillis = 0;
if (LOG.isDebugEnabled()) {
lastMillis = EnvironmentEdgeManager.currentTime();
lastMillis = currentTime;
}
CloseChecker closeChecker = new CloseChecker(conf, currentTime);
String compactionName = ThroughputControlUtil.getNameForThrottling(store, "compaction");
long now = 0;
boolean hasMore;
Expand Down Expand Up @@ -168,8 +169,13 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
}
do {
hasMore = scanner.next(cells, scannerContext);
currentTime = EnvironmentEdgeManager.currentTime();
if (LOG.isDebugEnabled()) {
now = EnvironmentEdgeManager.currentTime();
now = currentTime;
}
if (closeChecker.isTimeLimit(store, currentTime)) {
progress.cancel();
return false;
}
for (Cell c : cells) {
counter++;
Expand Down Expand Up @@ -291,16 +297,9 @@ protected boolean performCompaction(FileDetails fd, InternalScanner scanner, Cel
bytesWrittenProgressForLog += len;
}
throughputController.control(compactionName, len);
// check periodically to see if a system stop is requested
if (closeCheckSizeLimit > 0) {
bytesWrittenProgressForCloseCheck += len;
if (bytesWrittenProgressForCloseCheck > closeCheckSizeLimit) {
bytesWrittenProgressForCloseCheck = 0;
if (!store.areWritesEnabled()) {
progress.cancel();
return false;
}
}
if (closeChecker.isSizeLimit(store, len)) {
progress.cancel();
return false;
}
if (kvs != null && bytesWrittenProgressForShippedCall > shippedCallSizeLimit) {
((ShipperListener) writer).beforeShipped();
Expand Down
Loading