HBASE-18315 Eliminate the findbugs warnings for hbase-rest

This commit is contained in:
Chia-Ping Tsai 2017-08-08 10:48:04 +08:00
parent 4279f0d2f0
commit 8b6cf09869
11 changed files with 35 additions and 36 deletions

View File

@ -66,6 +66,7 @@ public class MultiRowResource extends ResourceBase implements Constants {
}
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION")
@GET
@Produces({ MIMETYPE_XML, MIMETYPE_JSON, MIMETYPE_PROTOBUF, MIMETYPE_PROTOBUF_IETF })
public Response get(final @Context UriInfo uriInfo) {

View File

@ -19,11 +19,9 @@
package org.apache.hadoop.hbase.rest;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import org.apache.commons.cli.CommandLine;
import org.apache.commons.cli.HelpFormatter;
@ -113,9 +111,6 @@ public class RESTServer implements Constants {
restCSRFEnabled = conf.getBoolean(REST_CSRF_ENABLED_KEY, REST_CSRF_ENABLED_DEFAULT);
if (restCSRFEnabled) {
String[] urls = { "/*" };
Set<String> restCsrfMethodsToIgnore = new HashSet<>();
restCsrfMethodsToIgnore.addAll(getTrimmedStringList(conf,
REST_CSRF_METHODS_TO_IGNORE_KEY, REST_CSRF_METHODS_TO_IGNORE_DEFAULT));
Map<String, String> restCsrfParams = RestCsrfPreventionFilter
.getFilterParams(conf, "hbase.rest-csrf.");
HttpServer.defineFilter(context, "csrf", RestCsrfPreventionFilter.class.getName(),

View File

@ -76,7 +76,7 @@ public class RowResource extends ResourceBase {
this.tableResource = tableResource;
this.rowspec = new RowSpec(rowspec);
if (versions != null) {
this.rowspec.setMaxVersions(Integer.valueOf(versions));
this.rowspec.setMaxVersions(Integer.parseInt(versions));
}
this.check = check;
}
@ -277,7 +277,7 @@ public class RowResource extends ResourceBase {
}
vals = headers.getRequestHeader("X-Timestamp");
if (vals != null && !vals.isEmpty()) {
timestamp = Long.valueOf(vals.get(0));
timestamp = Long.parseLong(vals.get(0));
}
if (column == null) {
servlet.getMetrics().incrementFailedPutRequests(1);

View File

@ -164,7 +164,7 @@ public class RowSpec {
i++;
}
try {
time0 = Long.valueOf(URLDecoder.decode(stamp.toString(),
time0 = Long.parseLong(URLDecoder.decode(stamp.toString(),
HConstants.UTF8_ENCODING));
} catch (NumberFormatException e) {
throw new IllegalArgumentException(e);
@ -177,7 +177,7 @@ public class RowSpec {
i++;
}
try {
time1 = Long.valueOf(URLDecoder.decode(stamp.toString(),
time1 = Long.parseLong(URLDecoder.decode(stamp.toString(),
HConstants.UTF8_ENCODING));
} catch (NumberFormatException e) {
throw new IllegalArgumentException(e);
@ -245,7 +245,7 @@ public class RowSpec {
}
sb.append(c);
}
maxVersions = Integer.valueOf(sb.toString());
maxVersions = Integer.parseInt(sb.toString());
} break;
case 'n': {
StringBuilder sb = new StringBuilder();
@ -257,7 +257,7 @@ public class RowSpec {
}
sb.append(c);
}
maxValues = Integer.valueOf(sb.toString());
maxValues = Integer.parseInt(sb.toString());
} break;
default:
throw new IllegalArgumentException("unknown parameter '" + c + "'");

View File

@ -119,6 +119,7 @@ public class TableResource extends ResourceBase {
return new RowResource(this, suffixglobbingspec, versions, check);
}
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="REC_CATCH_EXCEPTION")
@Path("{scanspec: .*[*]$}")
public TableScanResource getScanResource(
final @Context UriInfo uriInfo,

View File

@ -138,7 +138,7 @@ public class ColumnSchemaModel implements Serializable {
public boolean __getBlockcache() {
Object o = attrs.get(BLOCKCACHE);
return o != null ?
Boolean.valueOf(o.toString()) : HColumnDescriptor.DEFAULT_BLOCKCACHE;
Boolean.parseBoolean(o.toString()) : HColumnDescriptor.DEFAULT_BLOCKCACHE;
}
/**
@ -147,7 +147,7 @@ public class ColumnSchemaModel implements Serializable {
public int __getBlocksize() {
Object o = attrs.get(BLOCKSIZE);
return o != null ?
Integer.valueOf(o.toString()) : HColumnDescriptor.DEFAULT_BLOCKSIZE;
Integer.parseInt(o.toString()) : HColumnDescriptor.DEFAULT_BLOCKSIZE;
}
/**
@ -172,7 +172,7 @@ public class ColumnSchemaModel implements Serializable {
public boolean __getInMemory() {
Object o = attrs.get(IN_MEMORY);
return o != null ?
Boolean.valueOf(o.toString()) : HColumnDescriptor.DEFAULT_IN_MEMORY;
Boolean.parseBoolean(o.toString()) : HColumnDescriptor.DEFAULT_IN_MEMORY;
}
/**
@ -181,7 +181,7 @@ public class ColumnSchemaModel implements Serializable {
public int __getTTL() {
Object o = attrs.get(TTL);
return o != null ?
Integer.valueOf(o.toString()) : HColumnDescriptor.DEFAULT_TTL;
Integer.parseInt(o.toString()) : HColumnDescriptor.DEFAULT_TTL;
}
/**
@ -190,7 +190,7 @@ public class ColumnSchemaModel implements Serializable {
public int __getVersions() {
Object o = attrs.get(VERSIONS);
return o != null ?
Integer.valueOf(o.toString()) : HColumnDescriptor.DEFAULT_VERSIONS;
Integer.parseInt(o.toString()) : HColumnDescriptor.DEFAULT_VERSIONS;
}
/**

View File

@ -129,11 +129,11 @@ public class NamespacesInstanceModel implements Serializable, ProtobufMessageHan
sb.append(namespaceName);
sb.append("\'");
if(properties != null){
for(String key: properties.keySet()){
for (Map.Entry<String, String> entry : properties.entrySet()) {
sb.append(", ");
sb.append(key);
sb.append(entry.getKey());
sb.append(" => '");
sb.append(properties.get(key));
sb.append(entry.getValue());
sb.append("\'");
}
}
@ -145,10 +145,11 @@ public class NamespacesInstanceModel implements Serializable, ProtobufMessageHan
public byte[] createProtobufOutput() {
NamespaceProperties.Builder builder = NamespaceProperties.newBuilder();
if(properties != null){
for(String key: properties.keySet()){
for (Map.Entry<String, String> entry : properties.entrySet()) {
String key = entry.getKey();
NamespaceProperties.Property.Builder property = NamespaceProperties.Property.newBuilder();
property.setKey(key);
property.setValue(properties.get(key));
property.setValue(entry.getValue());
builder.addProps(property);
}
}

View File

@ -414,7 +414,7 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
}
} break;
case PageFilter:
filter = new PageFilter(Long.valueOf(value));
filter = new PageFilter(Long.parseLong(value));
break;
case PrefixFilter:
filter = new PrefixFilter(Base64.decode(value));

View File

@ -107,7 +107,8 @@ public class StorageClusterStatusModel
/**
* Represents a region hosted on a region server.
*/
public static class Region {
public static class Region implements Serializable {
private static final long serialVersionUID = -1326683840086398193L;
private byte[] name;
private int stores;
private int storefiles;

View File

@ -142,7 +142,7 @@ public class TableRegionModel implements Serializable {
this.startKey = Bytes.toBytes(split[1]);
String tail = split[2];
split = tail.split("\\.");
id = Long.valueOf(split[0]);
id = Long.parseLong(split[0]);
}
/**

View File

@ -223,7 +223,7 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
*/
public boolean __getIsMeta() {
Object o = attrs.get(IS_META);
return o != null ? Boolean.valueOf(o.toString()) : false;
return o != null ? Boolean.parseBoolean(o.toString()) : false;
}
/**
@ -231,7 +231,7 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
*/
public boolean __getIsRoot() {
Object o = attrs.get(IS_ROOT);
return o != null ? Boolean.valueOf(o.toString()) : false;
return o != null ? Boolean.parseBoolean(o.toString()) : false;
}
/**
@ -240,7 +240,7 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
public boolean __getReadOnly() {
Object o = attrs.get(READONLY);
return o != null ?
Boolean.valueOf(o.toString()) : HTableDescriptor.DEFAULT_READONLY;
Boolean.parseBoolean(o.toString()) : HTableDescriptor.DEFAULT_READONLY;
}
/**
@ -288,11 +288,11 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
}
if (familyAttrs.containsKey(TTL)) {
familyBuilder.setTtl(
Integer.valueOf(familyAttrs.get(TTL).toString()));
Integer.parseInt(familyAttrs.get(TTL).toString()));
}
if (familyAttrs.containsKey(VERSIONS)) {
familyBuilder.setMaxVersions(
Integer.valueOf(familyAttrs.get(VERSIONS).toString()));
Integer.parseInt(familyAttrs.get(VERSIONS).toString()));
}
if (familyAttrs.containsKey(COMPRESSION)) {
familyBuilder.setCompression(familyAttrs.get(COMPRESSION).toString());
@ -301,7 +301,7 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
}
if (attrs.containsKey(READONLY)) {
builder.setReadOnly(
Boolean.valueOf(attrs.get(READONLY).toString()));
Boolean.parseBoolean(attrs.get(READONLY).toString()));
}
return builder.build().toByteArray();
}