HBASE-20790 Fix the style issues on branch HBASE-19064 before merging back to master

This commit is contained in:
zhangduo 2018-06-27 16:45:53 +08:00
parent a84cdbd579
commit 0789e15b5e
4 changed files with 89 additions and 89 deletions

View File

@ -1594,14 +1594,12 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
@Override
public CompletableFuture<ReplicationPeerConfig> getReplicationPeerConfig(String peerId) {
return this
.<ReplicationPeerConfig> newMasterCaller()
.action(
(controller, stub) -> this
return this.<ReplicationPeerConfig> newMasterCaller().action((controller, stub) -> this
.<GetReplicationPeerConfigRequest, GetReplicationPeerConfigResponse, ReplicationPeerConfig> call(
controller, stub, RequestConverter.buildGetReplicationPeerConfigRequest(peerId), (
s, c, req, done) -> s.getReplicationPeerConfig(c, req, done),
(resp) -> ReplicationPeerConfigUtil.convert(resp.getPeerConfig()))).call();
controller, stub, RequestConverter.buildGetReplicationPeerConfigRequest(peerId),
(s, c, req, done) -> s.getReplicationPeerConfig(c, req, done),
(resp) -> ReplicationPeerConfigUtil.convert(resp.getPeerConfig())))
.call();
}
@Override

View File

@ -344,6 +344,7 @@ public class ReplicationPeerConfig {
return this;
}
@Override
public ReplicationPeerConfigBuilder setRemoteWALDir(String dir) {
this.remoteWALDir = dir;
return this;

View File

@ -2001,27 +2001,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
return false;
}
public boolean compact(CompactionContext compaction, HStore store,
ThroughputController throughputController, User user) throws IOException {
assert compaction != null && compaction.hasSelection();
assert !compaction.getRequest().getFiles().isEmpty();
if (this.closing.get() || this.closed.get()) {
LOG.debug("Skipping compaction on " + this + " because closing/closed");
store.cancelRequestedCompaction(compaction);
return false;
}
if (compaction.getRequest().isAllFiles() && shouldForbidMajorCompaction()) {
LOG.warn("Skipping major compaction on " + this
+ " because this cluster is transiting sync replication state"
+ " from STANDBY to DOWNGRADE_ACTIVE");
store.cancelRequestedCompaction(compaction);
return false;
}
MonitoredTask status = null;
boolean requestNeedsCancellation = true;
/*
/**
* We are trying to remove / relax the region read lock for compaction.
* Let's see what are the potential race conditions among the operations (user scan,
* region split, region close and region bulk load).
@ -2095,6 +2075,26 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* }
* }
*/
public boolean compact(CompactionContext compaction, HStore store,
ThroughputController throughputController, User user) throws IOException {
assert compaction != null && compaction.hasSelection();
assert !compaction.getRequest().getFiles().isEmpty();
if (this.closing.get() || this.closed.get()) {
LOG.debug("Skipping compaction on " + this + " because closing/closed");
store.cancelRequestedCompaction(compaction);
return false;
}
if (compaction.getRequest().isAllFiles() && shouldForbidMajorCompaction()) {
LOG.warn("Skipping major compaction on " + this
+ " because this cluster is transiting sync replication state"
+ " from STANDBY to DOWNGRADE_ACTIVE");
store.cancelRequestedCompaction(compaction);
return false;
}
MonitoredTask status = null;
boolean requestNeedsCancellation = true;
try {
byte[] cf = Bytes.toBytes(store.getColumnFamilyName());
if (stores.get(cf) != store) {

View File

@ -21,10 +21,10 @@ import static org.hamcrest.CoreMatchers.containsString;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.util.Arrays;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.DoNotRetryIOException;
@ -62,6 +62,7 @@ public class TestSyncReplicationStandBy extends SyncReplicationTestBase {
private void assertDisallow(Table table, TableAction action) throws IOException {
try {
action.call(table);
fail("Should not allow the action");
} catch (DoNotRetryIOException | RetriesExhaustedException e) {
// expected
assertThat(e.getMessage(), containsString("STANDBY"));