HBASE-23661 Reduced number of Checkstyle violations in hbase-rest

Signed-off-by: stack <stack@apache.org>
This commit is contained in:
Jan Hentschel 2020-01-26 13:38:42 +01:00 committed by GitHub
parent 21f2eddd20
commit 4a39f0a2c5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 91 additions and 126 deletions

View File

@ -16,14 +16,13 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.hbase.rest; package org.apache.hadoop.hbase.rest;
import java.io.IOException; import java.io.IOException;
import java.util.Iterator; import java.util.Iterator;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.filter.Filter; import org.apache.hadoop.hbase.filter.Filter;
import org.apache.hadoop.hbase.rest.model.ScannerModel; import org.apache.hadoop.hbase.rest.model.ScannerModel;

View File

@ -16,7 +16,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.hbase.rest; package org.apache.hadoop.hbase.rest;
import java.io.IOException; import java.io.IOException;
@ -25,10 +24,10 @@ import java.util.NoSuchElementException;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.NeedUnmanagedConnectionException; import org.apache.hadoop.hbase.client.NeedUnmanagedConnectionException;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
@ -46,11 +45,10 @@ public class RowResultGenerator extends ResultGenerator {
public RowResultGenerator(final String tableName, final RowSpec rowspec, public RowResultGenerator(final String tableName, final RowSpec rowspec,
final Filter filter, final boolean cacheBlocks) final Filter filter, final boolean cacheBlocks)
throws IllegalArgumentException, IOException { throws IllegalArgumentException, IOException {
Table table = RESTServlet.getInstance().getTable(tableName); try (Table table = RESTServlet.getInstance().getTable(tableName)) {
try {
Get get = new Get(rowspec.getRow()); Get get = new Get(rowspec.getRow());
if (rowspec.hasColumns()) { if (rowspec.hasColumns()) {
for (byte[] col: rowspec.getColumns()) { for (byte[] col : rowspec.getColumns()) {
byte[][] split = KeyValue.parseColumn(col); byte[][] split = KeyValue.parseColumn(col);
if (split.length == 1) { if (split.length == 1) {
get.addFamily(split[0]); get.addFamily(split[0]);
@ -79,8 +77,6 @@ public class RowResultGenerator extends ResultGenerator {
// help to avoid confusion by leaving a record of what happened here in // help to avoid confusion by leaving a record of what happened here in
// the log. // the log.
LOG.warn(StringUtils.stringifyException(e)); LOG.warn(StringUtils.stringifyException(e));
} finally {
table.close();
} }
} }

View File

@ -16,7 +16,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.hbase.rest.client; package org.apache.hadoop.hbase.rest.client;
import java.io.InputStream; import java.io.InputStream;
@ -26,7 +25,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.classification.InterfaceStability;
/** /**
* The HTTP result code, response headers, and body of a HTTP response. * The HTTP result code, response headers, and body of an HTTP response.
*/ */
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Stable @InterfaceStability.Stable
@ -66,7 +65,8 @@ public class Response {
} }
/** /**
* Constructor * Constructor.
*
* @param code the HTTP response code * @param code the HTTP response code
* @param headers headers the HTTP response headers * @param headers headers the HTTP response headers
* @param body the response body, can be null * @param body the response body, can be null
@ -103,7 +103,7 @@ public class Response {
} }
public String getHeader(String key) { public String getHeader(String key) {
for (Header header: headers) { for (Header header : headers) {
if (header.getName().equalsIgnoreCase(key)) { if (header.getName().equalsIgnoreCase(key)) {
return header.getValue(); return header.getValue();
} }

View File

@ -16,7 +16,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.hbase.rest.filter; package org.apache.hadoop.hbase.rest.filter;
import java.io.IOException; import java.io.IOException;
@ -28,8 +27,7 @@ import javax.servlet.http.HttpServletRequest;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
@InterfaceAudience.Private @InterfaceAudience.Private
public class GZIPRequestStream extends ServletInputStream public class GZIPRequestStream extends ServletInputStream {
{
private GZIPInputStream in; private GZIPInputStream in;
public GZIPRequestStream(HttpServletRequest request) throws IOException { public GZIPRequestStream(HttpServletRequest request) throws IOException {

View File

@ -16,7 +16,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.hbase.rest.filter; package org.apache.hadoop.hbase.rest.filter;
import java.io.IOException; import java.io.IOException;
@ -28,8 +27,7 @@ import javax.servlet.http.HttpServletResponse;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
@InterfaceAudience.Private @InterfaceAudience.Private
public class GZIPResponseStream extends ServletOutputStream public class GZIPResponseStream extends ServletOutputStream {
{
private HttpServletResponse response; private HttpServletResponse response;
private GZIPOutputStream out; private GZIPOutputStream out;

View File

@ -16,7 +16,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.hbase.rest.filter; package org.apache.hadoop.hbase.rest.filter;
import java.io.IOException; import java.io.IOException;
@ -35,15 +34,15 @@ import javax.servlet.ServletResponse;
import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse; import javax.servlet.http.HttpServletResponse;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HBaseInterfaceAudience; import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG) @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
public class GzipFilter implements Filter { public class GzipFilter implements Filter {
private Set<String> mimeTypes = new HashSet<String>(); private Set<String> mimeTypes = new HashSet<>();
@Override @Override
public void init(FilterConfig filterConfig) throws ServletException { public void init(FilterConfig filterConfig) {
String s = filterConfig.getInitParameter("mimeTypes"); String s = filterConfig.getInitParameter("mimeTypes");
if (s != null) { if (s != null) {
StringTokenizer tok = new StringTokenizer(s, ",", false); StringTokenizer tok = new StringTokenizer(s, ",", false);
@ -66,11 +65,11 @@ public class GzipFilter implements Filter {
String acceptEncoding = request.getHeader("accept-encoding"); String acceptEncoding = request.getHeader("accept-encoding");
String contentType = request.getHeader("content-type"); String contentType = request.getHeader("content-type");
if ((contentEncoding != null) && if ((contentEncoding != null) &&
(contentEncoding.toLowerCase(Locale.ROOT).indexOf("gzip") > -1)) { (contentEncoding.toLowerCase(Locale.ROOT).contains("gzip"))) {
request = new GZIPRequestWrapper(request); request = new GZIPRequestWrapper(request);
} }
if (((acceptEncoding != null) && if (((acceptEncoding != null) &&
(acceptEncoding.toLowerCase(Locale.ROOT).indexOf("gzip") > -1)) || (acceptEncoding.toLowerCase(Locale.ROOT).contains("gzip"))) ||
((contentType != null) && mimeTypes.contains(contentType))) { ((contentType != null) && mimeTypes.contains(contentType))) {
response = new GZIPResponseWrapper(response); response = new GZIPResponseWrapper(response);
} }
@ -82,5 +81,4 @@ public class GzipFilter implements Filter {
} }
} }
} }
} }

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.rest.filter; package org.apache.hadoop.hbase.rest.filter;
import java.io.IOException; import java.io.IOException;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Map; import java.util.Map;
@ -36,9 +37,9 @@ import javax.servlet.http.HttpServletResponse;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
/** /**
* This filter provides protection against cross site request forgery (CSRF) * This filter provides protection against cross site request forgery (CSRF)
@ -50,9 +51,7 @@ import org.apache.hadoop.conf.Configuration;
@InterfaceAudience.Public @InterfaceAudience.Public
@InterfaceStability.Evolving @InterfaceStability.Evolving
public class RestCsrfPreventionFilter implements Filter { public class RestCsrfPreventionFilter implements Filter {
private static final Log LOG = LogFactory.getLog(RestCsrfPreventionFilter.class);
private static final Log LOG =
LogFactory.getLog(RestCsrfPreventionFilter.class);
public static final String HEADER_USER_AGENT = "User-Agent"; public static final String HEADER_USER_AGENT = "User-Agent";
public static final String BROWSER_USER_AGENT_PARAM = public static final String BROWSER_USER_AGENT_PARAM =
@ -68,7 +67,7 @@ public class RestCsrfPreventionFilter implements Filter {
private Set<Pattern> browserUserAgents; private Set<Pattern> browserUserAgents;
@Override @Override
public void init(FilterConfig filterConfig) throws ServletException { public void init(FilterConfig filterConfig) {
String customHeader = filterConfig.getInitParameter(CUSTOM_HEADER_PARAM); String customHeader = filterConfig.getInitParameter(CUSTOM_HEADER_PARAM);
if (customHeader != null) { if (customHeader != null) {
headerName = customHeader; headerName = customHeader;
@ -93,7 +92,7 @@ public class RestCsrfPreventionFilter implements Filter {
void parseBrowserUserAgents(String userAgents) { void parseBrowserUserAgents(String userAgents) {
String[] agentsArray = userAgents.split(","); String[] agentsArray = userAgents.split(",");
browserUserAgents = new HashSet<Pattern>(); browserUserAgents = new HashSet<>();
for (String patternString : agentsArray) { for (String patternString : agentsArray) {
browserUserAgents.add(Pattern.compile(patternString)); browserUserAgents.add(Pattern.compile(patternString));
} }
@ -101,10 +100,8 @@ public class RestCsrfPreventionFilter implements Filter {
void parseMethodsToIgnore(String mti) { void parseMethodsToIgnore(String mti) {
String[] methods = mti.split(","); String[] methods = mti.split(",");
methodsToIgnore = new HashSet<String>(); methodsToIgnore = new HashSet<>();
for (int i = 0; i < methods.length; i++) { Collections.addAll(methodsToIgnore, methods);
methodsToIgnore.add(methods[i]);
}
} }
/** /**
@ -145,7 +142,6 @@ public class RestCsrfPreventionFilter implements Filter {
* container configuration mechanisms to insert the filter. * container configuration mechanisms to insert the filter.
*/ */
public interface HttpInteraction { public interface HttpInteraction {
/** /**
* Returns the value of a header. * Returns the value of a header.
* *
@ -225,8 +221,7 @@ public class RestCsrfPreventionFilter implements Filter {
* @return mapping of configuration properties to be used for filter * @return mapping of configuration properties to be used for filter
* initialization * initialization
*/ */
public static Map<String, String> getFilterParams(Configuration conf, public static Map<String, String> getFilterParams(Configuration conf, String confPrefix) {
String confPrefix) {
Map<String, String> filterConfigMap = new HashMap<>(); Map<String, String> filterConfigMap = new HashMap<>();
for (Map.Entry<String, String> entry : conf) { for (Map.Entry<String, String> entry : conf) {
String name = entry.getKey(); String name = entry.getKey();
@ -242,9 +237,7 @@ public class RestCsrfPreventionFilter implements Filter {
/** /**
* {@link HttpInteraction} implementation for use in the servlet filter. * {@link HttpInteraction} implementation for use in the servlet filter.
*/ */
private static final class ServletFilterHttpInteraction private static final class ServletFilterHttpInteraction implements HttpInteraction {
implements HttpInteraction {
private final FilterChain chain; private final FilterChain chain;
private final HttpServletRequest httpRequest; private final HttpServletRequest httpRequest;
private final HttpServletResponse httpResponse; private final HttpServletResponse httpResponse;

View File

@ -16,7 +16,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.hbase.rest.model; package org.apache.hadoop.hbase.rest.model;
import java.io.IOException; import java.io.IOException;
@ -29,30 +28,30 @@ import javax.xml.bind.annotation.XmlAccessorType;
import javax.xml.bind.annotation.XmlElement; import javax.xml.bind.annotation.XmlElement;
import javax.xml.bind.annotation.XmlRootElement; import javax.xml.bind.annotation.XmlRootElement;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.rest.ProtobufMessageHandler; import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
import org.apache.hadoop.hbase.rest.protobuf.generated.CellMessage.Cell; import org.apache.hadoop.hbase.rest.protobuf.generated.CellMessage.Cell;
import org.apache.hadoop.hbase.rest.protobuf.generated.CellSetMessage.CellSet; import org.apache.hadoop.hbase.rest.protobuf.generated.CellSetMessage.CellSet;
import org.apache.hadoop.hbase.util.ByteStringer;
/** /**
* Representation of a grouping of cells. May contain cells from more than * Representation of a grouping of cells. May contain cells from more than
* one row. Encapsulates RowModel and CellModel models. * one row. Encapsulates RowModel and CellModel models.
* *
* <pre> * <pre>
* &lt;complexType name="CellSet"&gt; * &lt;complexType name="CellSet"&gt;
* &lt;sequence&gt; * &lt;sequence&gt;
* &lt;element name="row" type="tns:Row" maxOccurs="unbounded" * &lt;element name="row" type="tns:Row" maxOccurs="unbounded"
* minOccurs="1"&gt;&lt;/element&gt; * minOccurs="1"&gt;&lt;/element&gt;
* &lt;/sequence&gt; * &lt;/sequence&gt;
* &lt;/complexType&gt; * &lt;/complexType&gt;
* *
* &lt;complexType name="Row"&gt; * &lt;complexType name="Row"&gt;
* &lt;sequence&gt; * &lt;sequence&gt;
* &lt;element name="key" type="base64Binary"&gt;&lt;/element&gt; * &lt;element name="key" type="base64Binary"&gt;&lt;/element&gt;
* &lt;element name="cell" type="tns:Cell" * &lt;element name="cell" type="tns:Cell"
* maxOccurs="unbounded" minOccurs="1"&gt;&lt;/element&gt; * maxOccurs="unbounded" minOccurs="1"&gt;&lt;/element&gt;
* &lt;/sequence&gt; * &lt;/sequence&gt;
* &lt;/complexType&gt; * &lt;/complexType&gt;
@ -74,19 +73,18 @@ import org.apache.hadoop.hbase.rest.protobuf.generated.CellSetMessage.CellSet;
@XmlAccessorType(XmlAccessType.FIELD) @XmlAccessorType(XmlAccessType.FIELD)
@InterfaceAudience.Private @InterfaceAudience.Private
public class CellSetModel implements Serializable, ProtobufMessageHandler { public class CellSetModel implements Serializable, ProtobufMessageHandler {
private static final long serialVersionUID = 1L; private static final long serialVersionUID = 1L;
@XmlElement(name="Row") @XmlElement(name="Row")
private List<RowModel> rows; private List<RowModel> rows;
/** /**
* Constructor * Constructor
*/ */
public CellSetModel() { public CellSetModel() {
this.rows = new ArrayList<RowModel>(); this.rows = new ArrayList<>();
} }
/** /**
* @param rows the rows * @param rows the rows
*/ */
@ -94,7 +92,7 @@ public class CellSetModel implements Serializable, ProtobufMessageHandler {
super(); super();
this.rows = rows; this.rows = rows;
} }
/** /**
* Add a row to this cell set * Add a row to this cell set
* @param row the row * @param row the row
@ -113,10 +111,10 @@ public class CellSetModel implements Serializable, ProtobufMessageHandler {
@Override @Override
public byte[] createProtobufOutput() { public byte[] createProtobufOutput() {
CellSet.Builder builder = CellSet.newBuilder(); CellSet.Builder builder = CellSet.newBuilder();
for (RowModel row: getRows()) { for (RowModel row : getRows()) {
CellSet.Row.Builder rowBuilder = CellSet.Row.newBuilder(); CellSet.Row.Builder rowBuilder = CellSet.Row.newBuilder();
rowBuilder.setKey(ByteStringer.wrap(row.getKey())); rowBuilder.setKey(ByteStringer.wrap(row.getKey()));
for (CellModel cell: row.getCells()) { for (CellModel cell : row.getCells()) {
Cell.Builder cellBuilder = Cell.newBuilder(); Cell.Builder cellBuilder = Cell.newBuilder();
cellBuilder.setColumn(ByteStringer.wrap(cell.getColumn())); cellBuilder.setColumn(ByteStringer.wrap(cell.getColumn()));
cellBuilder.setData(ByteStringer.wrap(cell.getValue())); cellBuilder.setData(ByteStringer.wrap(cell.getValue()));
@ -135,9 +133,9 @@ public class CellSetModel implements Serializable, ProtobufMessageHandler {
throws IOException { throws IOException {
CellSet.Builder builder = CellSet.newBuilder(); CellSet.Builder builder = CellSet.newBuilder();
ProtobufUtil.mergeFrom(builder, message); ProtobufUtil.mergeFrom(builder, message);
for (CellSet.Row row: builder.getRowsList()) { for (CellSet.Row row : builder.getRowsList()) {
RowModel rowModel = new RowModel(row.getKey().toByteArray()); RowModel rowModel = new RowModel(row.getKey().toByteArray());
for (Cell cell: row.getValuesList()) { for (Cell cell : row.getValuesList()) {
long timestamp = HConstants.LATEST_TIMESTAMP; long timestamp = HConstants.LATEST_TIMESTAMP;
if (cell.hasTimestamp()) { if (cell.hasTimestamp()) {
timestamp = cell.getTimestamp(); timestamp = cell.getTimestamp();

View File

@ -16,7 +16,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.hbase.rest.model; package org.apache.hadoop.hbase.rest.model;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
@ -31,11 +30,11 @@ import javax.xml.bind.annotation.XmlElement;
import javax.xml.bind.annotation.XmlElementWrapper; import javax.xml.bind.annotation.XmlElementWrapper;
import javax.xml.bind.annotation.XmlRootElement; import javax.xml.bind.annotation.XmlRootElement;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.rest.ProtobufMessageHandler; import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
import org.apache.hadoop.hbase.rest.protobuf.generated.StorageClusterStatusMessage.StorageClusterStatus; import org.apache.hadoop.hbase.rest.protobuf.generated.StorageClusterStatusMessage.StorageClusterStatus;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
/** /**
@ -96,8 +95,7 @@ import org.apache.hadoop.hbase.util.Bytes;
*/ */
@XmlRootElement(name="ClusterStatus") @XmlRootElement(name="ClusterStatus")
@InterfaceAudience.Private @InterfaceAudience.Private
public class StorageClusterStatusModel public class StorageClusterStatusModel implements Serializable, ProtobufMessageHandler {
implements Serializable, ProtobufMessageHandler {
private static final long serialVersionUID = 1L; private static final long serialVersionUID = 1L;
/** /**
@ -111,6 +109,7 @@ public class StorageClusterStatusModel
*/ */
public static class Region implements Serializable { public static class Region implements Serializable {
private static final long serialVersionUID = -1326683840086398193L; private static final long serialVersionUID = -1326683840086398193L;
private byte[] name; private byte[] name;
private int stores; private int stores;
private int storefiles; private int storefiles;
@ -374,7 +373,7 @@ public class StorageClusterStatusModel
private long requests; private long requests;
private int heapSizeMB; private int heapSizeMB;
private int maxHeapSizeMB; private int maxHeapSizeMB;
private List<Region> regions = new ArrayList<Region>(); private List<Region> regions = new ArrayList<>();
/** /**
* Add a region name to the list * Add a region name to the list
@ -498,16 +497,15 @@ public class StorageClusterStatusModel
} }
/** /**
* @param requests the number of requests per second processed by the * @param requests the number of requests per second processed by the region server
* region server
*/ */
public void setRequests(long requests) { public void setRequests(long requests) {
this.requests = requests; this.requests = requests;
} }
} }
private List<Node> liveNodes = new ArrayList<Node>(); private List<Node> liveNodes = new ArrayList<>();
private List<String> deadNodes = new ArrayList<String>(); private List<String> deadNodes = new ArrayList<>();
private int regions; private int regions;
private long requests; private long requests;
private double averageLoad; private double averageLoad;
@ -588,8 +586,8 @@ public class StorageClusterStatusModel
} }
/** /**
* @return the total number of requests per second handled by the cluster in * @return the total number of requests per second handled by the cluster in the last reporting
* the last reporting interval * interval
*/ */
@XmlAttribute @XmlAttribute
public long getRequests() { public long getRequests() {
@ -626,8 +624,7 @@ public class StorageClusterStatusModel
} }
/** /**
* @param requests the total number of requests per second handled by the * @param requests the total number of requests per second handled by the cluster
* cluster
*/ */
public void setRequests(int requests) { public void setRequests(int requests) {
this.requests = requests; this.requests = requests;
@ -640,10 +637,6 @@ public class StorageClusterStatusModel
this.averageLoad = averageLoad; this.averageLoad = averageLoad;
} }
/*
* (non-Javadoc)
* @see java.lang.Object#toString()
*/
@Override @Override
public String toString() { public String toString() {
StringBuilder sb = new StringBuilder(); StringBuilder sb = new StringBuilder();
@ -653,7 +646,7 @@ public class StorageClusterStatusModel
if (!liveNodes.isEmpty()) { if (!liveNodes.isEmpty()) {
sb.append(liveNodes.size()); sb.append(liveNodes.size());
sb.append(" live servers\n"); sb.append(" live servers\n");
for (Node node: liveNodes) { for (Node node : liveNodes) {
sb.append(" "); sb.append(" ");
sb.append(node.name); sb.append(node.name);
sb.append(' '); sb.append(' ');
@ -667,7 +660,7 @@ public class StorageClusterStatusModel
sb.append("\n maxHeapSizeMB="); sb.append("\n maxHeapSizeMB=");
sb.append(node.maxHeapSizeMB); sb.append(node.maxHeapSizeMB);
sb.append("\n\n"); sb.append("\n\n");
for (Node.Region region: node.regions) { for (Node.Region region : node.regions) {
sb.append(" "); sb.append(" ");
sb.append(Bytes.toString(region.name)); sb.append(Bytes.toString(region.name));
sb.append("\n stores="); sb.append("\n stores=");
@ -703,7 +696,7 @@ public class StorageClusterStatusModel
sb.append('\n'); sb.append('\n');
sb.append(deadNodes.size()); sb.append(deadNodes.size());
sb.append(" dead servers\n"); sb.append(" dead servers\n");
for (String node: deadNodes) { for (String node : deadNodes) {
sb.append(" "); sb.append(" ");
sb.append(node); sb.append(node);
sb.append('\n'); sb.append('\n');
@ -718,7 +711,7 @@ public class StorageClusterStatusModel
builder.setRegions(regions); builder.setRegions(regions);
builder.setRequests(requests); builder.setRequests(requests);
builder.setAverageLoad(averageLoad); builder.setAverageLoad(averageLoad);
for (Node node: liveNodes) { for (Node node : liveNodes) {
StorageClusterStatus.Node.Builder nodeBuilder = StorageClusterStatus.Node.Builder nodeBuilder =
StorageClusterStatus.Node.newBuilder(); StorageClusterStatus.Node.newBuilder();
nodeBuilder.setName(node.name); nodeBuilder.setName(node.name);
@ -726,7 +719,7 @@ public class StorageClusterStatusModel
nodeBuilder.setRequests(node.requests); nodeBuilder.setRequests(node.requests);
nodeBuilder.setHeapSizeMB(node.heapSizeMB); nodeBuilder.setHeapSizeMB(node.heapSizeMB);
nodeBuilder.setMaxHeapSizeMB(node.maxHeapSizeMB); nodeBuilder.setMaxHeapSizeMB(node.maxHeapSizeMB);
for (Node.Region region: node.regions) { for (Node.Region region : node.regions) {
StorageClusterStatus.Region.Builder regionBuilder = StorageClusterStatus.Region.Builder regionBuilder =
StorageClusterStatus.Region.newBuilder(); StorageClusterStatus.Region.newBuilder();
regionBuilder.setName(ByteStringer.wrap(region.name)); regionBuilder.setName(ByteStringer.wrap(region.name));
@ -746,15 +739,14 @@ public class StorageClusterStatusModel
} }
builder.addLiveNodes(nodeBuilder); builder.addLiveNodes(nodeBuilder);
} }
for (String node: deadNodes) { for (String node : deadNodes) {
builder.addDeadNodes(node); builder.addDeadNodes(node);
} }
return builder.build().toByteArray(); return builder.build().toByteArray();
} }
@Override @Override
public ProtobufMessageHandler getObjectFromMessage(byte[] message) public ProtobufMessageHandler getObjectFromMessage(byte[] message) throws IOException {
throws IOException {
StorageClusterStatus.Builder builder = StorageClusterStatus.newBuilder(); StorageClusterStatus.Builder builder = StorageClusterStatus.newBuilder();
ProtobufUtil.mergeFrom(builder, message); ProtobufUtil.mergeFrom(builder, message);
if (builder.hasRegions()) { if (builder.hasRegions()) {
@ -766,14 +758,14 @@ public class StorageClusterStatusModel
if (builder.hasAverageLoad()) { if (builder.hasAverageLoad()) {
averageLoad = builder.getAverageLoad(); averageLoad = builder.getAverageLoad();
} }
for (StorageClusterStatus.Node node: builder.getLiveNodesList()) { for (StorageClusterStatus.Node node : builder.getLiveNodesList()) {
long startCode = node.hasStartCode() ? node.getStartCode() : -1; long startCode = node.hasStartCode() ? node.getStartCode() : -1;
StorageClusterStatusModel.Node nodeModel = StorageClusterStatusModel.Node nodeModel =
addLiveNode(node.getName(), startCode, node.getHeapSizeMB(), addLiveNode(node.getName(), startCode, node.getHeapSizeMB(),
node.getMaxHeapSizeMB()); node.getMaxHeapSizeMB());
long requests = node.hasRequests() ? node.getRequests() : 0; long requests = node.hasRequests() ? node.getRequests() : 0;
nodeModel.setRequests(requests); nodeModel.setRequests(requests);
for (StorageClusterStatus.Region region: node.getRegionsList()) { for (StorageClusterStatus.Region region : node.getRegionsList()) {
nodeModel.addRegion( nodeModel.addRegion(
region.getName().toByteArray(), region.getName().toByteArray(),
region.getStores(), region.getStores(),
@ -790,7 +782,7 @@ public class StorageClusterStatusModel
region.getCurrentCompactedKVs()); region.getCurrentCompactedKVs());
} }
} }
for (String node: builder.getDeadNodesList()) { for (String node : builder.getDeadNodesList()) {
addDeadNode(node); addDeadNode(node);
} }
return this; return this;

View File

@ -16,7 +16,6 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.hbase.rest.model; package org.apache.hadoop.hbase.rest.model;
import com.fasterxml.jackson.annotation.JsonAnyGetter; import com.fasterxml.jackson.annotation.JsonAnyGetter;
@ -26,8 +25,8 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
import java.io.IOException; import java.io.IOException;
import java.io.Serializable; import java.io.Serializable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -37,11 +36,11 @@ import javax.xml.bind.annotation.XmlElement;
import javax.xml.bind.annotation.XmlRootElement; import javax.xml.bind.annotation.XmlRootElement;
import javax.xml.namespace.QName; import javax.xml.namespace.QName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HColumnDescriptor; import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.rest.ProtobufMessageHandler; import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
@ -51,11 +50,11 @@ import org.apache.hadoop.hbase.util.Bytes;
/** /**
* A representation of HBase table descriptors. * A representation of HBase table descriptors.
* *
* <pre> * <pre>
* &lt;complexType name="TableSchema"&gt; * &lt;complexType name="TableSchema"&gt;
* &lt;sequence&gt; * &lt;sequence&gt;
* &lt;element name="column" type="tns:ColumnSchema" * &lt;element name="column" type="tns:ColumnSchema"
* maxOccurs="unbounded" minOccurs="1"&gt;&lt;/element&gt; * maxOccurs="unbounded" minOccurs="1"&gt;&lt;/element&gt;
* &lt;/sequence&gt; * &lt;/sequence&gt;
* &lt;attribute name="name" type="string"&gt;&lt;/attribute&gt; * &lt;attribute name="name" type="string"&gt;&lt;/attribute&gt;
@ -72,13 +71,13 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
private static final QName READONLY = new QName(HTableDescriptor.READONLY); private static final QName READONLY = new QName(HTableDescriptor.READONLY);
private static final QName TTL = new QName(HColumnDescriptor.TTL); private static final QName TTL = new QName(HColumnDescriptor.TTL);
private static final QName VERSIONS = new QName(HConstants.VERSIONS); private static final QName VERSIONS = new QName(HConstants.VERSIONS);
private static final QName COMPRESSION = private static final QName COMPRESSION =
new QName(HColumnDescriptor.COMPRESSION); new QName(HColumnDescriptor.COMPRESSION);
private String name; private String name;
private Map<QName,Object> attrs = new LinkedHashMap<QName,Object>(); private Map<QName,Object> attrs = new LinkedHashMap<>();
private List<ColumnSchemaModel> columns = new ArrayList<ColumnSchemaModel>(); private List<ColumnSchemaModel> columns = new ArrayList<>();
/** /**
* Default constructor. * Default constructor.
*/ */
@ -90,17 +89,16 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
*/ */
public TableSchemaModel(HTableDescriptor htd) { public TableSchemaModel(HTableDescriptor htd) {
setName(htd.getTableName().getNameAsString()); setName(htd.getTableName().getNameAsString());
for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e : htd.getValues().entrySet()) {
htd.getValues().entrySet()) { addAttribute(Bytes.toString(e.getKey().get()),
addAttribute(Bytes.toString(e.getKey().get()),
Bytes.toString(e.getValue().get())); Bytes.toString(e.getValue().get()));
} }
for (HColumnDescriptor hcd: htd.getFamilies()) { for (HColumnDescriptor hcd : htd.getFamilies()) {
ColumnSchemaModel columnModel = new ColumnSchemaModel(); ColumnSchemaModel columnModel = new ColumnSchemaModel();
columnModel.setName(hcd.getNameAsString()); columnModel.setName(hcd.getNameAsString());
for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e: for (Map.Entry<ImmutableBytesWritable, ImmutableBytesWritable> e :
hcd.getValues().entrySet()) { hcd.getValues().entrySet()) {
columnModel.addAttribute(Bytes.toString(e.getKey().get()), columnModel.addAttribute(Bytes.toString(e.getKey().get()),
Bytes.toString(e.getValue().get())); Bytes.toString(e.getValue().get()));
} }
addColumnFamily(columnModel); addColumnFamily(columnModel);
@ -193,7 +191,7 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
sb.append("{ NAME=> '"); sb.append("{ NAME=> '");
sb.append(name); sb.append(name);
sb.append('\''); sb.append('\'');
for (Map.Entry<QName,Object> e: attrs.entrySet()) { for (Map.Entry<QName,Object> e : attrs.entrySet()) {
sb.append(", "); sb.append(", ");
sb.append(e.getKey().getLocalPart()); sb.append(e.getKey().getLocalPart());
sb.append(" => '"); sb.append(" => '");
@ -224,7 +222,7 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
*/ */
public boolean __getIsMeta() { public boolean __getIsMeta() {
Object o = attrs.get(IS_META); Object o = attrs.get(IS_META);
return o != null ? Boolean.parseBoolean(o.toString()) : false; return o != null && Boolean.parseBoolean(o.toString());
} }
/** /**
@ -232,7 +230,7 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
*/ */
public boolean __getIsRoot() { public boolean __getIsRoot() {
Object o = attrs.get(IS_ROOT); Object o = attrs.get(IS_ROOT);
return o != null ? Boolean.parseBoolean(o.toString()) : false; return o != null && Boolean.parseBoolean(o.toString());
} }
/** /**
@ -240,8 +238,7 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
*/ */
public boolean __getReadOnly() { public boolean __getReadOnly() {
Object o = attrs.get(READONLY); Object o = attrs.get(READONLY);
return o != null ? return o != null ? Boolean.parseBoolean(o.toString()) : HTableDescriptor.DEFAULT_READONLY;
Boolean.parseBoolean(o.toString()) : HTableDescriptor.DEFAULT_READONLY;
} }
/** /**
@ -269,31 +266,29 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
public byte[] createProtobufOutput() { public byte[] createProtobufOutput() {
TableSchema.Builder builder = TableSchema.newBuilder(); TableSchema.Builder builder = TableSchema.newBuilder();
builder.setName(name); builder.setName(name);
for (Map.Entry<QName, Object> e: attrs.entrySet()) { for (Map.Entry<QName, Object> e : attrs.entrySet()) {
TableSchema.Attribute.Builder attrBuilder = TableSchema.Attribute.Builder attrBuilder =
TableSchema.Attribute.newBuilder(); TableSchema.Attribute.newBuilder();
attrBuilder.setName(e.getKey().getLocalPart()); attrBuilder.setName(e.getKey().getLocalPart());
attrBuilder.setValue(e.getValue().toString()); attrBuilder.setValue(e.getValue().toString());
builder.addAttrs(attrBuilder); builder.addAttrs(attrBuilder);
} }
for (ColumnSchemaModel family: columns) { for (ColumnSchemaModel family : columns) {
Map<QName, Object> familyAttrs = family.getAny(); Map<QName, Object> familyAttrs = family.getAny();
ColumnSchema.Builder familyBuilder = ColumnSchema.newBuilder(); ColumnSchema.Builder familyBuilder = ColumnSchema.newBuilder();
familyBuilder.setName(family.getName()); familyBuilder.setName(family.getName());
for (Map.Entry<QName, Object> e: familyAttrs.entrySet()) { for (Map.Entry<QName, Object> e : familyAttrs.entrySet()) {
ColumnSchema.Attribute.Builder attrBuilder = ColumnSchema.Attribute.Builder attrBuilder =
ColumnSchema.Attribute.newBuilder(); ColumnSchema.Attribute.newBuilder();
attrBuilder.setName(e.getKey().getLocalPart()); attrBuilder.setName(e.getKey().getLocalPart());
attrBuilder.setValue(e.getValue().toString()); attrBuilder.setValue(e.getValue().toString());
familyBuilder.addAttrs(attrBuilder); familyBuilder.addAttrs(attrBuilder);
} }
if (familyAttrs.containsKey(TTL)) { if (familyAttrs.containsKey(TTL)) {
familyBuilder.setTtl( familyBuilder.setTtl(Integer.parseInt(familyAttrs.get(TTL).toString()));
Integer.parseInt(familyAttrs.get(TTL).toString()));
} }
if (familyAttrs.containsKey(VERSIONS)) { if (familyAttrs.containsKey(VERSIONS)) {
familyBuilder.setMaxVersions( familyBuilder.setMaxVersions(Integer.parseInt(familyAttrs.get(VERSIONS).toString()));
Integer.parseInt(familyAttrs.get(VERSIONS).toString()));
} }
if (familyAttrs.containsKey(COMPRESSION)) { if (familyAttrs.containsKey(COMPRESSION)) {
familyBuilder.setCompression(familyAttrs.get(COMPRESSION).toString()); familyBuilder.setCompression(familyAttrs.get(COMPRESSION).toString());
@ -301,28 +296,27 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
builder.addColumns(familyBuilder); builder.addColumns(familyBuilder);
} }
if (attrs.containsKey(READONLY)) { if (attrs.containsKey(READONLY)) {
builder.setReadOnly( builder.setReadOnly(Boolean.parseBoolean(attrs.get(READONLY).toString()));
Boolean.parseBoolean(attrs.get(READONLY).toString()));
} }
return builder.build().toByteArray(); return builder.build().toByteArray();
} }
@Override @Override
public ProtobufMessageHandler getObjectFromMessage(byte[] message) public ProtobufMessageHandler getObjectFromMessage(byte[] message)
throws IOException { throws IOException {
TableSchema.Builder builder = TableSchema.newBuilder(); TableSchema.Builder builder = TableSchema.newBuilder();
ProtobufUtil.mergeFrom(builder, message); ProtobufUtil.mergeFrom(builder, message);
this.setName(builder.getName()); this.setName(builder.getName());
for (TableSchema.Attribute attr: builder.getAttrsList()) { for (TableSchema.Attribute attr : builder.getAttrsList()) {
this.addAttribute(attr.getName(), attr.getValue()); this.addAttribute(attr.getName(), attr.getValue());
} }
if (builder.hasReadOnly()) { if (builder.hasReadOnly()) {
this.addAttribute(HTableDescriptor.READONLY, builder.getReadOnly()); this.addAttribute(HTableDescriptor.READONLY, builder.getReadOnly());
} }
for (ColumnSchema family: builder.getColumnsList()) { for (ColumnSchema family : builder.getColumnsList()) {
ColumnSchemaModel familyModel = new ColumnSchemaModel(); ColumnSchemaModel familyModel = new ColumnSchemaModel();
familyModel.setName(family.getName()); familyModel.setName(family.getName());
for (ColumnSchema.Attribute attr: family.getAttrsList()) { for (ColumnSchema.Attribute attr : family.getAttrsList()) {
familyModel.addAttribute(attr.getName(), attr.getValue()); familyModel.addAttribute(attr.getName(), attr.getValue());
} }
if (family.hasTtl()) { if (family.hasTtl()) {
@ -347,17 +341,16 @@ public class TableSchemaModel implements Serializable, ProtobufMessageHandler {
@JsonIgnore @JsonIgnore
public HTableDescriptor getTableDescriptor() { public HTableDescriptor getTableDescriptor() {
HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(getName())); HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(getName()));
for (Map.Entry<QName, Object> e: getAny().entrySet()) { for (Map.Entry<QName, Object> e : getAny().entrySet()) {
htd.setValue(e.getKey().getLocalPart(), e.getValue().toString()); htd.setValue(e.getKey().getLocalPart(), e.getValue().toString());
} }
for (ColumnSchemaModel column: getColumns()) { for (ColumnSchemaModel column : getColumns()) {
HColumnDescriptor hcd = new HColumnDescriptor(column.getName()); HColumnDescriptor hcd = new HColumnDescriptor(column.getName());
for (Map.Entry<QName, Object> e: column.getAny().entrySet()) { for (Map.Entry<QName, Object> e : column.getAny().entrySet()) {
hcd.setValue(e.getKey().getLocalPart(), e.getValue().toString()); hcd.setValue(e.getKey().getLocalPart(), e.getValue().toString());
} }
htd.addFamily(hcd); htd.addFamily(hcd);
} }
return htd; return htd;
} }
} }