You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/01/10 22:50:10 UTC

[GitHub] [hbase] ndimiduk opened a new pull request #1020: HBASE-23653 Expose content of meta table in web ui

ndimiduk opened a new pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020
 
 
   Adds a display of the content of 'hbase:meta' to the Master's
   table.jsp, when that table is selected. Supports basic pagination,
   filtering, &c.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366468548
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.StreamSupport;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser {
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  public Results getResults() {
+    final AsyncTable<AdvancedScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME);
+    return new Results(asyncTable.getScanner(buildScan()));
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
+      .append("scanStart", scanStart)
+      .append("scanLimit", scanLimit)
+      .append("scanTable", scanTable)
+      .append("scanRegionState", scanRegionState)
+      .toString();
+  }
+
+  private static String resolveName(final HttpServletRequest request) {
+    return resolveRequestParameter(request, NAME_PARAM);
+  }
+
+  private Integer resolveScanLimit(final HttpServletRequest request) {
+    final String requestValueStr = resolveRequestParameter(request, SCAN_LIMIT_PARAM);
+    if (StringUtils.isBlank(requestValueStr)) {
+      return null;
+    }
+
+    final Integer requestValue = tryParseInt(requestValueStr);
+    if (requestValue == null) {
+      errorMessages.add(buildScanLimitMalformedErrorMessage(requestValueStr));
+      return null;
+    }
+    if (requestValue <= 0) {
+      errorMessages.add(buildScanLimitLTEZero(requestValue));
+      return SCAN_LIMIT_DEFAULT;
+    }
+
+    final int truncatedValue = Math.min(requestValue, SCAN_LIMIT_MAX);
+    if (requestValue != truncatedValue) {
+      errorMessages.add(buildScanLimitExceededErrorMessage(requestValue));
+    }
+    return truncatedValue;
+  }
+
+  private RegionState.State resolveScanRegionState(final HttpServletRequest request) {
+    final String requestValueStr = resolveRequestParameter(request, SCAN_REGION_STATE_PARAM);
+    if (requestValueStr == null) {
+      return null;
+    }
+    final RegionState.State requestValue = tryValueOf(RegionState.State.class, requestValueStr);
+    if (requestValue == null) {
+      errorMessages.add(buildScanRegionStateMalformedErrorMessage(requestValueStr));
+      return null;
+    }
+    return requestValue;
+  }
+
+  private static byte[] resolveScanStart(final HttpServletRequest request) {
+    final String requestValue = resolveRequestParameter(request, SCAN_START_PARAM);
+    if (requestValue == null) {
+      return null;
+    }
+    return Bytes.toBytesBinary(requestValue);
+  }
+
+  private static TableName resolveScanTable(final HttpServletRequest request) {
+    final String requestValue = resolveRequestParameter(request, SCAN_TABLE_PARAM);
+    if (requestValue == null) {
+      return null;
+    }
+    return TableName.valueOf(requestValue);
+  }
+
+  private static String resolveRequestParameter(final HttpServletRequest request,
+    final String param) {
+    if (request == null) {
+      return null;
+    }
+    final String requestValueStrEnc = request.getParameter(param);
+    if (StringUtils.isBlank(requestValueStrEnc)) {
+      return null;
+    }
+    return urlDecode(requestValueStrEnc);
+  }
+
+  private static Filter buildTableFilter(final TableName tableName) {
+    return new PrefixFilter(tableName.toBytes());
+  }
+
+  private static Filter buildScanRegionStateFilter(final RegionState.State state) {
+    return new SingleColumnValueFilter(
+      HConstants.CATALOG_FAMILY,
+      HConstants.STATE_QUALIFIER,
+      CompareOperator.EQUAL,
+      // use the same serialization strategy as found in MetaTableAccessor#addRegionStateToPut
+      Bytes.toBytes(state.name()));
+  }
+
+  private Filter buildScanFilter() {
+    if (scanTable == null && scanRegionState == null) {
+      return null;
+    }
+
+    final List<Filter> filters = new ArrayList<>(2);
+    if (scanTable != null) {
+      filters.add(buildTableFilter(scanTable));
+    }
+    if (scanRegionState != null) {
+      filters.add(buildScanRegionStateFilter(scanRegionState));
+    }
+    if (filters.size() == 1) {
+      return filters.get(0);
+    }
+    return new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
+  }
+
+  private Scan buildScan() {
+    final Scan metaScan = new Scan()
+      .addFamily(HConstants.CATALOG_FAMILY)
+      .readVersions(1)
+      .setLimit((scanLimit != null ? scanLimit : SCAN_LIMIT_DEFAULT) + 1);
+    if (scanStart != null) {
+      metaScan.withStartRow(scanStart, false);
+    }
+    final Filter filter = buildScanFilter();
+    if (filter != null) {
+      metaScan.setFilter(filter);
+    }
+    return metaScan;
+  }
+
+  /**
+   * Adds {@code value} to {@code encoder} under {@code paramName} when {@code value} is non-null.
+   */
+  private <T> void addParam(final QueryStringEncoder encoder, final String paramName,
 
 Review comment:
   Meh, I suppose.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366112963
 
 

 ##########
 File path: hbase-server/src/main/resources/hbase-webapps/master/table.jsp
 ##########
 @@ -127,8 +147,11 @@
       pageTitle = "Table: " + escaped_fqtn;
   }
   pageContext.setAttribute("pageTitle", pageTitle);
-  AsyncConnection connection = ConnectionFactory.createAsyncConnection(master.getConfiguration()).get();
-  AsyncAdmin admin = connection.getAdminBuilder().setOperationTimeout(5, TimeUnit.SECONDS).build();
+  final AsyncConnection connection = master.getAsyncConnection();
 
 Review comment:
   nice..

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366083062
 
 

 ##########
 File path: hbase-server/src/main/resources/hbase-webapps/master/table.jsp
 ##########
 @@ -47,46 +52,63 @@
   import="org.apache.hadoop.hbase.client.RegionReplicaUtil"
   import="org.apache.hadoop.hbase.client.Table"
   import="org.apache.hadoop.hbase.master.HMaster"
-  import="org.apache.hadoop.hbase.master.assignment.RegionStates"
   import="org.apache.hadoop.hbase.master.RegionState"
-  import="org.apache.hadoop.hbase.quotas.QuotaSettingsFactory"
-  import="org.apache.hadoop.hbase.quotas.QuotaTableUtil"
-  import="org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot"
-  import="org.apache.hadoop.hbase.quotas.ThrottleSettings"
-  import="org.apache.hadoop.hbase.util.Bytes"
-  import="org.apache.hadoop.hbase.util.FSUtils"
-  import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator"
-  import="org.apache.hadoop.util.StringUtils"
-  import="org.apache.hbase.thirdparty.com.google.protobuf.ByteString"%>
+  import="org.apache.hadoop.hbase.master.assignment.RegionStates"
+  import="org.apache.hadoop.hbase.master.webapp.LimitIterator"
+  import="org.apache.hadoop.hbase.master.webapp.MetaBrowser"
+  import="org.apache.hadoop.hbase.master.webapp.RegionReplicaInfo"
+  import="org.apache.hadoop.hbase.quotas.QuotaSettingsFactory"%>
+<%@ page import="org.apache.hadoop.hbase.quotas.QuotaTableUtil" %>
+<%@ page import="org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot" %>
+<%@ page import="org.apache.hadoop.hbase.quotas.ThrottleSettings" %>
+<%@ page import="org.apache.hadoop.hbase.util.Bytes" %>
+<%@ page import="org.apache.hadoop.hbase.util.FSUtils" %>
+<%@ page import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator" %>
+<%@ page import="org.apache.hadoop.util.StringUtils" %>
+<%@ page import="org.apache.hbase.thirdparty.com.google.protobuf.ByteString" %>
 <%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos" %>
 <%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos" %>
 <%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas" %>
 <%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota" %>
-<%@ page import="org.apache.hadoop.hbase.ServerMetrics" %>
-<%@ page import="org.apache.hadoop.hbase.RegionMetrics" %>
-<%@ page import="org.apache.hadoop.hbase.Size" %>
-<%@ page import="org.apache.hadoop.hbase.RegionMetricsBuilder" %>
 <%!
   /**
    * @return An empty region load stamped with the passed in <code>regionInfo</code>
    * region name.
    */
-  private RegionMetrics getEmptyRegionMetrics(final RegionInfo regionInfo) {
+  private static RegionMetrics getEmptyRegionMetrics(final RegionInfo regionInfo) {
     return RegionMetricsBuilder.toRegionMetrics(ClusterStatusProtos.RegionLoad.newBuilder().
             setRegionSpecifier(HBaseProtos.RegionSpecifier.newBuilder().
                     setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME).
                     setValue(ByteString.copyFrom(regionInfo.getRegionName())).build()).build());
   }
+
+  /**
+   * Given dicey information that may or not be available in meta, render a link to the region on
+   * its region server.
+   * @return an anchor tag if one can be built, {@code null} otherwise.
+   */
+  private static String buildRegionServerLink(final ServerName serverName, final int rsInfoPort,
+    final RegionInfo regionInfo, final RegionState.State regionState) {
+    if (serverName == null || regionInfo == null) { return null; }
+
+    if (regionState != RegionState.State.OPEN) {
+      // region is assigned to RS, but RS knows nothing of it. don't bother with a link.
+      return serverName.getServerName();
+    }
+
+    final String socketAddress = serverName.getHostname() + ":" + rsInfoPort;
+    final String URI = "//" + socketAddress + "/region.jsp"
+      + "?name=" + regionInfo.getEncodedName();
+    return "<a href=\"" + URI + "\">" + serverName.getServerName() + "</a>";
 
 Review comment:
   Is there a method in the jsp already that makes a line of server name? IIRC?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366040013
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final ExecutorService pool;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.pool = buildThreadPool();
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  @Override
+  public Iterator<RegionReplicaInfo> iterator() {
+    return limitIterator();
+  }
+
+  public LimitIterator<RegionReplicaInfo> limitIterator() {
+    logger.debug("initiating meta scan, {}", this);
+
+    final AsyncTable<ScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME, pool);
+    // TODO: buffering the entire result set seems unnecessary.
 
 Review comment:
   So this gets a bit messy. A `ResultScanner` needs to be closed when the underlying iterator is exhausted. Right now there's no interface for closing that loop. I haven't found a great way to both (1) keep the interface in `table.jsp` limited to an `Iterator<RegionReplicaInfo>` and (2) ensure the scanner resources are cleaned up. The alternative I've thought of is to have `MetaBrowser` return a `ResultScanner` and leave `table.jsp` to do the work of transforming the results and limiting iteration. It seems like too much work left up to the caller.
   
   Suggestions?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366093095
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final ExecutorService pool;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.pool = buildThreadPool();
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  @Override
+  public Iterator<RegionReplicaInfo> iterator() {
+    return limitIterator();
+  }
+
+  public LimitIterator<RegionReplicaInfo> limitIterator() {
+    logger.debug("initiating meta scan, {}", this);
+
+    final AsyncTable<ScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME, pool);
+    // TODO: buffering the entire result set seems unnecessary.
 
 Review comment:
   Okay, I believe I have a solution to this. One sec.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366463677
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/hamcrest/BytesMatchers.java
 ##########
 @@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client.hamcrest;
+
+import static org.hamcrest.core.Is.is;
 
 Review comment:
   _is_ is provided by the existing dependency, `hamcrest-core`. I've added the dependency on `hamcrest-library` for the the collection- and bean-related matchers that it provides. This patch makes use of _contains_ and _hasProperty_, but I look forward to writing more tests that make use of others. It's quite unusual to depend only on `hamcrest-core`; usually `-core` and `-library` come as a pair in user applications. I believe `-core` is stand-alone so that 3rd party extension libraries (such as Spotify's [java-hamcrest](https://github.com/spotify/java-hamcrest) can pull in the minimal set if they so choose.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366019408
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/LimitIterator.java
 ##########
 @@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+
+/**
+ * An {@link Iterator} over {@code delegate} that limits results to the first {@code limit}
 
 Review comment:
   Please read the rest of the comment :)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366047470
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterRule.java
 ##########
 @@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.junit.Rule;
+import org.junit.rules.ExternalResource;
+
+/**
+ * A {@link Rule} that manages an instance of the {@link MiniHBaseCluster}.
+ */
+public class TestClusterRule extends ExternalResource {
+  private final HBaseTestingUtility testingUtility;
+  private final StartMiniClusterOption miniClusterOptions;
+
+  private MiniHBaseCluster miniCluster;
+
+  public TestClusterRule() {
+    this(StartMiniClusterOption.builder().build());
+  }
+
+  public TestClusterRule(final StartMiniClusterOption miniClusterOptions) {
+    this.testingUtility = new HBaseTestingUtility();
+    this.miniClusterOptions = miniClusterOptions;
+  }
+
+  public CompletableFuture<AsyncConnection> createConnection() {
+    if (miniCluster == null) { throw new IllegalStateException("test cluster not initialized"); }
 
 Review comment:
   Yeah but I like them better... reminds me I want to see if check style supports this style and propose the change on dev@.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366082204
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/RegionReplicaInfo.java
 ##########
 @@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A POJO that consolidates the information about a single region replica that's stored in meta.
 
 Review comment:
   How's this differ from a RegionInfo?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366096720
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/RegionReplicaInfo.java
 ##########
 @@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A POJO that consolidates the information about a single region replica that's stored in meta.
 
 Review comment:
   > How's this differ from a RegionInfo?
   
   I guess it's an HRegionLocation + meta table's rowkey + RegionState with best-effort accessors. I don't think subclassing buys us much. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366468396
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.StreamSupport;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser {
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  public Results getResults() {
+    final AsyncTable<AdvancedScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME);
+    return new Results(asyncTable.getScanner(buildScan()));
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
+      .append("scanStart", scanStart)
+      .append("scanLimit", scanLimit)
+      .append("scanTable", scanTable)
+      .append("scanRegionState", scanRegionState)
+      .toString();
+  }
+
+  private static String resolveName(final HttpServletRequest request) {
+    return resolveRequestParameter(request, NAME_PARAM);
+  }
+
+  private Integer resolveScanLimit(final HttpServletRequest request) {
+    final String requestValueStr = resolveRequestParameter(request, SCAN_LIMIT_PARAM);
+    if (StringUtils.isBlank(requestValueStr)) {
+      return null;
 
 Review comment:
   I think you're right. Now that I've implemented a protocol for error handling, I should be able to clean up this case. Let me review all the accessors under the same lens.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#issuecomment-574448926
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 13s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 7 new or modified test files.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 43s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 46s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 20s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   3m  0s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 59s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 49s |  master passed  |
   | +0 :ok: |  spotbugs  |   4m 46s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |  22m 14s |  master passed  |
   | -0 :warning: |  patch  |   5m  7s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 21s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 32s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 19s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 19s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   3m  3s |  root: The patch generated 0 new + 241 unchanged - 1 fixed = 241 total (was 242)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  3s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedjars  |   4m 59s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  17m 38s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | -1 :x: |  javadoc  |   0m 43s |  hbase-server generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   | -1 :x: |  javadoc  |   3m 16s |  root generated 1 new + 3 unchanged - 0 fixed = 4 total (was 3)  |
   | +1 :green_heart: |  findbugs  |  21m 51s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 234m 23s |  root in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   1m 33s |  The patch does not generate ASF License warnings.  |
   |  |   | 349m 39s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1020 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile xml |
   | uname | Linux 7b7cb3d80937 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 08:06:28 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1020/out/precommit/personality/provided.sh |
   | git revision | master / 499ff32f00 |
   | Default Java | 1.8.0_181 |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/4/artifact/out/diff-javadoc-javadoc-hbase-server.txt |
   | javadoc | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/4/artifact/out/diff-javadoc-javadoc-root.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/4/testReport/ |
   | Max. process+thread count | 5162 (vs. ulimit of 10000) |
   | modules | C: hbase-client hbase-server . U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/4/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366098608
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/ClearUserNamespacesAndTablesRule.java
 ##########
 @@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.StringJoiner;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.client.AsyncAdmin;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.junit.Rule;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link Rule} that clears all user tables and namespaces before the test executes.
+ */
+public class ClearUserNamespacesAndTablesRule extends ExternalResource {
+  private static final Logger logger =
+    LoggerFactory.getLogger(ClearUserNamespacesAndTablesRule.class);
+
+  private final Supplier<AsyncConnection> connectionSupplier;
+  private AsyncAdmin admin;
+
+  public ClearUserNamespacesAndTablesRule(final Supplier<AsyncConnection> connectionSupplier) {
+    this.connectionSupplier = connectionSupplier;
+  }
+
+  @Override
+  protected void before() throws Throwable {
+    final AsyncConnection connection = Objects.requireNonNull(connectionSupplier.get());
+    admin = connection.getAdmin();
+
+    clearTablesAndNamespaces().join();
+  }
+
+  private CompletableFuture<Void> clearTablesAndNamespaces() {
+    return deleteUserTables().thenCompose(_void -> deleteUserNamespaces());
+  }
+
+  private CompletableFuture<Void> deleteUserTables() {
+    return listTableNames()
+      .thenApply(tableNames -> tableNames.stream()
+        .map(tableName -> disableIfEnabled(tableName).thenCompose(_void -> deleteTable(tableName)))
+        .toArray(CompletableFuture[]::new))
+      .thenCompose(CompletableFuture::allOf);
+  }
+
+  private CompletableFuture<List<TableName>> listTableNames() {
+    return CompletableFuture.runAsync(() -> logger.info("listing tables"))
+      .thenCompose(_void -> admin.listTableNames(false))
+      .thenApply(tableNames -> {
+        final StringJoiner joiner = new StringJoiner(", ", "[", "]");
+        tableNames.stream().map(TableName::getNameAsString).forEach(joiner::add);
+        logger.info("found existing tables {}", joiner.toString());
+        return tableNames;
+      });
+  }
+
+  private CompletableFuture<Boolean> isTableEnabled(final TableName tableName) {
+    return admin.isTableEnabled(tableName)
+      .thenApply(isEnabled -> {
+        logger.info("table {} is enabled.", tableName);
+        return isEnabled;
+      });
+  }
+
+  private CompletableFuture<Void> disableIfEnabled(final TableName tableName) {
+    return isTableEnabled(tableName)
+      .thenCompose(isEnabled -> {
+        if (isEnabled) { return disableTable(tableName); }
+        return CompletableFuture.completedFuture(null);
+      });
+  }
+
+  private CompletableFuture<Void> disableTable(final TableName tableName) {
+    return CompletableFuture.runAsync(() -> logger.info("disabling enabled table {}", tableName))
+      .thenCompose(_void -> admin.disableTable(tableName));
+  }
+
+  private CompletableFuture<Void> deleteTable(final TableName tableName) {
+    return CompletableFuture.runAsync(() -> logger.info("deleting disabled table {}", tableName))
+      .thenCompose(_void -> admin.deleteTable(tableName));
+  }
+
+  private CompletableFuture<List<String>> listUserNamespaces() {
+    return CompletableFuture.runAsync(() -> logger.info("listing namespaces"))
+      .thenCompose(_void -> admin.listNamespaceDescriptors())
+      .thenApply(namespaceDescriptors -> {
+        final StringJoiner joiner = new StringJoiner(", ", "[", "]");
+        final List<String> names = namespaceDescriptors.stream()
+          .map(NamespaceDescriptor::getName)
+          .peek(joiner::add)
+          .collect(Collectors.toList());
+        logger.info("found existing namespaces {}", joiner.toString());
+        return names;
+      })
+      .thenApply(namespaces -> namespaces.stream()
+        .filter(namespace -> !Objects.equals(
+          namespace, NamespaceDescriptor.SYSTEM_NAMESPACE.getName()))
+        .filter(namespace -> !Objects.equals(
+          namespace, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()))
+        .collect(Collectors.toList()));
+  }
+
+  private CompletableFuture<Void> deleteNamespace(final String namespace) {
+    return CompletableFuture.runAsync(() -> logger.info("deleting namespace {}", namespace))
+      .thenCompose(_void -> admin.deleteNamespace(namespace));
+  }
+
+  private CompletableFuture<Void> deleteUserNamespaces() {
+    return listUserNamespaces()
+      .thenCompose(namespaces -> CompletableFuture.allOf(
+        namespaces.stream()
+          .map(this::deleteNamespace)
+          .toArray(CompletableFuture[]::new)));
+  }
+}
 
 Review comment:
   I'm writing tests over the content of meta. So I can
   1. clean up after each test method; or
   1. tear down and spin up a new mini cluster for each test method; or
   1. create a fixed set of tables `@BeforeClass` and write all test methods around those tables.
   
   I chose the first option because (2) would make for a really slow test run and (3) is very inflexible in terms of how each test method is written, and renders all other test methods brittle to tweaks of any one of them.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366091737
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/LimitIterator.java
 ##########
 @@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+
+/**
+ * An {@link Iterator} over {@code delegate} that limits results to the first {@code limit}
+ * entries.
+ * <p>Could just use {@link Iterators#limit(Iterator, int)} except that our consumer needs an API
+ * to check if the underlying iterator is not yet exhausted.
+ */
+@InterfaceAudience.Private
+public class LimitIterator<T> implements Iterator<T> {
+
+  private final Iterator<T> delegate;
+  private final int limit;
+  private int count;
+
+  LimitIterator(final Iterator<T> delegate, final int limit) {
+    this.delegate = delegate;
+    this.limit = limit;
+    this.count = 0;
+  }
+
+  /**
+   * @return {@code true} when {@code delegate} has more entries, {@code false} otherwise.
+   */
+  public boolean delegateHasMore() {
+    return delegate.hasNext();
+  }
+
+  @Override
+  public boolean hasNext() {
+    if (count < limit) {
+      return delegate.hasNext();
+    }
+    return false;
+  }
+
+  @Override
+  public T next() {
+    if (!hasNext()) { throw new NoSuchElementException(); }
+    count++;
+    return delegate.next();
+  }
+}
 
 Review comment:
   Actually, now that I've refactored off the `delegateHasMore` method, I can go back to using the implementation provided by Guava.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366083750
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/ClearUserNamespacesAndTablesRule.java
 ##########
 @@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.StringJoiner;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.client.AsyncAdmin;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.junit.Rule;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link Rule} that clears all user tables and namespaces before the test executes.
+ */
+public class ClearUserNamespacesAndTablesRule extends ExternalResource {
+  private static final Logger logger =
+    LoggerFactory.getLogger(ClearUserNamespacesAndTablesRule.class);
+
+  private final Supplier<AsyncConnection> connectionSupplier;
+  private AsyncAdmin admin;
+
+  public ClearUserNamespacesAndTablesRule(final Supplier<AsyncConnection> connectionSupplier) {
+    this.connectionSupplier = connectionSupplier;
+  }
+
+  @Override
+  protected void before() throws Throwable {
+    final AsyncConnection connection = Objects.requireNonNull(connectionSupplier.get());
+    admin = connection.getAdmin();
+
+    clearTablesAndNamespaces().join();
+  }
+
+  private CompletableFuture<Void> clearTablesAndNamespaces() {
+    return deleteUserTables().thenCompose(_void -> deleteUserNamespaces());
+  }
+
+  private CompletableFuture<Void> deleteUserTables() {
+    return listTableNames()
+      .thenApply(tableNames -> tableNames.stream()
+        .map(tableName -> disableIfEnabled(tableName).thenCompose(_void -> deleteTable(tableName)))
+        .toArray(CompletableFuture[]::new))
+      .thenCompose(CompletableFuture::allOf);
+  }
+
+  private CompletableFuture<List<TableName>> listTableNames() {
+    return CompletableFuture.runAsync(() -> logger.info("listing tables"))
+      .thenCompose(_void -> admin.listTableNames(false))
+      .thenApply(tableNames -> {
+        final StringJoiner joiner = new StringJoiner(", ", "[", "]");
+        tableNames.stream().map(TableName::getNameAsString).forEach(joiner::add);
+        logger.info("found existing tables {}", joiner.toString());
+        return tableNames;
+      });
+  }
+
+  private CompletableFuture<Boolean> isTableEnabled(final TableName tableName) {
+    return admin.isTableEnabled(tableName)
+      .thenApply(isEnabled -> {
+        logger.info("table {} is enabled.", tableName);
+        return isEnabled;
+      });
+  }
+
+  private CompletableFuture<Void> disableIfEnabled(final TableName tableName) {
+    return isTableEnabled(tableName)
+      .thenCompose(isEnabled -> {
+        if (isEnabled) { return disableTable(tableName); }
+        return CompletableFuture.completedFuture(null);
+      });
+  }
+
+  private CompletableFuture<Void> disableTable(final TableName tableName) {
+    return CompletableFuture.runAsync(() -> logger.info("disabling enabled table {}", tableName))
+      .thenCompose(_void -> admin.disableTable(tableName));
+  }
+
+  private CompletableFuture<Void> deleteTable(final TableName tableName) {
+    return CompletableFuture.runAsync(() -> logger.info("deleting disabled table {}", tableName))
+      .thenCompose(_void -> admin.deleteTable(tableName));
+  }
+
+  private CompletableFuture<List<String>> listUserNamespaces() {
+    return CompletableFuture.runAsync(() -> logger.info("listing namespaces"))
+      .thenCompose(_void -> admin.listNamespaceDescriptors())
+      .thenApply(namespaceDescriptors -> {
+        final StringJoiner joiner = new StringJoiner(", ", "[", "]");
+        final List<String> names = namespaceDescriptors.stream()
+          .map(NamespaceDescriptor::getName)
+          .peek(joiner::add)
+          .collect(Collectors.toList());
+        logger.info("found existing namespaces {}", joiner.toString());
+        return names;
+      })
+      .thenApply(namespaces -> namespaces.stream()
+        .filter(namespace -> !Objects.equals(
+          namespace, NamespaceDescriptor.SYSTEM_NAMESPACE.getName()))
+        .filter(namespace -> !Objects.equals(
+          namespace, NamespaceDescriptor.DEFAULT_NAMESPACE.getName()))
+        .collect(Collectors.toList()));
+  }
+
+  private CompletableFuture<Void> deleteNamespace(final String namespace) {
+    return CompletableFuture.runAsync(() -> logger.info("deleting namespace {}", namespace))
+      .thenCompose(_void -> admin.deleteNamespace(namespace));
+  }
+
+  private CompletableFuture<Void> deleteUserNamespaces() {
+    return listUserNamespaces()
+      .thenCompose(namespaces -> CompletableFuture.allOf(
+        namespaces.stream()
+          .map(this::deleteNamespace)
+          .toArray(CompletableFuture[]::new)));
+  }
+}
 
 Review comment:
   One thought, why cleanup after a test? What if it takes a bunch of work cleaning up, more than just start fresh?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r365474032
 
 

 ##########
 File path: hbase-server/src/main/resources/hbase-webapps/master/table.jsp
 ##########
 @@ -129,6 +151,7 @@
   pageContext.setAttribute("pageTitle", pageTitle);
   AsyncConnection connection = ConnectionFactory.createAsyncConnection(master.getConfiguration()).get();
 
 Review comment:
   Not your change, but creating a new connection every time this page loads seems a little heavy? Why not reuse the async connection created at the master startup? (master.getAsyncConnection()).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366081807
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final ExecutorService pool;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.pool = buildThreadPool();
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  @Override
+  public Iterator<RegionReplicaInfo> iterator() {
+    return limitIterator();
+  }
+
+  public LimitIterator<RegionReplicaInfo> limitIterator() {
+    logger.debug("initiating meta scan, {}", this);
+
+    final AsyncTable<ScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME, pool);
+    // TODO: buffering the entire result set seems unnecessary.
+    final List<RegionReplicaInfo> results = asyncTable.scanAll(buildScan()).join()
 
 Review comment:
   MetaTable Visitor no good to you in here?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r365479666
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final ExecutorService pool;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.pool = buildThreadPool();
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  @Override
+  public Iterator<RegionReplicaInfo> iterator() {
+    return limitIterator();
+  }
+
+  public LimitIterator<RegionReplicaInfo> limitIterator() {
+    logger.debug("initiating meta scan, {}", this);
+
+    final AsyncTable<ScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME, pool);
+    // TODO: buffering the entire result set seems unnecessary.
+    final List<RegionReplicaInfo> results = asyncTable.scanAll(buildScan()).join()
+      .stream()
+      .map(RegionReplicaInfo::from)
+      .flatMap(Collection::stream)
+      .collect(Collectors.toList());
+    return new LimitIterator<>(
+      results.iterator(), Optional.ofNullable(scanLimit).orElse(SCAN_LIMIT_DEFAULT));
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
+      .append("scanStart", scanStart)
+      .append("scanLimit", scanLimit)
+      .append("scanTable", scanTable)
+      .append("scanRegionState", scanRegionState)
+      .toString();
+  }
+
+  private static ExecutorService buildThreadPool() {
+    return Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+      .setNameFormat("MetaBrowser-%d")
+      .setDaemon(true)
+      .setUncaughtExceptionHandler(
+        (thread, throwable) -> logger.info("Error in worker thread, {}", throwable.getMessage()))
+      .build());
+  }
+
+  private static String resolveName(final HttpServletRequest request) {
+    return Optional.ofNullable(request)
+      .map(req -> req.getParameter(NAME_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode)
+      .orElse(null);
+  }
+
+  private Integer resolveScanLimit(final HttpServletRequest request) {
+    final Optional<String> requestValueStr = Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_LIMIT_PARAM))
+      .filter(StringUtils::isNotBlank);
+    if (!requestValueStr.isPresent()) { return null; }
+
+    final Integer requestValue = requestValueStr
+      .flatMap(MetaBrowser::tryParseInt)
+      .orElse(null);
+    if (requestValue == null) {
+      errorMessages.add(buildScanLimitMalformedErrorMessage(requestValueStr.get()));
+      return null;
+    }
+    if (requestValue <= 0) {
+      errorMessages.add(buildScanLimitLTEZero(requestValue));
+      return SCAN_LIMIT_DEFAULT;
+    }
+
+    final int truncatedValue = Math.min(requestValue, SCAN_LIMIT_MAX);
+    if (requestValue != truncatedValue) {
+      errorMessages.add(buildScanLimitExceededErrorMessage(requestValue));
+    }
+    return truncatedValue;
+  }
+
+  private RegionState.State resolveScanRegionState(final HttpServletRequest request) {
+    final Optional<String> requestValueStr = Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_REGION_STATE_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode);
+    if (!requestValueStr.isPresent()) { return null; }
+
+    final RegionState.State requestValue = requestValueStr
+      .flatMap(val -> tryValueOf(RegionState.State.class, val))
+      .orElse(null);
+    if (requestValue == null) {
+      errorMessages.add(buildScanRegionStateMalformedErrorMessage(requestValueStr.get()));
+      return null;
+    }
+    return requestValue;
+  }
+
+  private static byte[] resolveScanStart(final HttpServletRequest request) {
+    return Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_START_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode)
+      .map(Bytes::toBytesBinary)
+      .orElse(null);
+  }
+
+  private static TableName resolveScanTable(final HttpServletRequest request) {
+    return Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_TABLE_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode)
+      .map(TableName::valueOf)
+      .orElse(null);
+  }
+
+  private static Filter buildTableFilter(final TableName tableName) {
+    return new PrefixFilter(tableName.toBytes());
+  }
+
+  private static Filter buildScanRegionStateFilter(final RegionState.State state) {
+    return new SingleColumnValueFilter(
+      HConstants.CATALOG_FAMILY,
+      HConstants.TABLE_STATE_QUALIFIER,
 
 Review comment:
   this should be `HConstants.STATE_QUALIFIER`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on issue #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on issue #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#issuecomment-574779379
 
 
   Huh. Javadoc doesn't like link to nested classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r367223865
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/hamcrest/BytesMatchers.java
 ##########
 @@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client.hamcrest;
+
+import static org.hamcrest.core.Is.is;
 
 Review comment:
   ok

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366045795
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/RegionReplicaInfo.java
 ##########
 @@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A POJO that consolidates the information about a single region replica that's stored in meta.
+ */
+@InterfaceAudience.Private
+public final class RegionReplicaInfo {
+  private final byte[] row;
+  private final RegionInfo regionInfo;
+  private final byte[] regionName;
+  private final byte[] startKey;
+  private final byte[] endKey;
+  private final Integer replicaId;
+  private final RegionState.State regionState;
+  private final ServerName serverName;
+
+  private RegionReplicaInfo(final Result result, final HRegionLocation location) {
+    final Optional<Result> maybeResult = Optional.ofNullable(result);
+    final Optional<HRegionLocation> maybeLocation = Optional.ofNullable(location);
+    final Optional<RegionInfo> maybeRegionInfo = maybeLocation.map(HRegionLocation::getRegion);
+
+    this.row = maybeResult.map(Result::getRow).orElse(null);
 
 Review comment:
   Pushes the null-checking out to access instead of at object creation. I generally prefer checking to happen at object creation, since the values don't change. You think the one approach will be easier to read/maintain than the other?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#issuecomment-574202556
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m  6s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 7 new or modified test files.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 47s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 36s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 37s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 58s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 52s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 59s |  master passed  |
   | +0 :ok: |  spotbugs  |   3m 53s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |  19m 50s |  master passed  |
   | -0 :warning: |  patch  |   4m 17s |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 23s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m  6s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  3s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  3s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   2m 44s |  root: The patch generated 0 new + 241 unchanged - 1 fixed = 241 total (was 242)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  3s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedjars  |   5m  1s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  18m 28s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   4m 10s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |  22m 56s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  52m  8s |  root in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 57s |  The patch does not generate ASF License warnings.  |
   |  |   | 165m 36s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.master.webapp.TestMetaBrowserNoCluster |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1020 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile xml |
   | uname | Linux 22d6007baf3e 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1020/out/precommit/personality/provided.sh |
   | git revision | master / 499ff32f00 |
   | Default Java | 1.8.0_181 |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/2/artifact/out/patch-unit-root.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/2/testReport/ |
   | Max. process+thread count | 776 (vs. ulimit of 10000) |
   | modules | C: hbase-client hbase-server . U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/2/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366080981
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final ExecutorService pool;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.pool = buildThreadPool();
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  @Override
+  public Iterator<RegionReplicaInfo> iterator() {
+    return limitIterator();
+  }
+
+  public LimitIterator<RegionReplicaInfo> limitIterator() {
+    logger.debug("initiating meta scan, {}", this);
+
+    final AsyncTable<ScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME, pool);
+    // TODO: buffering the entire result set seems unnecessary.
+    final List<RegionReplicaInfo> results = asyncTable.scanAll(buildScan()).join()
 
 Review comment:
   Scan all each time we page?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366080344
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final ExecutorService pool;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.pool = buildThreadPool();
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
 
 Review comment:
   Above all have to public?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366092854
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
 
 Review comment:
   Yes, but not in a single page. A user can view up to `SCAN_LIMIT_MAX` rows in a single request. If there are more results, the next page link is rendered. When clicked, the next page worth of results are rendered. This pagination continues until all results are exhausted.
   
   The scanner is not held across button clicks, so the result set is not stable. If new entries are inserted before the "lastRow" pagination token, then the user will no see them. There's probably also a bug around region replicas, because I don't track how many replicas have been rendered by the page, only the last row rendered. Thus it's possible to have up to all non-primary replicas for one region slip through the cracks.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366064862
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/ClearUserNamespacesAndTablesRule.java
 ##########
 @@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.StringJoiner;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.client.AsyncAdmin;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.junit.Rule;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link Rule} that clears all user tables and namespaces before the test executes.
+ */
+public class ClearUserNamespacesAndTablesRule extends ExternalResource {
+  private static final Logger logger =
+    LoggerFactory.getLogger(ClearUserNamespacesAndTablesRule.class);
+
+  private final Supplier<AsyncConnection> connectionSupplier;
+  private AsyncAdmin admin;
+
+  public ClearUserNamespacesAndTablesRule(final Supplier<AsyncConnection> connectionSupplier) {
+    this.connectionSupplier = connectionSupplier;
+  }
+
+  @Override
+  protected void before() throws Throwable {
+    final AsyncConnection connection = Objects.requireNonNull(connectionSupplier.get());
+    admin = connection.getAdmin();
+
+    clearTablesAndNamespaces().join();
+  }
+
+  private CompletableFuture<Void> clearTablesAndNamespaces() {
+    return deleteUserTables().thenCompose(_void -> deleteUserNamespaces());
+  }
+
+  private CompletableFuture<Void> deleteUserTables() {
+    return listTableNames()
+      .thenApply(tableNames -> tableNames.stream()
+        .map(tableName -> disableIfEnabled(tableName).thenCompose(_void -> deleteTable(tableName)))
+        .toArray(CompletableFuture[]::new))
+      .thenCompose(CompletableFuture::allOf);
+  }
+
+  private CompletableFuture<List<TableName>> listTableNames() {
+    return CompletableFuture.runAsync(() -> logger.info("listing tables"))
 
 Review comment:
   Oops. All this logging should be at TRACE level.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366110661
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.StreamSupport;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser {
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  public Results getResults() {
+    final AsyncTable<AdvancedScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME);
+    return new Results(asyncTable.getScanner(buildScan()));
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
+      .append("scanStart", scanStart)
+      .append("scanLimit", scanLimit)
+      .append("scanTable", scanTable)
+      .append("scanRegionState", scanRegionState)
+      .toString();
+  }
+
+  private static String resolveName(final HttpServletRequest request) {
+    return resolveRequestParameter(request, NAME_PARAM);
+  }
+
+  private Integer resolveScanLimit(final HttpServletRequest request) {
+    final String requestValueStr = resolveRequestParameter(request, SCAN_LIMIT_PARAM);
+    if (StringUtils.isBlank(requestValueStr)) {
+      return null;
+    }
+
+    final Integer requestValue = tryParseInt(requestValueStr);
+    if (requestValue == null) {
+      errorMessages.add(buildScanLimitMalformedErrorMessage(requestValueStr));
+      return null;
+    }
+    if (requestValue <= 0) {
+      errorMessages.add(buildScanLimitLTEZero(requestValue));
+      return SCAN_LIMIT_DEFAULT;
+    }
+
+    final int truncatedValue = Math.min(requestValue, SCAN_LIMIT_MAX);
+    if (requestValue != truncatedValue) {
+      errorMessages.add(buildScanLimitExceededErrorMessage(requestValue));
+    }
+    return truncatedValue;
+  }
+
+  private RegionState.State resolveScanRegionState(final HttpServletRequest request) {
+    final String requestValueStr = resolveRequestParameter(request, SCAN_REGION_STATE_PARAM);
+    if (requestValueStr == null) {
+      return null;
+    }
+    final RegionState.State requestValue = tryValueOf(RegionState.State.class, requestValueStr);
+    if (requestValue == null) {
+      errorMessages.add(buildScanRegionStateMalformedErrorMessage(requestValueStr));
+      return null;
+    }
+    return requestValue;
+  }
+
+  private static byte[] resolveScanStart(final HttpServletRequest request) {
+    final String requestValue = resolveRequestParameter(request, SCAN_START_PARAM);
+    if (requestValue == null) {
+      return null;
+    }
+    return Bytes.toBytesBinary(requestValue);
+  }
+
+  private static TableName resolveScanTable(final HttpServletRequest request) {
+    final String requestValue = resolveRequestParameter(request, SCAN_TABLE_PARAM);
+    if (requestValue == null) {
+      return null;
+    }
+    return TableName.valueOf(requestValue);
+  }
+
+  private static String resolveRequestParameter(final HttpServletRequest request,
+    final String param) {
+    if (request == null) {
+      return null;
+    }
+    final String requestValueStrEnc = request.getParameter(param);
+    if (StringUtils.isBlank(requestValueStrEnc)) {
+      return null;
+    }
+    return urlDecode(requestValueStrEnc);
+  }
+
+  private static Filter buildTableFilter(final TableName tableName) {
+    return new PrefixFilter(tableName.toBytes());
+  }
+
+  private static Filter buildScanRegionStateFilter(final RegionState.State state) {
+    return new SingleColumnValueFilter(
+      HConstants.CATALOG_FAMILY,
+      HConstants.STATE_QUALIFIER,
+      CompareOperator.EQUAL,
+      // use the same serialization strategy as found in MetaTableAccessor#addRegionStateToPut
+      Bytes.toBytes(state.name()));
+  }
+
+  private Filter buildScanFilter() {
+    if (scanTable == null && scanRegionState == null) {
+      return null;
+    }
+
+    final List<Filter> filters = new ArrayList<>(2);
+    if (scanTable != null) {
+      filters.add(buildTableFilter(scanTable));
+    }
+    if (scanRegionState != null) {
+      filters.add(buildScanRegionStateFilter(scanRegionState));
+    }
+    if (filters.size() == 1) {
+      return filters.get(0);
+    }
+    return new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
+  }
+
+  private Scan buildScan() {
+    final Scan metaScan = new Scan()
+      .addFamily(HConstants.CATALOG_FAMILY)
+      .readVersions(1)
+      .setLimit((scanLimit != null ? scanLimit : SCAN_LIMIT_DEFAULT) + 1);
+    if (scanStart != null) {
+      metaScan.withStartRow(scanStart, false);
+    }
+    final Filter filter = buildScanFilter();
+    if (filter != null) {
+      metaScan.setFilter(filter);
+    }
+    return metaScan;
+  }
+
+  /**
+   * Adds {@code value} to {@code encoder} under {@code paramName} when {@code value} is non-null.
+   */
+  private <T> void addParam(final QueryStringEncoder encoder, final String paramName,
+    final T value) {
+    if (value != null) {
+      encoder.addParam(paramName, value.toString());
+    }
+  }
+
+  private QueryStringEncoder buildFirstPageEncoder() {
+    final QueryStringEncoder encoder =
+      new QueryStringEncoder(request.getRequestURI());
+    addParam(encoder, NAME_PARAM, name);
+    addParam(encoder, SCAN_LIMIT_PARAM, scanLimit);
+    addParam(encoder, SCAN_REGION_STATE_PARAM, scanRegionState);
+    addParam(encoder, SCAN_TABLE_PARAM, scanTable);
+    return encoder;
+  }
+
+  public String buildFirstPageUrl() {
+    return buildFirstPageEncoder().toString();
+  }
+
+  public static String buildStartParamFrom(final byte[] lastRow) {
+    if (lastRow == null) {
+      return null;
+    }
+    return urlEncode(Bytes.toStringBinary(lastRow));
+  }
+
+  public String buildNextPageUrl(final byte[] lastRow) {
+    final QueryStringEncoder encoder = buildFirstPageEncoder();
+    final String startRow = buildStartParamFrom(lastRow);
+    addParam(encoder, SCAN_START_PARAM, startRow);
+    return encoder.toString();
+  }
+
+  private static String urlEncode(final String val) {
+    if (StringUtils.isEmpty(val)) {
+      return null;
+    }
+    try {
+      return URLEncoder.encode(val, StandardCharsets.UTF_8.toString());
+    } catch (UnsupportedEncodingException e) {
+      return null;
+    }
+  }
+
+  private static String urlDecode(final String val) {
+    if (StringUtils.isEmpty(val)) {
+      return null;
+    }
+    try {
+      return URLDecoder.decode(val, StandardCharsets.UTF_8.toString());
+    } catch (UnsupportedEncodingException e) {
+      return null;
+    }
+  }
+
+  private static Integer tryParseInt(final String val) {
+    if (StringUtils.isEmpty(val)) {
+      return null;
+    }
+    try {
+      return Integer.parseInt(val);
+    } catch (NumberFormatException e) {
+      return null;
+    }
+  }
+
+  private static <T extends Enum<T>> T tryValueOf(final Class<T> clazz,
+    final String value) {
+    if (clazz == null || value == null) {
+      return null;
+    }
+    try {
+      return T.valueOf(clazz, value);
+    } catch (IllegalArgumentException e) {
+      return null;
+    }
+  }
+
+  private static String buildScanLimitExceededErrorMessage(final int requestValue) {
+    return String.format(
+      "Requested SCAN_LIMIT value %d exceeds maximum value %d.", requestValue, SCAN_LIMIT_MAX);
+  }
+
+  private static String buildScanLimitMalformedErrorMessage(final String requestValue) {
+    return String.format(
+      "Requested SCAN_LIMIT value '%s' cannot be parsed as an integer.", requestValue);
+  }
+
+  private static String buildScanLimitLTEZero(final int requestValue) {
+    return String.format("Requested SCAN_LIMIT value %d is <= 0.", requestValue);
+  }
+
+  private static String buildScanRegionStateMalformedErrorMessage(final String requestValue) {
+    return String.format(
+      "Requested SCAN_REGION_STATE value '%s' cannot be parsed as a RegionState.", requestValue);
+  }
+
+  /**
+   * Encapsulates the results produced by this {@link MetaBrowser} instance.
+   */
+  public final class Results implements AutoCloseable, Iterable<RegionReplicaInfo> {
 
 Review comment:
   I was about to suggest something like this.. wrap the output in some auto-closeable implementation that can be used in callers.. neat...

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366105313
 
 

 ##########
 File path: hbase-server/src/main/resources/hbase-webapps/master/table.jsp
 ##########
 @@ -47,46 +52,63 @@
   import="org.apache.hadoop.hbase.client.RegionReplicaUtil"
   import="org.apache.hadoop.hbase.client.Table"
   import="org.apache.hadoop.hbase.master.HMaster"
-  import="org.apache.hadoop.hbase.master.assignment.RegionStates"
   import="org.apache.hadoop.hbase.master.RegionState"
-  import="org.apache.hadoop.hbase.quotas.QuotaSettingsFactory"
-  import="org.apache.hadoop.hbase.quotas.QuotaTableUtil"
-  import="org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot"
-  import="org.apache.hadoop.hbase.quotas.ThrottleSettings"
-  import="org.apache.hadoop.hbase.util.Bytes"
-  import="org.apache.hadoop.hbase.util.FSUtils"
-  import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator"
-  import="org.apache.hadoop.util.StringUtils"
-  import="org.apache.hbase.thirdparty.com.google.protobuf.ByteString"%>
+  import="org.apache.hadoop.hbase.master.assignment.RegionStates"
+  import="org.apache.hadoop.hbase.master.webapp.LimitIterator"
+  import="org.apache.hadoop.hbase.master.webapp.MetaBrowser"
+  import="org.apache.hadoop.hbase.master.webapp.RegionReplicaInfo"
+  import="org.apache.hadoop.hbase.quotas.QuotaSettingsFactory"%>
+<%@ page import="org.apache.hadoop.hbase.quotas.QuotaTableUtil" %>
+<%@ page import="org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot" %>
+<%@ page import="org.apache.hadoop.hbase.quotas.ThrottleSettings" %>
+<%@ page import="org.apache.hadoop.hbase.util.Bytes" %>
+<%@ page import="org.apache.hadoop.hbase.util.FSUtils" %>
+<%@ page import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator" %>
+<%@ page import="org.apache.hadoop.util.StringUtils" %>
+<%@ page import="org.apache.hbase.thirdparty.com.google.protobuf.ByteString" %>
 <%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos" %>
 <%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos" %>
 <%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.Quotas" %>
 <%@ page import="org.apache.hadoop.hbase.shaded.protobuf.generated.QuotaProtos.SpaceQuota" %>
-<%@ page import="org.apache.hadoop.hbase.ServerMetrics" %>
-<%@ page import="org.apache.hadoop.hbase.RegionMetrics" %>
-<%@ page import="org.apache.hadoop.hbase.Size" %>
-<%@ page import="org.apache.hadoop.hbase.RegionMetricsBuilder" %>
 <%!
   /**
    * @return An empty region load stamped with the passed in <code>regionInfo</code>
    * region name.
    */
-  private RegionMetrics getEmptyRegionMetrics(final RegionInfo regionInfo) {
+  private static RegionMetrics getEmptyRegionMetrics(final RegionInfo regionInfo) {
     return RegionMetricsBuilder.toRegionMetrics(ClusterStatusProtos.RegionLoad.newBuilder().
             setRegionSpecifier(HBaseProtos.RegionSpecifier.newBuilder().
                     setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME).
                     setValue(ByteString.copyFrom(regionInfo.getRegionName())).build()).build());
   }
+
+  /**
+   * Given dicey information that may or not be available in meta, render a link to the region on
+   * its region server.
+   * @return an anchor tag if one can be built, {@code null} otherwise.
+   */
+  private static String buildRegionServerLink(final ServerName serverName, final int rsInfoPort,
+    final RegionInfo regionInfo, final RegionState.State regionState) {
+    if (serverName == null || regionInfo == null) { return null; }
+
+    if (regionState != RegionState.State.OPEN) {
+      // region is assigned to RS, but RS knows nothing of it. don't bother with a link.
+      return serverName.getServerName();
+    }
+
+    final String socketAddress = serverName.getHostname() + ":" + rsInfoPort;
+    final String URI = "//" + socketAddress + "/region.jsp"
+      + "?name=" + regionInfo.getEncodedName();
+    return "<a href=\"" + URI + "\">" + serverName.getServerName() + "</a>";
 
 Review comment:
   Well, almost. I found one spot: the region server generates these links for /rs-status. All it has to do is drop the encoded region name onto a path, no hostname at all. So, not really reusable.
   
   https://github.com/apache/hbase/blob/master/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon#L108

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366157431
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/hamcrest/BytesMatchers.java
 ##########
 @@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client.hamcrest;
+
+import static org.hamcrest.core.Is.is;
 
 Review comment:
   Is this the only class we use from hamcrest? Are we adding a new dependency to modules just for one class? Its a good one I presume.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366027154
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final ExecutorService pool;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.pool = buildThreadPool();
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  @Override
+  public Iterator<RegionReplicaInfo> iterator() {
+    return limitIterator();
+  }
+
+  public LimitIterator<RegionReplicaInfo> limitIterator() {
+    logger.debug("initiating meta scan, {}", this);
+
+    final AsyncTable<ScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME, pool);
+    // TODO: buffering the entire result set seems unnecessary.
+    final List<RegionReplicaInfo> results = asyncTable.scanAll(buildScan()).join()
+      .stream()
+      .map(RegionReplicaInfo::from)
+      .flatMap(Collection::stream)
+      .collect(Collectors.toList());
+    return new LimitIterator<>(
+      results.iterator(), Optional.ofNullable(scanLimit).orElse(SCAN_LIMIT_DEFAULT));
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
+      .append("scanStart", scanStart)
+      .append("scanLimit", scanLimit)
+      .append("scanTable", scanTable)
+      .append("scanRegionState", scanRegionState)
+      .toString();
+  }
+
+  private static ExecutorService buildThreadPool() {
+    return Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+      .setNameFormat("MetaBrowser-%d")
+      .setDaemon(true)
+      .setUncaughtExceptionHandler(
+        (thread, throwable) -> logger.info("Error in worker thread, {}", throwable.getMessage()))
+      .build());
+  }
+
+  private static String resolveName(final HttpServletRequest request) {
+    return Optional.ofNullable(request)
+      .map(req -> req.getParameter(NAME_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode)
+      .orElse(null);
+  }
+
+  private Integer resolveScanLimit(final HttpServletRequest request) {
+    final Optional<String> requestValueStr = Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_LIMIT_PARAM))
+      .filter(StringUtils::isNotBlank);
+    if (!requestValueStr.isPresent()) { return null; }
+
+    final Integer requestValue = requestValueStr
+      .flatMap(MetaBrowser::tryParseInt)
+      .orElse(null);
+    if (requestValue == null) {
+      errorMessages.add(buildScanLimitMalformedErrorMessage(requestValueStr.get()));
+      return null;
+    }
+    if (requestValue <= 0) {
+      errorMessages.add(buildScanLimitLTEZero(requestValue));
+      return SCAN_LIMIT_DEFAULT;
+    }
+
+    final int truncatedValue = Math.min(requestValue, SCAN_LIMIT_MAX);
+    if (requestValue != truncatedValue) {
+      errorMessages.add(buildScanLimitExceededErrorMessage(requestValue));
+    }
+    return truncatedValue;
+  }
+
+  private RegionState.State resolveScanRegionState(final HttpServletRequest request) {
+    final Optional<String> requestValueStr = Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_REGION_STATE_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode);
+    if (!requestValueStr.isPresent()) { return null; }
+
+    final RegionState.State requestValue = requestValueStr
+      .flatMap(val -> tryValueOf(RegionState.State.class, val))
+      .orElse(null);
+    if (requestValue == null) {
+      errorMessages.add(buildScanRegionStateMalformedErrorMessage(requestValueStr.get()));
+      return null;
+    }
+    return requestValue;
+  }
+
+  private static byte[] resolveScanStart(final HttpServletRequest request) {
+    return Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_START_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode)
+      .map(Bytes::toBytesBinary)
+      .orElse(null);
+  }
+
+  private static TableName resolveScanTable(final HttpServletRequest request) {
+    return Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_TABLE_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode)
+      .map(TableName::valueOf)
+      .orElse(null);
+  }
+
+  private static Filter buildTableFilter(final TableName tableName) {
+    return new PrefixFilter(tableName.toBytes());
+  }
+
+  private static Filter buildScanRegionStateFilter(final RegionState.State state) {
+    return new SingleColumnValueFilter(
+      HConstants.CATALOG_FAMILY,
+      HConstants.TABLE_STATE_QUALIFIER,
+      CompareOperator.EQUAL,
+      // use the same serialization strategy as found in MetaTableAccessor#addRegionStateToPut
+      Bytes.toBytes(state.name()));
+  }
+
+  private Optional<Filter> buildScanFilter() {
+    if (scanTable == null && scanRegionState == null) {
+      return Optional.empty();
+    }
+
+    final List<Filter> filters = new ArrayList<>(2);
+    Optional.ofNullable(scanTable)
+      .map(MetaBrowser::buildTableFilter)
+      .ifPresent(filters::add);
+    Optional.ofNullable(scanRegionState)
 
 Review comment:
   Yeah, it's probably a bit overboard here. I started a back port to branch-1 and found there's only a couple places where Optional makes things easier.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366084083
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/hamcrest/BytesMatchers.java
 ##########
 @@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client.hamcrest;
+
+import static org.hamcrest.core.Is.is;
 
 Review comment:
   This only reason for import?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366074950
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/LimitIterator.java
 ##########
 @@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+
+/**
+ * An {@link Iterator} over {@code delegate} that limits results to the first {@code limit}
+ * entries.
+ * <p>Could just use {@link Iterators#limit(Iterator, int)} except that our consumer needs an API
+ * to check if the underlying iterator is not yet exhausted.
+ */
+@InterfaceAudience.Private
+public class LimitIterator<T> implements Iterator<T> {
+
+  private final Iterator<T> delegate;
+  private final int limit;
+  private int count;
+
+  LimitIterator(final Iterator<T> delegate, final int limit) {
+    this.delegate = delegate;
+    this.limit = limit;
+    this.count = 0;
+  }
+
+  /**
+   * @return {@code true} when {@code delegate} has more entries, {@code false} otherwise.
+   */
+  public boolean delegateHasMore() {
+    return delegate.hasNext();
+  }
+
+  @Override
+  public boolean hasNext() {
+    if (count < limit) {
+      return delegate.hasNext();
+    }
+    return false;
+  }
+
+  @Override
+  public T next() {
+    if (!hasNext()) { throw new NoSuchElementException(); }
+    count++;
+    return delegate.next();
+  }
+}
 
 Review comment:
   hbase-common? Especially given this is generic.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366083808
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/ConnectionRule.java
 ##########
 @@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Supplier;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.junit.Rule;
+import org.junit.rules.ExternalResource;
+
+/**
+ * A {@link Rule} that manages an instance of {@link AsyncConnection}.
+ */
+public class ConnectionRule extends ExternalResource {
+
+  private final Supplier<CompletableFuture<AsyncConnection>> connectionSupplier;
+  private AsyncConnection connection;
+
+  public ConnectionRule(final Supplier<CompletableFuture<AsyncConnection>> connectionSupplier) {
+    this.connectionSupplier = connectionSupplier;
+  }
+
+  public AsyncConnection getConnection() {
+    return connection;
+  }
+
+  @Override
+  protected void before() throws Throwable {
+    this.connection = connectionSupplier.get().join();
+  }
+
+  @Override
+  protected void after() {
+    if (this.connection != null) {
+      try {
+        connection.close();
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+}
 
 Review comment:
   Fun

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk merged pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk merged pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r365480952
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/ClearUserNamespacesAndTablesRule.java
 ##########
 @@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.StringJoiner;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.client.AsyncAdmin;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.junit.Rule;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link Rule} that clears all user tables and namespaces before the test executes.
+ */
+public class ClearUserNamespacesAndTablesRule extends ExternalResource {
+  private static final Logger logger =
+    LoggerFactory.getLogger(ClearUserNamespacesAndTablesRule.class);
+
+  private final Supplier<AsyncConnection> connectionSupplier;
 
 Review comment:
   May be add a comment that this class doesn't close the connection by the end of it and needs to chained with TestConnectionRule which does it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366096964
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/RegionReplicaInfo.java
 ##########
 @@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A POJO that consolidates the information about a single region replica that's stored in meta.
+ */
+@InterfaceAudience.Private
+public final class RegionReplicaInfo {
+  private final byte[] row;
+  private final RegionInfo regionInfo;
+  private final byte[] regionName;
+  private final byte[] startKey;
+  private final byte[] endKey;
+  private final Integer replicaId;
+  private final RegionState.State regionState;
+  private final ServerName serverName;
+
+  private RegionReplicaInfo(final Result result, final HRegionLocation location) {
+    final Optional<Result> maybeResult = Optional.ofNullable(result);
+    final Optional<HRegionLocation> maybeLocation = Optional.ofNullable(location);
+    final Optional<RegionInfo> maybeRegionInfo = maybeLocation.map(HRegionLocation::getRegion);
+
+    this.row = maybeResult.map(Result::getRow).orElse(null);
+    this.regionInfo = maybeRegionInfo.orElse(null);
+    this.regionName = maybeRegionInfo.map(RegionInfo::getRegionName).orElse(null);
+    this.startKey = maybeRegionInfo.map(RegionInfo::getStartKey).orElse(null);
+    this.endKey = maybeRegionInfo.map(RegionInfo::getEndKey).orElse(null);
+    this.replicaId = maybeRegionInfo.map(RegionInfo::getReplicaId).orElse(null);
+    this.regionState = result != null && maybeRegionInfo.isPresent()
+      ? RegionStateStore.getRegionState(result, maybeRegionInfo.get())
+      : null;
+    this.serverName = maybeLocation.map(HRegionLocation::getServerName).orElse(null);
+  }
+
+  public static List<RegionReplicaInfo> from(final Result result) {
+    if (result == null) {
+      return Collections.singletonList(null);
+    }
+
+    final RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
+    if (locations == null) {
+      return Collections.singletonList(null);
+    }
+
+    return StreamSupport.stream(locations.spliterator(), false)
+      .map(location -> new RegionReplicaInfo(result, location))
+      .collect(Collectors.toList());
+  }
+
+  public byte[] getRow() {
+    return row;
+  }
+
+  public RegionInfo getRegionInfo() {
+    return regionInfo;
+  }
+
+  public byte[] getRegionName() {
+    return regionName;
+  }
+
+  public byte[] getStartKey() {
+    return startKey;
+  }
+
+  public byte[] getEndKey() {
+    return endKey;
+  }
+
+  public Integer getReplicaId() {
+    return replicaId;
+  }
+
+  public RegionState.State getRegionState() {
+    return regionState;
+  }
+
+  public ServerName getServerName() {
 
 Review comment:
   For the same reason as the accessors in `MetaBrowser`: they need to be reachable by `table.jsp`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r365468451
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/LimitIterator.java
 ##########
 @@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+
+/**
+ * An {@link Iterator} over {@code delegate} that limits results to the first {@code limit}
 
 Review comment:
   Any reason not to use Iterators.limit() from guava?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r365476199
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final ExecutorService pool;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.pool = buildThreadPool();
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  @Override
+  public Iterator<RegionReplicaInfo> iterator() {
+    return limitIterator();
+  }
+
+  public LimitIterator<RegionReplicaInfo> limitIterator() {
+    logger.debug("initiating meta scan, {}", this);
+
+    final AsyncTable<ScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME, pool);
+    // TODO: buffering the entire result set seems unnecessary.
+    final List<RegionReplicaInfo> results = asyncTable.scanAll(buildScan()).join()
+      .stream()
+      .map(RegionReplicaInfo::from)
+      .flatMap(Collection::stream)
+      .collect(Collectors.toList());
+    return new LimitIterator<>(
+      results.iterator(), Optional.ofNullable(scanLimit).orElse(SCAN_LIMIT_DEFAULT));
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
+      .append("scanStart", scanStart)
+      .append("scanLimit", scanLimit)
+      .append("scanTable", scanTable)
+      .append("scanRegionState", scanRegionState)
+      .toString();
+  }
+
+  private static ExecutorService buildThreadPool() {
 
 Review comment:
   Curious why it runs in its own thread. Add a clarifying comment?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366483832
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.StreamSupport;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser {
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  public Results getResults() {
+    final AsyncTable<AdvancedScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME);
+    return new Results(asyncTable.getScanner(buildScan()));
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
+      .append("scanStart", scanStart)
+      .append("scanLimit", scanLimit)
+      .append("scanTable", scanTable)
+      .append("scanRegionState", scanRegionState)
+      .toString();
+  }
+
+  private static String resolveName(final HttpServletRequest request) {
+    return resolveRequestParameter(request, NAME_PARAM);
+  }
+
+  private Integer resolveScanLimit(final HttpServletRequest request) {
+    final String requestValueStr = resolveRequestParameter(request, SCAN_LIMIT_PARAM);
+    if (StringUtils.isBlank(requestValueStr)) {
+      return null;
+    }
+
+    final Integer requestValue = tryParseInt(requestValueStr);
+    if (requestValue == null) {
+      errorMessages.add(buildScanLimitMalformedErrorMessage(requestValueStr));
+      return null;
+    }
+    if (requestValue <= 0) {
+      errorMessages.add(buildScanLimitLTEZero(requestValue));
+      return SCAN_LIMIT_DEFAULT;
+    }
+
+    final int truncatedValue = Math.min(requestValue, SCAN_LIMIT_MAX);
+    if (requestValue != truncatedValue) {
+      errorMessages.add(buildScanLimitExceededErrorMessage(requestValue));
+    }
+    return truncatedValue;
+  }
+
+  private RegionState.State resolveScanRegionState(final HttpServletRequest request) {
+    final String requestValueStr = resolveRequestParameter(request, SCAN_REGION_STATE_PARAM);
+    if (requestValueStr == null) {
+      return null;
+    }
+    final RegionState.State requestValue = tryValueOf(RegionState.State.class, requestValueStr);
+    if (requestValue == null) {
+      errorMessages.add(buildScanRegionStateMalformedErrorMessage(requestValueStr));
+      return null;
+    }
+    return requestValue;
+  }
+
+  private static byte[] resolveScanStart(final HttpServletRequest request) {
+    final String requestValue = resolveRequestParameter(request, SCAN_START_PARAM);
+    if (requestValue == null) {
+      return null;
+    }
+    return Bytes.toBytesBinary(requestValue);
+  }
+
+  private static TableName resolveScanTable(final HttpServletRequest request) {
+    final String requestValue = resolveRequestParameter(request, SCAN_TABLE_PARAM);
+    if (requestValue == null) {
+      return null;
+    }
+    return TableName.valueOf(requestValue);
+  }
+
+  private static String resolveRequestParameter(final HttpServletRequest request,
+    final String param) {
+    if (request == null) {
+      return null;
+    }
+    final String requestValueStrEnc = request.getParameter(param);
+    if (StringUtils.isBlank(requestValueStrEnc)) {
+      return null;
+    }
+    return urlDecode(requestValueStrEnc);
+  }
+
+  private static Filter buildTableFilter(final TableName tableName) {
+    return new PrefixFilter(tableName.toBytes());
+  }
+
+  private static Filter buildScanRegionStateFilter(final RegionState.State state) {
+    return new SingleColumnValueFilter(
+      HConstants.CATALOG_FAMILY,
+      HConstants.STATE_QUALIFIER,
+      CompareOperator.EQUAL,
+      // use the same serialization strategy as found in MetaTableAccessor#addRegionStateToPut
+      Bytes.toBytes(state.name()));
+  }
+
+  private Filter buildScanFilter() {
+    if (scanTable == null && scanRegionState == null) {
+      return null;
+    }
+
+    final List<Filter> filters = new ArrayList<>(2);
+    if (scanTable != null) {
+      filters.add(buildTableFilter(scanTable));
+    }
+    if (scanRegionState != null) {
+      filters.add(buildScanRegionStateFilter(scanRegionState));
+    }
+    if (filters.size() == 1) {
+      return filters.get(0);
+    }
+    return new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
+  }
+
+  private Scan buildScan() {
+    final Scan metaScan = new Scan()
+      .addFamily(HConstants.CATALOG_FAMILY)
+      .readVersions(1)
+      .setLimit((scanLimit != null ? scanLimit : SCAN_LIMIT_DEFAULT) + 1);
+    if (scanStart != null) {
+      metaScan.withStartRow(scanStart, false);
+    }
+    final Filter filter = buildScanFilter();
+    if (filter != null) {
+      metaScan.setFilter(filter);
+    }
+    return metaScan;
+  }
+
+  /**
+   * Adds {@code value} to {@code encoder} under {@code paramName} when {@code value} is non-null.
+   */
+  private <T> void addParam(final QueryStringEncoder encoder, final String paramName,
+    final T value) {
+    if (value != null) {
+      encoder.addParam(paramName, value.toString());
+    }
+  }
+
+  private QueryStringEncoder buildFirstPageEncoder() {
+    final QueryStringEncoder encoder =
+      new QueryStringEncoder(request.getRequestURI());
+    addParam(encoder, NAME_PARAM, name);
+    addParam(encoder, SCAN_LIMIT_PARAM, scanLimit);
+    addParam(encoder, SCAN_REGION_STATE_PARAM, scanRegionState);
+    addParam(encoder, SCAN_TABLE_PARAM, scanTable);
+    return encoder;
+  }
+
+  public String buildFirstPageUrl() {
+    return buildFirstPageEncoder().toString();
+  }
+
+  public static String buildStartParamFrom(final byte[] lastRow) {
 
 Review comment:
   this can be package-protected as it's only used internally and within the unit test.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366082645
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/RegionReplicaInfo.java
 ##########
 @@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A POJO that consolidates the information about a single region replica that's stored in meta.
+ */
+@InterfaceAudience.Private
+public final class RegionReplicaInfo {
+  private final byte[] row;
+  private final RegionInfo regionInfo;
+  private final byte[] regionName;
+  private final byte[] startKey;
+  private final byte[] endKey;
+  private final Integer replicaId;
+  private final RegionState.State regionState;
+  private final ServerName serverName;
+
+  private RegionReplicaInfo(final Result result, final HRegionLocation location) {
+    final Optional<Result> maybeResult = Optional.ofNullable(result);
+    final Optional<HRegionLocation> maybeLocation = Optional.ofNullable(location);
+    final Optional<RegionInfo> maybeRegionInfo = maybeLocation.map(HRegionLocation::getRegion);
+
+    this.row = maybeResult.map(Result::getRow).orElse(null);
+    this.regionInfo = maybeRegionInfo.orElse(null);
+    this.regionName = maybeRegionInfo.map(RegionInfo::getRegionName).orElse(null);
+    this.startKey = maybeRegionInfo.map(RegionInfo::getStartKey).orElse(null);
+    this.endKey = maybeRegionInfo.map(RegionInfo::getEndKey).orElse(null);
+    this.replicaId = maybeRegionInfo.map(RegionInfo::getReplicaId).orElse(null);
+    this.regionState = result != null && maybeRegionInfo.isPresent()
+      ? RegionStateStore.getRegionState(result, maybeRegionInfo.get())
+      : null;
+    this.serverName = maybeLocation.map(HRegionLocation::getServerName).orElse(null);
+  }
+
+  public static List<RegionReplicaInfo> from(final Result result) {
+    if (result == null) {
+      return Collections.singletonList(null);
+    }
+
+    final RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
+    if (locations == null) {
+      return Collections.singletonList(null);
+    }
+
+    return StreamSupport.stream(locations.spliterator(), false)
+      .map(location -> new RegionReplicaInfo(result, location))
+      .collect(Collectors.toList());
+  }
+
+  public byte[] getRow() {
+    return row;
+  }
+
+  public RegionInfo getRegionInfo() {
+    return regionInfo;
+  }
+
+  public byte[] getRegionName() {
+    return regionName;
+  }
+
+  public byte[] getStartKey() {
+    return startKey;
+  }
+
+  public byte[] getEndKey() {
+    return endKey;
+  }
+
+  public Integer getReplicaId() {
+    return replicaId;
+  }
+
+  public RegionState.State getRegionState() {
+    return regionState;
+  }
+
+  public ServerName getServerName() {
 
 Review comment:
   These have to be public?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366074628
 
 

 ##########
 File path: hbase-server/pom.xml
 ##########
 @@ -436,6 +436,11 @@
       <artifactId>hamcrest-core</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-library</artifactId>
 
 Review comment:
   Why?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366513636
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
 
 Review comment:
   Big fat javadoc applied.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366467988
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.StreamSupport;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser {
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  public Results getResults() {
+    final AsyncTable<AdvancedScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME);
+    return new Results(asyncTable.getScanner(buildScan()));
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
+      .append("scanStart", scanStart)
+      .append("scanLimit", scanLimit)
+      .append("scanTable", scanTable)
+      .append("scanRegionState", scanRegionState)
+      .toString();
+  }
+
+  private static String resolveName(final HttpServletRequest request) {
+    return resolveRequestParameter(request, NAME_PARAM);
+  }
+
+  private Integer resolveScanLimit(final HttpServletRequest request) {
+    final String requestValueStr = resolveRequestParameter(request, SCAN_LIMIT_PARAM);
+    if (StringUtils.isBlank(requestValueStr)) {
+      return null;
+    }
+
+    final Integer requestValue = tryParseInt(requestValueStr);
+    if (requestValue == null) {
+      errorMessages.add(buildScanLimitMalformedErrorMessage(requestValueStr));
+      return null;
+    }
+    if (requestValue <= 0) {
+      errorMessages.add(buildScanLimitLTEZero(requestValue));
+      return SCAN_LIMIT_DEFAULT;
+    }
+
+    final int truncatedValue = Math.min(requestValue, SCAN_LIMIT_MAX);
+    if (requestValue != truncatedValue) {
+      errorMessages.add(buildScanLimitExceededErrorMessage(requestValue));
+    }
+    return truncatedValue;
+  }
+
+  private RegionState.State resolveScanRegionState(final HttpServletRequest request) {
+    final String requestValueStr = resolveRequestParameter(request, SCAN_REGION_STATE_PARAM);
+    if (requestValueStr == null) {
+      return null;
+    }
+    final RegionState.State requestValue = tryValueOf(RegionState.State.class, requestValueStr);
+    if (requestValue == null) {
+      errorMessages.add(buildScanRegionStateMalformedErrorMessage(requestValueStr));
+      return null;
+    }
+    return requestValue;
+  }
+
+  private static byte[] resolveScanStart(final HttpServletRequest request) {
+    final String requestValue = resolveRequestParameter(request, SCAN_START_PARAM);
+    if (requestValue == null) {
+      return null;
+    }
+    return Bytes.toBytesBinary(requestValue);
+  }
+
+  private static TableName resolveScanTable(final HttpServletRequest request) {
+    final String requestValue = resolveRequestParameter(request, SCAN_TABLE_PARAM);
+    if (requestValue == null) {
+      return null;
+    }
+    return TableName.valueOf(requestValue);
+  }
+
+  private static String resolveRequestParameter(final HttpServletRequest request,
+    final String param) {
+    if (request == null) {
+      return null;
+    }
+    final String requestValueStrEnc = request.getParameter(param);
+    if (StringUtils.isBlank(requestValueStrEnc)) {
+      return null;
+    }
+    return urlDecode(requestValueStrEnc);
+  }
+
+  private static Filter buildTableFilter(final TableName tableName) {
+    return new PrefixFilter(tableName.toBytes());
+  }
+
+  private static Filter buildScanRegionStateFilter(final RegionState.State state) {
+    return new SingleColumnValueFilter(
+      HConstants.CATALOG_FAMILY,
+      HConstants.STATE_QUALIFIER,
+      CompareOperator.EQUAL,
+      // use the same serialization strategy as found in MetaTableAccessor#addRegionStateToPut
+      Bytes.toBytes(state.name()));
+  }
+
+  private Filter buildScanFilter() {
+    if (scanTable == null && scanRegionState == null) {
+      return null;
+    }
+
+    final List<Filter> filters = new ArrayList<>(2);
+    if (scanTable != null) {
+      filters.add(buildTableFilter(scanTable));
+    }
+    if (scanRegionState != null) {
+      filters.add(buildScanRegionStateFilter(scanRegionState));
+    }
+    if (filters.size() == 1) {
+      return filters.get(0);
+    }
+    return new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
+  }
+
+  private Scan buildScan() {
+    final Scan metaScan = new Scan()
+      .addFamily(HConstants.CATALOG_FAMILY)
+      .readVersions(1)
+      .setLimit((scanLimit != null ? scanLimit : SCAN_LIMIT_DEFAULT) + 1);
 
 Review comment:
   Rgr.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366099209
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/client/hamcrest/BytesMatchers.java
 ##########
 @@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.client.hamcrest;
+
+import static org.hamcrest.core.Is.is;
 
 Review comment:
   I don't follow the question. This class comes out of hamcrest-core.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366106432
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
 
 Review comment:
   I think some form of this could be captured in the javadoc class comment. Otherwise, one needs to understand how this class paginates the result by setting the start row intelligently..

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366094019
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final ExecutorService pool;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.pool = buildThreadPool();
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  @Override
+  public Iterator<RegionReplicaInfo> iterator() {
+    return limitIterator();
+  }
+
+  public LimitIterator<RegionReplicaInfo> limitIterator() {
+    logger.debug("initiating meta scan, {}", this);
+
+    final AsyncTable<ScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME, pool);
+    // TODO: buffering the entire result set seems unnecessary.
+    final List<RegionReplicaInfo> results = asyncTable.scanAll(buildScan()).join()
 
 Review comment:
   > Scan all each time we page?
   
   Scan all of meta? No. Scanning starts at the row represented by `SCAN_START_PARAM` and is limited by `SCAN_LIMIT_PARAM` and the other filter parameters. See the `TestMetaBrowser#paginate*` tests.
   
   > MetaTable Visitor no good to you in here?
   
   Is that a class? I figured out how to do away with `scanAll`; moving to the old fashioned `ResultScanner`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r365477170
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final ExecutorService pool;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.pool = buildThreadPool();
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  @Override
+  public Iterator<RegionReplicaInfo> iterator() {
+    return limitIterator();
+  }
+
+  public LimitIterator<RegionReplicaInfo> limitIterator() {
+    logger.debug("initiating meta scan, {}", this);
+
+    final AsyncTable<ScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME, pool);
+    // TODO: buffering the entire result set seems unnecessary.
+    final List<RegionReplicaInfo> results = asyncTable.scanAll(buildScan()).join()
+      .stream()
+      .map(RegionReplicaInfo::from)
+      .flatMap(Collection::stream)
+      .collect(Collectors.toList());
+    return new LimitIterator<>(
+      results.iterator(), Optional.ofNullable(scanLimit).orElse(SCAN_LIMIT_DEFAULT));
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
+      .append("scanStart", scanStart)
+      .append("scanLimit", scanLimit)
+      .append("scanTable", scanTable)
+      .append("scanRegionState", scanRegionState)
+      .toString();
+  }
+
+  private static ExecutorService buildThreadPool() {
+    return Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+      .setNameFormat("MetaBrowser-%d")
+      .setDaemon(true)
+      .setUncaughtExceptionHandler(
+        (thread, throwable) -> logger.info("Error in worker thread, {}", throwable.getMessage()))
+      .build());
+  }
+
+  private static String resolveName(final HttpServletRequest request) {
+    return Optional.ofNullable(request)
+      .map(req -> req.getParameter(NAME_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode)
+      .orElse(null);
+  }
+
+  private Integer resolveScanLimit(final HttpServletRequest request) {
+    final Optional<String> requestValueStr = Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_LIMIT_PARAM))
+      .filter(StringUtils::isNotBlank);
+    if (!requestValueStr.isPresent()) { return null; }
+
+    final Integer requestValue = requestValueStr
+      .flatMap(MetaBrowser::tryParseInt)
+      .orElse(null);
+    if (requestValue == null) {
+      errorMessages.add(buildScanLimitMalformedErrorMessage(requestValueStr.get()));
+      return null;
+    }
+    if (requestValue <= 0) {
+      errorMessages.add(buildScanLimitLTEZero(requestValue));
+      return SCAN_LIMIT_DEFAULT;
+    }
+
+    final int truncatedValue = Math.min(requestValue, SCAN_LIMIT_MAX);
+    if (requestValue != truncatedValue) {
+      errorMessages.add(buildScanLimitExceededErrorMessage(requestValue));
+    }
+    return truncatedValue;
+  }
+
+  private RegionState.State resolveScanRegionState(final HttpServletRequest request) {
+    final Optional<String> requestValueStr = Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_REGION_STATE_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode);
+    if (!requestValueStr.isPresent()) { return null; }
+
+    final RegionState.State requestValue = requestValueStr
+      .flatMap(val -> tryValueOf(RegionState.State.class, val))
+      .orElse(null);
+    if (requestValue == null) {
+      errorMessages.add(buildScanRegionStateMalformedErrorMessage(requestValueStr.get()));
+      return null;
+    }
+    return requestValue;
+  }
+
+  private static byte[] resolveScanStart(final HttpServletRequest request) {
+    return Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_START_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode)
+      .map(Bytes::toBytesBinary)
+      .orElse(null);
+  }
+
+  private static TableName resolveScanTable(final HttpServletRequest request) {
+    return Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_TABLE_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode)
+      .map(TableName::valueOf)
+      .orElse(null);
+  }
+
+  private static Filter buildTableFilter(final TableName tableName) {
+    return new PrefixFilter(tableName.toBytes());
+  }
+
+  private static Filter buildScanRegionStateFilter(final RegionState.State state) {
+    return new SingleColumnValueFilter(
+      HConstants.CATALOG_FAMILY,
+      HConstants.TABLE_STATE_QUALIFIER,
+      CompareOperator.EQUAL,
+      // use the same serialization strategy as found in MetaTableAccessor#addRegionStateToPut
+      Bytes.toBytes(state.name()));
+  }
+
+  private Optional<Filter> buildScanFilter() {
+    if (scanTable == null && scanRegionState == null) {
+      return Optional.empty();
+    }
+
+    final List<Filter> filters = new ArrayList<>(2);
+    Optional.ofNullable(scanTable)
+      .map(MetaBrowser::buildTableFilter)
+      .ifPresent(filters::add);
+    Optional.ofNullable(scanRegionState)
+      .map(MetaBrowser::buildScanRegionStateFilter)
+      .ifPresent(filters::add);
+
+    if (filters.size() == 1) {
+      return Optional.of(filters.get(0));
+    }
+
+    return Optional.of(new FilterList(FilterList.Operator.MUST_PASS_ALL, filters));
+  }
+
+  private Scan buildScan() {
+    final Scan metaScan = new Scan()
+      .addFamily(HConstants.CATALOG_FAMILY)
+      .readVersions(1)
+      .setLimit(Optional.ofNullable(scanLimit).orElse(SCAN_LIMIT_DEFAULT) + 1);
+    Optional.ofNullable(scanStart)
+      .ifPresent(startRow -> metaScan.withStartRow(startRow, false));
+    buildScanFilter().ifPresent(metaScan::setFilter);
+    return metaScan;
+  }
+
+  private <T> void maybeAddParam(final QueryStringEncoder encoder, final String paramName,
 
 Review comment:
   nit: I think this could use a javadoc.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366022509
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final ExecutorService pool;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.pool = buildThreadPool();
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  @Override
+  public Iterator<RegionReplicaInfo> iterator() {
+    return limitIterator();
+  }
+
+  public LimitIterator<RegionReplicaInfo> limitIterator() {
+    logger.debug("initiating meta scan, {}", this);
+
+    final AsyncTable<ScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME, pool);
+    // TODO: buffering the entire result set seems unnecessary.
+    final List<RegionReplicaInfo> results = asyncTable.scanAll(buildScan()).join()
+      .stream()
+      .map(RegionReplicaInfo::from)
+      .flatMap(Collection::stream)
+      .collect(Collectors.toList());
+    return new LimitIterator<>(
+      results.iterator(), Optional.ofNullable(scanLimit).orElse(SCAN_LIMIT_DEFAULT));
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
+      .append("scanStart", scanStart)
+      .append("scanLimit", scanLimit)
+      .append("scanTable", scanTable)
+      .append("scanRegionState", scanRegionState)
+      .toString();
+  }
+
+  private static ExecutorService buildThreadPool() {
 
 Review comment:
   The docs say it's dangerous to not provide an executor service for callback handlers that do work. Without it, they'll execute on an RPC thread, which leads to starvation. On the other hand, it looks like `AsyncTableImpl` doesn't use provided `ExecutorService` when executing `getScanner`. Bleh, confusing. Will remove.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366109181
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.StreamSupport;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser {
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  public Results getResults() {
+    final AsyncTable<AdvancedScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME);
+    return new Results(asyncTable.getScanner(buildScan()));
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
+      .append("scanStart", scanStart)
+      .append("scanLimit", scanLimit)
+      .append("scanTable", scanTable)
+      .append("scanRegionState", scanRegionState)
+      .toString();
+  }
+
+  private static String resolveName(final HttpServletRequest request) {
+    return resolveRequestParameter(request, NAME_PARAM);
+  }
+
+  private Integer resolveScanLimit(final HttpServletRequest request) {
+    final String requestValueStr = resolveRequestParameter(request, SCAN_LIMIT_PARAM);
+    if (StringUtils.isBlank(requestValueStr)) {
+      return null;
 
 Review comment:
   nit: return SCAN_LIMIT_DEFAULT here and simplify callers?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366111041
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/RegionReplicaInfo.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A POJO that consolidates the information about a single region replica that's stored in meta.
+ */
+@InterfaceAudience.Private
+public final class RegionReplicaInfo {
+  private final byte[] row;
+  private final RegionInfo regionInfo;
+  private final RegionState.State regionState;
+  private final ServerName serverName;
+
+  private RegionReplicaInfo(final Result result, final HRegionLocation location) {
+    this.row = result != null ? result.getRow() : null;
+    this.regionInfo = location != null ? location.getRegion() : null;
+    this.regionState = (result != null && regionInfo != null)
+      ? RegionStateStore.getRegionState(result, regionInfo)
+      : null;
+    this.serverName = location != null ? location.getServerName() : null;
+  }
+
+  public static List<RegionReplicaInfo> from(final Result result) {
+    if (result == null) {
+      return Collections.singletonList(null);
+    }
+
+    final RegionLocations locations = MetaTableAccessor.getRegionLocations(result);
+    if (locations == null) {
+      return Collections.singletonList(null);
+    }
+
+    return StreamSupport.stream(locations.spliterator(), false)
+      .map(location -> new RegionReplicaInfo(result, location))
+      .collect(Collectors.toList());
+  }
+
+  public byte[] getRow() {
+    return row;
+  }
+
+  public RegionInfo getRegionInfo() {
+    return regionInfo;
+  }
+
+  public byte[] getRegionName() {
 
 Review comment:
   Thanks for the clean up.. this is what I had in mind.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#issuecomment-574912054
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 17s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 7 new or modified test files.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 45s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 15s |  master passed  |
   | +1 :green_heart: |  compile  |   3m  2s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 37s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 38s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   4m  0s |  master passed  |
   | +0 :ok: |  spotbugs  |   4m 18s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |  20m 17s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 24s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m  9s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  7s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  7s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   2m 40s |  root: The patch generated 0 new + 241 unchanged - 1 fixed = 241 total (was 242)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  3s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedjars  |   4m 46s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  16m 18s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   4m 15s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |  25m  0s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 247m  8s |  root in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   2m  5s |  The patch does not generate ASF License warnings.  |
   |  |   | 361m  5s |   |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1020 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile xml |
   | uname | Linux 7b0f968253c1 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1020/out/precommit/personality/provided.sh |
   | git revision | master / fd05aabf02 |
   | Default Java | 1.8.0_181 |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/6/testReport/ |
   | Max. process+thread count | 5634 (vs. ulimit of 10000) |
   | modules | C: hbase-client hbase-server . U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/6/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366108162
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.StreamSupport;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser {
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  public Results getResults() {
+    final AsyncTable<AdvancedScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME);
+    return new Results(asyncTable.getScanner(buildScan()));
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
+      .append("scanStart", scanStart)
+      .append("scanLimit", scanLimit)
+      .append("scanTable", scanTable)
+      .append("scanRegionState", scanRegionState)
+      .toString();
+  }
+
+  private static String resolveName(final HttpServletRequest request) {
+    return resolveRequestParameter(request, NAME_PARAM);
+  }
+
+  private Integer resolveScanLimit(final HttpServletRequest request) {
+    final String requestValueStr = resolveRequestParameter(request, SCAN_LIMIT_PARAM);
+    if (StringUtils.isBlank(requestValueStr)) {
+      return null;
+    }
+
+    final Integer requestValue = tryParseInt(requestValueStr);
+    if (requestValue == null) {
+      errorMessages.add(buildScanLimitMalformedErrorMessage(requestValueStr));
+      return null;
+    }
+    if (requestValue <= 0) {
+      errorMessages.add(buildScanLimitLTEZero(requestValue));
+      return SCAN_LIMIT_DEFAULT;
+    }
+
+    final int truncatedValue = Math.min(requestValue, SCAN_LIMIT_MAX);
+    if (requestValue != truncatedValue) {
+      errorMessages.add(buildScanLimitExceededErrorMessage(requestValue));
+    }
+    return truncatedValue;
+  }
+
+  private RegionState.State resolveScanRegionState(final HttpServletRequest request) {
+    final String requestValueStr = resolveRequestParameter(request, SCAN_REGION_STATE_PARAM);
+    if (requestValueStr == null) {
+      return null;
+    }
+    final RegionState.State requestValue = tryValueOf(RegionState.State.class, requestValueStr);
+    if (requestValue == null) {
+      errorMessages.add(buildScanRegionStateMalformedErrorMessage(requestValueStr));
+      return null;
+    }
+    return requestValue;
+  }
+
+  private static byte[] resolveScanStart(final HttpServletRequest request) {
+    final String requestValue = resolveRequestParameter(request, SCAN_START_PARAM);
+    if (requestValue == null) {
+      return null;
+    }
+    return Bytes.toBytesBinary(requestValue);
+  }
+
+  private static TableName resolveScanTable(final HttpServletRequest request) {
+    final String requestValue = resolveRequestParameter(request, SCAN_TABLE_PARAM);
+    if (requestValue == null) {
+      return null;
+    }
+    return TableName.valueOf(requestValue);
+  }
+
+  private static String resolveRequestParameter(final HttpServletRequest request,
+    final String param) {
+    if (request == null) {
+      return null;
+    }
+    final String requestValueStrEnc = request.getParameter(param);
+    if (StringUtils.isBlank(requestValueStrEnc)) {
+      return null;
+    }
+    return urlDecode(requestValueStrEnc);
+  }
+
+  private static Filter buildTableFilter(final TableName tableName) {
+    return new PrefixFilter(tableName.toBytes());
+  }
+
+  private static Filter buildScanRegionStateFilter(final RegionState.State state) {
+    return new SingleColumnValueFilter(
+      HConstants.CATALOG_FAMILY,
+      HConstants.STATE_QUALIFIER,
+      CompareOperator.EQUAL,
+      // use the same serialization strategy as found in MetaTableAccessor#addRegionStateToPut
+      Bytes.toBytes(state.name()));
+  }
+
+  private Filter buildScanFilter() {
+    if (scanTable == null && scanRegionState == null) {
+      return null;
+    }
+
+    final List<Filter> filters = new ArrayList<>(2);
+    if (scanTable != null) {
+      filters.add(buildTableFilter(scanTable));
+    }
+    if (scanRegionState != null) {
+      filters.add(buildScanRegionStateFilter(scanRegionState));
+    }
+    if (filters.size() == 1) {
+      return filters.get(0);
+    }
+    return new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
+  }
+
+  private Scan buildScan() {
+    final Scan metaScan = new Scan()
+      .addFamily(HConstants.CATALOG_FAMILY)
+      .readVersions(1)
+      .setLimit((scanLimit != null ? scanLimit : SCAN_LIMIT_DEFAULT) + 1);
+    if (scanStart != null) {
+      metaScan.withStartRow(scanStart, false);
+    }
+    final Filter filter = buildScanFilter();
+    if (filter != null) {
+      metaScan.setFilter(filter);
+    }
+    return metaScan;
+  }
+
+  /**
+   * Adds {@code value} to {@code encoder} under {@code paramName} when {@code value} is non-null.
+   */
+  private <T> void addParam(final QueryStringEncoder encoder, final String paramName,
 
 Review comment:
   nit: If toString() is the only needed call, no need to use generics / templatize it? Just pass an object?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] Apache-HBase commented on issue #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
Apache-HBase commented on issue #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#issuecomment-573265347
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |:----:|----------:|--------:|:--------|
   | +0 :ok: |  reexec  |   1m 58s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 8 new or modified test files.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 46s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 14s |  master passed  |
   | +1 :green_heart: |  compile  |   2m 58s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   2m 36s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   4m 37s |  branch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   3m 48s |  master passed  |
   | +0 :ok: |  spotbugs  |   3m 49s |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |  19m  2s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 22s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   4m 57s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m  0s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m  0s |  the patch passed  |
   | -1 :x: |  checkstyle  |   2m 33s |  root: The patch generated 10 new + 242 unchanged - 1 fixed = 252 total (was 243)  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  xml  |   0m  3s |  The patch has no ill-formed XML file.  |
   | +1 :green_heart: |  shadedjars  |   4m 35s |  patch has no errors when building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  15m 42s |  Patch does not cause any errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   3m 50s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |  19m 42s |  the patch passed  |
   ||| _ Other Tests _ |
   | -1 :x: |  unit  |  47m 56s |  root in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   0m 57s |  The patch does not generate ASF License warnings.  |
   |  |   | 151m 46s |   |
   
   
   | Reason | Tests |
   |-------:|:------|
   | Failed junit tests | hadoop.hbase.master.webapp.TestMetaBrowserNoCluster |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | Client=19.03.5 Server=19.03.5 base: https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hbase/pull/1020 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs shadedjars hadoopcheck hbaseanti checkstyle compile xml |
   | uname | Linux e612dcd65782 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | /home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1020/out/precommit/personality/provided.sh |
   | git revision | master / 8ca614857d |
   | Default Java | 1.8.0_181 |
   | checkstyle | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/1/artifact/out/diff-checkstyle-root.txt |
   | unit | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/1/artifact/out/patch-unit-root.txt |
   |  Test Results | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/1/testReport/ |
   | Max. process+thread count | 706 (vs. ulimit of 10000) |
   | modules | C: hbase-client hbase-server . U: . |
   | Console output | https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1020/1/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366091518
 
 

 ##########
 File path: hbase-server/pom.xml
 ##########
 @@ -436,6 +436,11 @@
       <artifactId>hamcrest-core</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-library</artifactId>
 
 Review comment:
   That's where the collection matchers live. See the test where I make assertions over the contents of lists using the `contains` method. That comes from the library. In reality, all the useful stuff for Hamcrest comes out of the `Matchers` class provided by hamcrest-library.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r365478702
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/RegionReplicaInfo.java
 ##########
 @@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A POJO that consolidates the information about a single region replica that's stored in meta.
+ */
+@InterfaceAudience.Private
+public final class RegionReplicaInfo {
+  private final byte[] row;
+  private final RegionInfo regionInfo;
+  private final byte[] regionName;
+  private final byte[] startKey;
+  private final byte[] endKey;
+  private final Integer replicaId;
+  private final RegionState.State regionState;
+  private final ServerName serverName;
+
+  private RegionReplicaInfo(final Result result, final HRegionLocation location) {
+    final Optional<Result> maybeResult = Optional.ofNullable(result);
+    final Optional<HRegionLocation> maybeLocation = Optional.ofNullable(location);
+    final Optional<RegionInfo> maybeRegionInfo = maybeLocation.map(HRegionLocation::getRegion);
+
+    this.row = maybeResult.map(Result::getRow).orElse(null);
 
 Review comment:
   nit: the accessors methods could work directly on regioninfo object ? (less code).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366074442
 
 

 ##########
 File path: hbase-client/src/main/java/org/apache/hadoop/hbase/RegionLocations.java
 ##########
 @@ -31,7 +34,7 @@
  * (assuming small number of locations)
  */
 @InterfaceAudience.Private
-public class RegionLocations {
+public class RegionLocations implements Iterable<HRegionLocation> {
 
 Review comment:
   Good

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r365480306
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/ClearUserNamespacesAndTablesRule.java
 ##########
 @@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.StringJoiner;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.client.AsyncAdmin;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.junit.Rule;
+import org.junit.rules.ExternalResource;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link Rule} that clears all user tables and namespaces before the test executes.
+ */
+public class ClearUserNamespacesAndTablesRule extends ExternalResource {
 
 Review comment:
   nice.. for some reason, I thought we already had this somewhere, looked around but didn't find one.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366092986
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final ExecutorService pool;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.pool = buildThreadPool();
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
 
 Review comment:
   To be accessed from `table.jsp`, yes, I believe so.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r365481059
 
 

 ##########
 File path: hbase-server/src/test/java/org/apache/hadoop/hbase/TestClusterRule.java
 ##########
 @@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.junit.Rule;
+import org.junit.rules.ExternalResource;
+
+/**
+ * A {@link Rule} that manages an instance of the {@link MiniHBaseCluster}.
+ */
+public class TestClusterRule extends ExternalResource {
+  private final HBaseTestingUtility testingUtility;
+  private final StartMiniClusterOption miniClusterOptions;
+
+  private MiniHBaseCluster miniCluster;
+
+  public TestClusterRule() {
+    this(StartMiniClusterOption.builder().build());
+  }
+
+  public TestClusterRule(final StartMiniClusterOption miniClusterOptions) {
+    this.testingUtility = new HBaseTestingUtility();
+    this.miniClusterOptions = miniClusterOptions;
+  }
+
+  public CompletableFuture<AsyncConnection> createConnection() {
+    if (miniCluster == null) { throw new IllegalStateException("test cluster not initialized"); }
 
 Review comment:
   nit: You'll probably run into checkstyle issues with inline if blocks.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366046820
 
 

 ##########
 File path: hbase-server/src/main/resources/hbase-webapps/master/table.jsp
 ##########
 @@ -129,6 +151,7 @@
   pageContext.setAttribute("pageTitle", pageTitle);
   AsyncConnection connection = ConnectionFactory.createAsyncConnection(master.getConfiguration()).get();
 
 Review comment:
   Sure, I think that's a fine improvement.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366112562
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.StreamSupport;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser {
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  public Results getResults() {
+    final AsyncTable<AdvancedScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME);
+    return new Results(asyncTable.getScanner(buildScan()));
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
+      .append("scanStart", scanStart)
+      .append("scanLimit", scanLimit)
+      .append("scanTable", scanTable)
+      .append("scanRegionState", scanRegionState)
+      .toString();
+  }
+
+  private static String resolveName(final HttpServletRequest request) {
+    return resolveRequestParameter(request, NAME_PARAM);
+  }
+
+  private Integer resolveScanLimit(final HttpServletRequest request) {
+    final String requestValueStr = resolveRequestParameter(request, SCAN_LIMIT_PARAM);
+    if (StringUtils.isBlank(requestValueStr)) {
+      return null;
+    }
+
+    final Integer requestValue = tryParseInt(requestValueStr);
+    if (requestValue == null) {
+      errorMessages.add(buildScanLimitMalformedErrorMessage(requestValueStr));
+      return null;
+    }
+    if (requestValue <= 0) {
+      errorMessages.add(buildScanLimitLTEZero(requestValue));
+      return SCAN_LIMIT_DEFAULT;
+    }
+
+    final int truncatedValue = Math.min(requestValue, SCAN_LIMIT_MAX);
+    if (requestValue != truncatedValue) {
+      errorMessages.add(buildScanLimitExceededErrorMessage(requestValue));
+    }
+    return truncatedValue;
+  }
+
+  private RegionState.State resolveScanRegionState(final HttpServletRequest request) {
+    final String requestValueStr = resolveRequestParameter(request, SCAN_REGION_STATE_PARAM);
+    if (requestValueStr == null) {
+      return null;
+    }
+    final RegionState.State requestValue = tryValueOf(RegionState.State.class, requestValueStr);
+    if (requestValue == null) {
+      errorMessages.add(buildScanRegionStateMalformedErrorMessage(requestValueStr));
+      return null;
+    }
+    return requestValue;
+  }
+
+  private static byte[] resolveScanStart(final HttpServletRequest request) {
+    final String requestValue = resolveRequestParameter(request, SCAN_START_PARAM);
+    if (requestValue == null) {
+      return null;
+    }
+    return Bytes.toBytesBinary(requestValue);
+  }
+
+  private static TableName resolveScanTable(final HttpServletRequest request) {
+    final String requestValue = resolveRequestParameter(request, SCAN_TABLE_PARAM);
+    if (requestValue == null) {
+      return null;
+    }
+    return TableName.valueOf(requestValue);
+  }
+
+  private static String resolveRequestParameter(final HttpServletRequest request,
+    final String param) {
+    if (request == null) {
+      return null;
+    }
+    final String requestValueStrEnc = request.getParameter(param);
+    if (StringUtils.isBlank(requestValueStrEnc)) {
+      return null;
+    }
+    return urlDecode(requestValueStrEnc);
+  }
+
+  private static Filter buildTableFilter(final TableName tableName) {
+    return new PrefixFilter(tableName.toBytes());
+  }
+
+  private static Filter buildScanRegionStateFilter(final RegionState.State state) {
+    return new SingleColumnValueFilter(
+      HConstants.CATALOG_FAMILY,
+      HConstants.STATE_QUALIFIER,
+      CompareOperator.EQUAL,
+      // use the same serialization strategy as found in MetaTableAccessor#addRegionStateToPut
+      Bytes.toBytes(state.name()));
+  }
+
+  private Filter buildScanFilter() {
+    if (scanTable == null && scanRegionState == null) {
+      return null;
+    }
+
+    final List<Filter> filters = new ArrayList<>(2);
+    if (scanTable != null) {
+      filters.add(buildTableFilter(scanTable));
+    }
+    if (scanRegionState != null) {
+      filters.add(buildScanRegionStateFilter(scanRegionState));
+    }
+    if (filters.size() == 1) {
+      return filters.get(0);
+    }
+    return new FilterList(FilterList.Operator.MUST_PASS_ALL, filters);
+  }
+
+  private Scan buildScan() {
+    final Scan metaScan = new Scan()
+      .addFamily(HConstants.CATALOG_FAMILY)
+      .readVersions(1)
+      .setLimit((scanLimit != null ? scanLimit : SCAN_LIMIT_DEFAULT) + 1);
 
 Review comment:
   I think, the +1 part is pretty subtle.. add a comment that it is needed for pagination hasNext() support?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r365475813
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final ExecutorService pool;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.pool = buildThreadPool();
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  @Override
+  public Iterator<RegionReplicaInfo> iterator() {
+    return limitIterator();
+  }
+
+  public LimitIterator<RegionReplicaInfo> limitIterator() {
+    logger.debug("initiating meta scan, {}", this);
+
+    final AsyncTable<ScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME, pool);
+    // TODO: buffering the entire result set seems unnecessary.
 
 Review comment:
   Doesn't ResultScanner support an iterator()? Wrap it with a limit iterator instead of scan all? This way, I guess 10 such requests with max limit can be heavy for the master.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366082414
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/RegionReplicaInfo.java
 ##########
 @@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.RegionLocations;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * A POJO that consolidates the information about a single region replica that's stored in meta.
 
 Review comment:
   It has state and serverName? A location has RI and SN? Should this subclass one of them then?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
bharathv commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r365477051
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final ExecutorService pool;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.pool = buildThreadPool();
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  @Override
+  public Iterator<RegionReplicaInfo> iterator() {
+    return limitIterator();
+  }
+
+  public LimitIterator<RegionReplicaInfo> limitIterator() {
+    logger.debug("initiating meta scan, {}", this);
+
+    final AsyncTable<ScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME, pool);
+    // TODO: buffering the entire result set seems unnecessary.
+    final List<RegionReplicaInfo> results = asyncTable.scanAll(buildScan()).join()
+      .stream()
+      .map(RegionReplicaInfo::from)
+      .flatMap(Collection::stream)
+      .collect(Collectors.toList());
+    return new LimitIterator<>(
+      results.iterator(), Optional.ofNullable(scanLimit).orElse(SCAN_LIMIT_DEFAULT));
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
+      .append("scanStart", scanStart)
+      .append("scanLimit", scanLimit)
+      .append("scanTable", scanTable)
+      .append("scanRegionState", scanRegionState)
+      .toString();
+  }
+
+  private static ExecutorService buildThreadPool() {
+    return Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
+      .setNameFormat("MetaBrowser-%d")
+      .setDaemon(true)
+      .setUncaughtExceptionHandler(
+        (thread, throwable) -> logger.info("Error in worker thread, {}", throwable.getMessage()))
+      .build());
+  }
+
+  private static String resolveName(final HttpServletRequest request) {
+    return Optional.ofNullable(request)
+      .map(req -> req.getParameter(NAME_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode)
+      .orElse(null);
+  }
+
+  private Integer resolveScanLimit(final HttpServletRequest request) {
+    final Optional<String> requestValueStr = Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_LIMIT_PARAM))
+      .filter(StringUtils::isNotBlank);
+    if (!requestValueStr.isPresent()) { return null; }
+
+    final Integer requestValue = requestValueStr
+      .flatMap(MetaBrowser::tryParseInt)
+      .orElse(null);
+    if (requestValue == null) {
+      errorMessages.add(buildScanLimitMalformedErrorMessage(requestValueStr.get()));
+      return null;
+    }
+    if (requestValue <= 0) {
+      errorMessages.add(buildScanLimitLTEZero(requestValue));
+      return SCAN_LIMIT_DEFAULT;
+    }
+
+    final int truncatedValue = Math.min(requestValue, SCAN_LIMIT_MAX);
+    if (requestValue != truncatedValue) {
+      errorMessages.add(buildScanLimitExceededErrorMessage(requestValue));
+    }
+    return truncatedValue;
+  }
+
+  private RegionState.State resolveScanRegionState(final HttpServletRequest request) {
+    final Optional<String> requestValueStr = Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_REGION_STATE_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode);
+    if (!requestValueStr.isPresent()) { return null; }
+
+    final RegionState.State requestValue = requestValueStr
+      .flatMap(val -> tryValueOf(RegionState.State.class, val))
+      .orElse(null);
+    if (requestValue == null) {
+      errorMessages.add(buildScanRegionStateMalformedErrorMessage(requestValueStr.get()));
+      return null;
+    }
+    return requestValue;
+  }
+
+  private static byte[] resolveScanStart(final HttpServletRequest request) {
+    return Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_START_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode)
+      .map(Bytes::toBytesBinary)
+      .orElse(null);
+  }
+
+  private static TableName resolveScanTable(final HttpServletRequest request) {
+    return Optional.ofNullable(request)
+      .map(req -> req.getParameter(SCAN_TABLE_PARAM))
+      .filter(StringUtils::isNotBlank)
+      .map(MetaBrowser::urlDecode)
+      .map(TableName::valueOf)
+      .orElse(null);
+  }
+
+  private static Filter buildTableFilter(final TableName tableName) {
+    return new PrefixFilter(tableName.toBytes());
+  }
+
+  private static Filter buildScanRegionStateFilter(final RegionState.State state) {
+    return new SingleColumnValueFilter(
+      HConstants.CATALOG_FAMILY,
+      HConstants.TABLE_STATE_QUALIFIER,
+      CompareOperator.EQUAL,
+      // use the same serialization strategy as found in MetaTableAccessor#addRegionStateToPut
+      Bytes.toBytes(state.name()));
+  }
+
+  private Optional<Filter> buildScanFilter() {
+    if (scanTable == null && scanRegionState == null) {
+      return Optional.empty();
+    }
+
+    final List<Filter> filters = new ArrayList<>(2);
+    Optional.ofNullable(scanTable)
+      .map(MetaBrowser::buildTableFilter)
+      .ifPresent(filters::add);
+    Optional.ofNullable(scanRegionState)
 
 Review comment:
   haha , functional style is not as readable :D (my personal opinion)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
ndimiduk commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366505312
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.stream.StreamSupport;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AdvancedScanResultConsumer;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.apache.hbase.thirdparty.com.google.common.collect.Iterators;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser {
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
+
+  private final AsyncConnection connection;
+  private final HttpServletRequest request;
+  private final List<String> errorMessages;
+  private final String name;
+  private final Integer scanLimit;
+  private final RegionState.State scanRegionState;
+  private final byte[] scanStart;
+  private final TableName scanTable;
+
+  public MetaBrowser(final AsyncConnection connection, final HttpServletRequest request) {
+    this.connection = connection;
+    this.request = request;
+    this.errorMessages = new LinkedList<>();
+    this.name = resolveName(request);
+    this.scanLimit = resolveScanLimit(request);
+    this.scanRegionState = resolveScanRegionState(request);
+    this.scanStart = resolveScanStart(request);
+    this.scanTable = resolveScanTable(request);
+  }
+
+  public List<String> getErrorMessages() {
+    return errorMessages;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public Integer getScanLimit() {
+    return scanLimit;
+  }
+
+  public byte[] getScanStart() {
+    return scanStart;
+  }
+
+  public RegionState.State getScanRegionState() {
+    return scanRegionState;
+  }
+
+  public TableName getScanTable() {
+    return scanTable;
+  }
+
+  public Results getResults() {
+    final AsyncTable<AdvancedScanResultConsumer> asyncTable =
+      connection.getTable(TableName.META_TABLE_NAME);
+    return new Results(asyncTable.getScanner(buildScan()));
+  }
+
+  @Override
+  public String toString() {
+    return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
+      .append("scanStart", scanStart)
+      .append("scanLimit", scanLimit)
+      .append("scanTable", scanTable)
+      .append("scanRegionState", scanRegionState)
+      .toString();
+  }
+
+  private static String resolveName(final HttpServletRequest request) {
+    return resolveRequestParameter(request, NAME_PARAM);
+  }
+
+  private Integer resolveScanLimit(final HttpServletRequest request) {
+    final String requestValueStr = resolveRequestParameter(request, SCAN_LIMIT_PARAM);
+    if (StringUtils.isBlank(requestValueStr)) {
+      return null;
 
 Review comment:
   Oh, no we cannot. We need to differentiate between a value specified or not.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [hbase] saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui

Posted by GitBox <gi...@apache.org>.
saintstack commented on a change in pull request #1020: HBASE-23653 Expose content of meta table in web ui
URL: https://github.com/apache/hbase/pull/1020#discussion_r366080164
 
 

 ##########
 File path: hbase-server/src/main/java/org/apache/hadoop/hbase/master/webapp/MetaBrowser.java
 ##########
 @@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.webapp;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+import org.apache.hadoop.hbase.CompareOperator;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.AsyncConnection;
+import org.apache.hadoop.hbase.client.AsyncTable;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.ScanResultConsumer;
+import org.apache.hadoop.hbase.filter.Filter;
+import org.apache.hadoop.hbase.filter.FilterList;
+import org.apache.hadoop.hbase.filter.PrefixFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hbase.thirdparty.io.netty.handler.codec.http.QueryStringEncoder;
+
+/**
+ * A support class for the "Meta Entries" section in
+ * {@code resources/hbase-webapps/master/table.jsp}.
+ */
+@InterfaceAudience.Private
+public class MetaBrowser implements Iterable<RegionReplicaInfo> {
+  private static final Logger logger = LoggerFactory.getLogger(MetaBrowser.class);
+
+  public static final String NAME_PARAM = "name";
+  public static final String SCAN_LIMIT_PARAM = "scan_limit";
+  public static final String SCAN_REGION_STATE_PARAM = "scan_region_state";
+  public static final String SCAN_START_PARAM = "scan_start";
+  public static final String SCAN_TABLE_PARAM = "scan_table";
+
+  public static final int SCAN_LIMIT_DEFAULT = 10;
+  public static final int SCAN_LIMIT_MAX = 10_000;
 
 Review comment:
   Can I page through a 100k hbase:meta ?  

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services