Skip to content

Commit

Permalink
Fix/fix code format (sofastack#12)
Browse files Browse the repository at this point in the history
* (fix) missing format

* (fix) shutdownContinuations is always not null

* (fix) correct SuppressWarnings

* (fix) correct SuppressWarnings

* (fix) improve: try with resource

* (fix) never return null

* (fix) read requestBuilder#fields should be in lock

* (fix) unnecessary SuppressWarnings

* (fix) yd replace with direct object class access

* (fix) remove necessary null check

* (fix) remove necessary null check

* (fix) SuppressWarnings("SameParameterValue")

* (fix) SuppressWarnings("uncheck")

* (fix) SuppressWarnings("SameParameterValue")

* (fix) SuppressWarnings("SameParameterValue")

* (fix) SuppressWarnings("SameParameterValue")

* (fix) remove necessary null check

* (fix) toArray() call style

* (fix) try with resources

* (fix) SuppressWarnings("SameParameterValue")

* (fix) unnecessary String.valueOf()

* (fix) unnecessary String.valueOf()

* (fix) SuppressWarnings("SameParameterValue")

* (fix) necessary generic arg

* (fix) Long -> long

* (fix) clear code with example

* (fix) format
  • Loading branch information
fengjiachun committed Mar 7, 2019
1 parent 616e619 commit 5462f91
Show file tree
Hide file tree
Showing 33 changed files with 80 additions and 90 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1848,7 +1848,7 @@ private void afterShutdown() {
List<Closure> savedDones = null;
writeLock.lock();
try {
if (this.shutdownContinuations != null) {
if (!this.shutdownContinuations.isEmpty()) {
savedDones = new ArrayList<>(this.shutdownContinuations);
}
if (logStorage != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ public void onStartFollowing(LeaderChangeContext ctx) {
LOG.info("onStartFollowing: {}", ctx);
}

@SuppressWarnings("all")
@SuppressWarnings("SameParameterValue")
private void runClosure(Closure done, String methodName) {
done.run(new Status(-1, "%s doesn't implement %s", getClassName(), methodName));
}
Expand All @@ -98,7 +98,7 @@ private String getClassName() {
return this.getClass().getName();
}

@SuppressWarnings("all")
@SuppressWarnings("SameParameterValue")
private void error(String methodName) {
this.error(methodName, "");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -350,18 +350,13 @@ public long getFirstLogIndex() {
@Override
public long getLastLogIndex() {
readLock.lock();
RocksIterator it = null;
try {
it = this.db.newIterator(this.defaultHandle, this.totalOrderReadOptions);
try (final RocksIterator it = this.db.newIterator(this.defaultHandle, this.totalOrderReadOptions)) {
it.seekToLast();
if (it.isValid()) {
return Bits.getLong(it.key(), 0);
}
return 0L;
} finally {
if (it != null) {
it.close();
}
readLock.unlock();
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -271,10 +271,7 @@ boolean filterBeforeCopy(LocalSnapshotWriter writer, SnapshotReader lastSnapshot
final String destPath = writer.getPath() + File.separator + fileName;
FileUtils.deleteQuietly(new File(destPath));
try {
if (Files.createLink(Paths.get(destPath), Paths.get(sourcePath)) == null) {
LOG.error("Fail to link {} to {}", sourcePath, destPath);
continue;
}
Files.createLink(Paths.get(destPath), Paths.get(sourcePath));
} catch (final IOException e) {
LOG.error("Fail to link {} to {}", sourcePath, destPath, e);
continue;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ public class BoltSession implements Session {
private boolean finished;
private ByteBufferCollector destBuf;
private CopyOptions copyOptions;
private GetFileRequest.Builder requestBuilder;
private final GetFileRequest.Builder requestBuilder;
private final CountDownLatch finishLatch;
private OutputStream outputStream;
private final Endpoint endpoint;
Expand All @@ -90,7 +90,6 @@ private class GetFileResponseClosure extends RpcResponseClosureAdapter<GetFileRe
public void run(Status status) {
onRpcReturned(status, getResponse());
}

}

public void setDestPath(String destPath) {
Expand All @@ -99,28 +98,28 @@ public void setDestPath(String destPath) {

@OnlyForTest
GetFileResponseClosure getDone() {
return this.done;
return done;
}

@OnlyForTest
Future<Message> getRpcCall() {
return this.rpcCall;
return rpcCall;
}

@OnlyForTest
ScheduledFuture<?> getTimer() {
return this.timer;
return timer;
}

@Override
public void close() throws IOException {
lock.lock();
this.lock.lock();
try {
if (!this.finished) {
Utils.closeQuietly(this.outputStream);
}
} finally {
lock.unlock();
this.lock.unlock();
}
}

Expand All @@ -138,7 +137,6 @@ public BoltSession(RaftClientService rpcService, TimerManager timerManager, Snap
this.endpoint = ep;
this.st = Status.OK();
this.finishLatch = new CountDownLatch(1);

}

public void setDestBuf(ByteBufferCollector bufRef) {
Expand All @@ -155,7 +153,7 @@ public void setOutputStream(OutputStream out) {

@Override
public void cancel() {
lock.lock();
this.lock.lock();
try {
if (this.finished) {
return;
Expand All @@ -166,15 +164,14 @@ public void cancel() {
if (this.rpcCall != null) {
this.rpcCall.cancel(true);
}
if (st.isOk()) {
st.setError(RaftError.ECANCELED, RaftError.ECANCELED.name());
if (this.st.isOk()) {
this.st.setError(RaftError.ECANCELED, RaftError.ECANCELED.name());
}

this.onFinished();
} finally {
lock.unlock();
this.lock.unlock();
}

}

@Override
Expand All @@ -189,7 +186,7 @@ public Status status() {

private void onFinished() {
if (!this.finished) {
if (outputStream != null) {
if (this.outputStream != null) {
Utils.closeQuietly(this.outputStream);
this.outputStream = null;
}
Expand All @@ -210,17 +207,17 @@ private void onTimer() {
}

void onRpcReturned(Status status, GetFileResponse response) {
lock.lock();
this.lock.lock();
try {
if (this.finished) {
return;
}
if (!status.isOk()) {
// Reset count to make next rpc retry the previous one
requestBuilder.setCount(0);
this.requestBuilder.setCount(0);
if (status.getCode() == RaftError.ECANCELED.getNumber()) {
if (st.isOk()) {
st.setError(status.getCode(), status.getErrorMsg());
if (this.st.isOk()) {
this.st.setError(status.getCode(), status.getErrorMsg());
this.onFinished();
return;
}
Expand All @@ -229,8 +226,8 @@ void onRpcReturned(Status status, GetFileResponse response) {
// Throttled reading failure does not increase _retry_times
if (status.getCode() != RaftError.EAGAIN.getNumber()
&& ++this.retryTimes >= this.copyOptions.getMaxRetry()) {
if (st.isOk()) {
st.setError(status.getCode(), status.getErrorMsg());
if (this.st.isOk()) {
this.st.setError(status.getCode(), status.getErrorMsg());
this.onFinished();
return;
}
Expand All @@ -245,12 +242,12 @@ void onRpcReturned(Status status, GetFileResponse response) {
if (response.hasReadSize() && response.getReadSize() != 0) {
this.requestBuilder.setCount(response.getReadSize());
}
if (outputStream != null) {
if (this.outputStream != null) {
try {
response.getData().writeTo(outputStream);
response.getData().writeTo(this.outputStream);
} catch (final IOException e) {
LOG.error("Fail to write into file {}", this.destPath);
st.setError(RaftError.EIO, RaftError.EIO.name());
this.st.setError(RaftError.EIO, RaftError.EIO.name());
this.onFinished();
return;
}
Expand All @@ -263,21 +260,22 @@ void onRpcReturned(Status status, GetFileResponse response) {
return;
}
} finally {
lock.unlock();
this.lock.unlock();
}
this.sendNextRpc();
sendNextRpc();
}

/**
* Send next RPC request to get a piece of file data.
*/
void sendNextRpc() {
this.timer = null;
final long offset = requestBuilder.getOffset() + requestBuilder.getCount();
final long maxCount = this.destBuf == null ? raftOptions.getMaxByteCountPerRpc() : Integer.MAX_VALUE;
this.requestBuilder = requestBuilder.setOffset(offset).setCount(maxCount).setReadPartly(true);
this.lock.lock();
try {
this.timer = null;
final long offset = this.requestBuilder.getOffset() + this.requestBuilder.getCount();
final long maxCount = this.destBuf == null ? this.raftOptions.getMaxByteCountPerRpc() : Integer.MAX_VALUE;
this.requestBuilder.setOffset(offset).setCount(maxCount).setReadPartly(true);

if (this.finished) {
return;
}
Expand All @@ -298,7 +296,7 @@ void sendNextRpc() {
this.rpcCall = this.rpcService.getFile(endpoint, this.requestBuilder.build(),
this.copyOptions.getTimeoutMs(), done);
} finally {
lock.unlock();
this.lock.unlock();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -572,7 +572,6 @@ void remove() {
}
}

@SuppressWarnings("all")
public boolean compareAndSetState(int expected, int state) {
return STATE_UPDATER.compareAndSet(this, expected, state);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ public class ZeroByteStringHelper {
// Try to get defineAnonymousClass method handle.
try {
final Class<?> clazz = ByteString.class;
final Method method = clazz.getDeclaredMethod("wrap", (new byte[0]).getClass());
final Method method = clazz.getDeclaredMethod("wrap", byte[].class);
if (method != null) {
WRAP_BYTES_HANDLE = MethodHandles.lookup().unreflect(method);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ public static void main(final String[] args) {
.build() //
.start(30, TimeUnit.SECONDS);

Runtime.getRuntime().addShutdownHook(new Thread(() -> node.stop()));
Runtime.getRuntime().addShutdownHook(new Thread(node::stop));
LOG.info("BenchmarkServer start OK, options: {}", opts);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
*/
public class IncrementAndAddClosure implements Closure {

@SuppressWarnings("all")
@SuppressWarnings({ "FieldCanBeLocal", "unused" })
private CounterServer counterServer;
private IncrementAndGetRequest request;
private ValueResponse response;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,6 @@

import java.util.concurrent.CompletableFuture;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import com.alipay.sofa.jraft.rhea.client.FutureHelper;
import com.alipay.sofa.jraft.rhea.client.RheaKVStore;

Expand All @@ -32,8 +29,6 @@
*/
public class DeleteRangeExample {

private static final Logger LOG = LoggerFactory.getLogger(DeleteRangeExample.class);

public static void main(final String[] args) throws Exception {
final Client client = new Client();
client.init();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ public static void main(final String[] args) throws Exception {
System.out.println(opts);
final Node node = new Node(opts);
node.start();
Runtime.getRuntime().addShutdownHook(new Thread(() -> node.stop()));
Runtime.getRuntime().addShutdownHook(new Thread(node::stop));
System.out.println("server1 start OK");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ public static void main(final String[] args) throws Exception {
System.out.println(opts);
final Node node = new Node(opts);
node.start();
Runtime.getRuntime().addShutdownHook(new Thread(() -> node.stop()));
Runtime.getRuntime().addShutdownHook(new Thread(node::stop));
System.out.println("server2 start OK");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ public static void main(final String[] args) throws Exception {
System.out.println(opts);
final Node node = new Node(opts);
node.start();
Runtime.getRuntime().addShutdownHook(new Thread(() -> node.stop()));
Runtime.getRuntime().addShutdownHook(new Thread(node::stop));
System.out.println("server3 start OK");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -587,15 +587,15 @@ private static <T> List<T> requireNonEmpty(final List<T> target, final String me
return target;
}

@SuppressWarnings("all")
@SuppressWarnings("SameParameterValue")
private static int requirePositive(final int value, final String message) {
if (value <= 0) {
throw new InvalidParameterException(message);
}
return value;
}

@SuppressWarnings("all")
@SuppressWarnings("SameParameterValue")
private static long requirePositive(final long value, final String message) {
if (value <= 0) {
throw new InvalidParameterException(message);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -138,7 +138,7 @@ public synchronized boolean init(final RegionEngineOptions opts) {
final ScheduledExecutorService scheduler = this.storeEngine.getMetricsScheduler();
// start raft node metrics reporter
this.regionMetricsReporter = Slf4jReporter.forRegistry(metricRegistry) //
.prefixedWith("region_" + String.valueOf(this.region.getId())) //
.prefixedWith("region_" + this.region.getId()) //
.withLoggingLevel(Slf4jReporter.LoggingLevel.INFO) //
.outputTo(LOG) //
.scheduleOn(scheduler) //
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -557,7 +557,7 @@ private void startMetricReporters(final long metricsReportPeriod) {
}
// start kv store metrics reporter
this.kvMetricsReporter = Slf4jReporter.forRegistry(KVMetrics.metricRegistry()) //
.prefixedWith("store_" + String.valueOf(this.storeId)) //
.prefixedWith("store_" + this.storeId) //
.withLoggingLevel(Slf4jReporter.LoggingLevel.INFO) //
.outputTo(LOG) //
.scheduleOn(this.metricsScheduler) //
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,7 +115,7 @@ private static ExecutorService newPool(final int coreThreads, final int maxThrea
return newPool(coreThreads, maxThreads, name, defaultHandler);
}

@SuppressWarnings("all")
@SuppressWarnings("SameParameterValue")
private static ExecutorService newPool(final int coreThreads, final int maxThreads, final String name,
final BlockingQueue<Runnable> workQueue) {
final RejectedExecutionHandler defaultHandler = new CallerRunsPolicyWithReport(name, name);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -87,13 +87,13 @@
*/
public class RegionRouteTable {

private static final Logger LOG = LoggerFactory.getLogger(RegionRouteTable.class);
private static final Logger LOG = LoggerFactory.getLogger(RegionRouteTable.class);

private static final Comparator<byte[]> keyBytesComparator = BytesUtil.getDefaultByteArrayComparator();
private static final Comparator<byte[]> keyBytesComparator = BytesUtil.getDefaultByteArrayComparator();

private final StampedLock stampedLock = new StampedLock();
private final NavigableMap<byte[], Long> rangeTable = new TreeMap<>(keyBytesComparator);
private final Map<Long, Region> regionTable = Maps.newHashMap();
private final StampedLock stampedLock = new StampedLock();
private final NavigableMap<byte[], Long> rangeTable = new TreeMap<>(keyBytesComparator);
private final Map<Long, Region> regionTable = Maps.newHashMap();

public Region getRegionById(final long regionId) {
final StampedLock stampedLock = this.stampedLock;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -323,7 +323,7 @@ public Endpoint getLuckyPeer(final long regionId, final boolean forceRefresh, fi
for (int i = 0; i < size; i++) {
final PeerId candidate = balancer.select(peerList);
final Endpoint luckyOne = candidate.getEndpoint();
if (unExpect == null || !luckyOne.equals(unExpect)) {
if (!luckyOne.equals(unExpect)) {
return luckyOne;
}
}
Expand Down
Loading

0 comments on commit 5462f91

Please sign in to comment.