HBASE-18434 Address some alerts raised by lgtm.com

Signed-off-by: Ramkrishna <ramkrishna.s.vasudevan@intel.com>
This commit is contained in:
Malcolm Taylor 2017-07-22 10:40:27 +01:00 committed by Ramkrishna
parent e24f7f4675
commit 7517f93265
10 changed files with 125 additions and 115 deletions

View File

@ -91,7 +91,7 @@ class NettyRpcDuplexHandler extends ChannelDuplexHandler {
: sizeWithoutCellBlock;
ByteBuf buf = ctx.alloc().buffer(sizeWithoutCellBlock + 4);
buf.writeInt(totalSize);
ByteBufOutputStream bbos = new ByteBufOutputStream(buf);
try (ByteBufOutputStream bbos = new ByteBufOutputStream(buf)) {
requestHeader.writeDelimitedTo(bbos);
if (call.param != null) {
call.param.writeDelimitedTo(bbos);
@ -108,6 +108,7 @@ class NettyRpcDuplexHandler extends ChannelDuplexHandler {
ctx.write(buf, promise);
}
}
}
@Override
public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise)

View File

@ -1930,9 +1930,8 @@ public class ZKUtil {
socket.connect(sockAddr, timeout);
socket.setSoTimeout(timeout);
PrintWriter out = new PrintWriter(socket.getOutputStream(), true);
BufferedReader in = new BufferedReader(new InputStreamReader(
socket.getInputStream()));
try (PrintWriter out = new PrintWriter(socket.getOutputStream(), true);
BufferedReader in = new BufferedReader(new InputStreamReader(socket.getInputStream()))) {
out.println("stat");
out.flush();
ArrayList<String> res = new ArrayList<>();
@ -1947,6 +1946,7 @@ public class ZKUtil {
return res.toArray(new String[res.size()]);
}
}
}
private static void logRetrievedMsg(final ZooKeeperWatcher zkw,
final String znode, final byte [] data, final boolean watcherSet) {

View File

@ -315,7 +315,7 @@ public final class BucketAllocator {
this.bucketSizes = bucketSizes == null ? DEFAULT_BUCKET_SIZES : bucketSizes;
Arrays.sort(this.bucketSizes);
this.bigItemSize = Ints.max(this.bucketSizes);
this.bucketCapacity = FEWEST_ITEMS_IN_BUCKET * bigItemSize;
this.bucketCapacity = FEWEST_ITEMS_IN_BUCKET * (long) bigItemSize;
buckets = new Bucket[(int) (availableSpace / bucketCapacity)];
if (buckets.length < this.bucketSizes.length)
throw new BucketAllocatorException("Bucket allocator size too small (" + buckets.length +

View File

@ -54,7 +54,7 @@ public class MasterDumpServlet extends StateDumpServlet {
response.setContentType("text/plain");
OutputStream os = response.getOutputStream();
PrintWriter out = new PrintWriter(os);
try (PrintWriter out = new PrintWriter(os)) {
out.println("Master status for " + master.getServerName()
+ " as of " + new Date());
@ -109,6 +109,7 @@ public class MasterDumpServlet extends StateDumpServlet {
}
out.flush();
}
}
private void dumpRIT(HMaster master, PrintWriter out) {

View File

@ -1291,7 +1291,7 @@ public class MasterRpcServices extends RSRpcServices
throws ServiceException {
try {
master.checkInitialized();
Boolean result = master.getHFileCleaner().runCleaner() && master.getLogCleaner().runCleaner();
boolean result = master.getHFileCleaner().runCleaner() && master.getLogCleaner().runCleaner();
return ResponseConverter.buildRunCleanerChoreResponse(result);
} catch (IOException ioe) {
throw new ServiceException(ioe);

View File

@ -55,7 +55,7 @@ public class RSDumpServlet extends StateDumpServlet {
}
OutputStream os = response.getOutputStream();
PrintWriter out = new PrintWriter(os);
try (PrintWriter out = new PrintWriter(os)) {
out.println("RegionServer status for " + hrs.getServerName()
+ " as of " + new Date());
@ -102,6 +102,7 @@ public class RSDumpServlet extends StateDumpServlet {
out.flush();
}
}
public static void dumpRowLock(HRegionServer hrs, PrintWriter out) {
StringBuilder sb = new StringBuilder();

View File

@ -101,10 +101,9 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
long currentTime = EnvironmentEdgeManager.currentTime();
for(StoreFile sf: files){
// Check MIN_VERSIONS is in HStore removeUnneededFiles
Long maxTs = sf.getReader().getMaxTimestamp();
long maxTs = sf.getReader().getMaxTimestamp();
long maxTtl = storeConfigInfo.getStoreFileTtl();
if(maxTs == null
|| maxTtl == Long.MAX_VALUE
if (maxTtl == Long.MAX_VALUE
|| (currentTime - maxTtl < maxTs)){
continue;
} else{
@ -120,10 +119,9 @@ public class FIFOCompactionPolicy extends ExploringCompactionPolicy {
Collection<StoreFile> expiredStores = new ArrayList<>();
for(StoreFile sf: files){
// Check MIN_VERSIONS is in HStore removeUnneededFiles
Long maxTs = sf.getReader().getMaxTimestamp();
long maxTs = sf.getReader().getMaxTimestamp();
long maxTtl = storeConfigInfo.getStoreFileTtl();
if(maxTs == null
|| maxTtl == Long.MAX_VALUE
if (maxTtl == Long.MAX_VALUE
|| (currentTime - maxTtl < maxTs)){
continue;
} else if(filesCompacting == null || filesCompacting.contains(sf) == false){

View File

@ -427,7 +427,7 @@ public class RegionMover extends AbstractHBaseTool {
moveRegionsPool.shutdown();
long timeoutInSeconds =
regionsToMove.size()
* admin.getConfiguration().getInt(MOVE_WAIT_MAX_KEY, DEFAULT_MOVE_WAIT_MAX);
* admin.getConfiguration().getLong(MOVE_WAIT_MAX_KEY, DEFAULT_MOVE_WAIT_MAX);
try {
if (!moveRegionsPool.awaitTermination(timeoutInSeconds, TimeUnit.SECONDS)) {
moveRegionsPool.shutdownNow();
@ -501,7 +501,7 @@ public class RegionMover extends AbstractHBaseTool {
moveRegionsPool.shutdown();
long timeoutInSeconds =
regionsToMove.size()
* admin.getConfiguration().getInt(MOVE_WAIT_MAX_KEY, DEFAULT_MOVE_WAIT_MAX);
* admin.getConfiguration().getLong(MOVE_WAIT_MAX_KEY, DEFAULT_MOVE_WAIT_MAX);
try {
if (!moveRegionsPool.awaitTermination(timeoutInSeconds, TimeUnit.SECONDS)) {
moveRegionsPool.shutdownNow();

View File

@ -711,7 +711,8 @@ public class WALSplitter {
for (FileStatus status : files) {
String fileName = status.getPath().getName();
try {
Long tmpSeqId = Long.parseLong(fileName.substring(0, fileName.length()
long tmpSeqId =
Long.parseLong(fileName.substring(0, fileName.length()
- SEQUENCE_ID_FILE_SUFFIX_LENGTH));
maxSeqId = Math.max(tmpSeqId, maxSeqId);
} catch (NumberFormatException ex) {

View File

@ -268,4 +268,12 @@ public class TestBucketCache {
TEST_UTIL.cleanupTestDir();
}
@Test
public void testBucketAllocatorLargeBuckets() throws BucketAllocatorException {
long availableSpace = 20 * 1024L * 1024 * 1024;
int[] bucketSizes = new int[] { 1024, 1024 * 1024, 1024 * 1024 * 1024 };
BucketAllocator allocator = new BucketAllocator(availableSpace, bucketSizes);
assertTrue(allocator.getBuckets().length > 0);
}
}