HBASE-27765 Add biggest cell related info into web ui (#5151)
Signed-off-by: Duo Zhang <zhangduo@apache.org>
(cherry picked from commit a363b76d81
)
This commit is contained in:
parent
cc00b8b8a8
commit
85793655d3
|
@ -28,6 +28,9 @@
|
|||
org.apache.hadoop.hbase.client.RegionInfo;
|
||||
org.apache.hadoop.hbase.client.RegionInfoDisplay;
|
||||
org.apache.hadoop.hbase.regionserver.Region;
|
||||
org.apache.hadoop.hbase.regionserver.HStoreFile;
|
||||
org.apache.hadoop.hbase.regionserver.Store;
|
||||
org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
org.apache.hadoop.hbase.ServerName;
|
||||
org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
|
@ -168,6 +171,7 @@
|
|||
<th>Index Size</th>
|
||||
<th>Bloom Size</th>
|
||||
<th>Data Locality</th>
|
||||
<th>Len Of Biggest Cell</th>
|
||||
</tr>
|
||||
</thead>
|
||||
|
||||
|
@ -207,6 +211,21 @@
|
|||
* TraditionalBinaryPrefix.KILO.value, "B", 1);
|
||||
}
|
||||
}
|
||||
long lenOfBiggestCellInRegion = -1L;
|
||||
Region region = regionServer.getRegion(r.getEncodedName());
|
||||
if (region != null) {
|
||||
List<? extends Store> stores = region.getStores();
|
||||
for (Store store : stores) {
|
||||
Collection<? extends StoreFile> storeFiles = store.getStorefiles();
|
||||
for (StoreFile sf : storeFiles) {
|
||||
long lenOfBiggestCell = ((HStoreFile)sf).getFileInfo().getHFileInfo().getLenOfBiggestCell();
|
||||
if (lenOfBiggestCellInRegion < lenOfBiggestCell) {
|
||||
lenOfBiggestCellInRegion = lenOfBiggestCell;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
</%java>
|
||||
<td><a href="region.jsp?name=<% r.getEncodedName() %>"><% displayName %></a></td>
|
||||
<%if load != null %>
|
||||
|
@ -217,6 +236,7 @@
|
|||
<td><% indexSizeStr %></td>
|
||||
<td><% bloomSizeStr %></td>
|
||||
<td><% load.getDataLocality() %></td>
|
||||
<td><% String.format("%,1d", lenOfBiggestCellInRegion) %></td>
|
||||
</%if>
|
||||
</tr>
|
||||
</%for>
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.commons.io.IOUtils;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.io.crypto.Cipher;
|
||||
import org.apache.hadoop.hbase.io.crypto.Encryption;
|
||||
|
@ -74,6 +75,8 @@ public class HFileInfo implements SortedMap<byte[], byte[]> {
|
|||
static final byte[] AVG_VALUE_LEN = Bytes.toBytes(RESERVED_PREFIX + "AVG_VALUE_LEN");
|
||||
static final byte[] CREATE_TIME_TS = Bytes.toBytes(RESERVED_PREFIX + "CREATE_TIME_TS");
|
||||
static final byte[] TAGS_COMPRESSED = Bytes.toBytes(RESERVED_PREFIX + "TAGS_COMPRESSED");
|
||||
static final byte[] KEY_OF_BIGGEST_CELL = Bytes.toBytes(RESERVED_PREFIX + "KEY_OF_BIGGEST_CELL");
|
||||
static final byte[] LEN_OF_BIGGEST_CELL = Bytes.toBytes(RESERVED_PREFIX + "LEN_OF_BIGGEST_CELL");
|
||||
public static final byte[] MAX_TAGS_LEN = Bytes.toBytes(RESERVED_PREFIX + "MAX_TAGS_LEN");
|
||||
private final SortedMap<byte[], byte[]> map = new TreeMap<>(Bytes.BYTES_COMPARATOR);
|
||||
|
||||
|
@ -93,6 +96,10 @@ public class HFileInfo implements SortedMap<byte[], byte[]> {
|
|||
private int avgKeyLen = -1;
|
||||
/** Average value length read from file info */
|
||||
private int avgValueLen = -1;
|
||||
/** Biggest Cell in the file, key only. Filled in when we read in the file info */
|
||||
private Cell biggestCell = null;
|
||||
/** Length of the biggest Cell */
|
||||
private long lenOfBiggestCell = -1;
|
||||
private boolean includesMemstoreTS = false;
|
||||
private boolean decodeMemstoreTS = false;
|
||||
|
||||
|
@ -435,6 +442,10 @@ public class HFileInfo implements SortedMap<byte[], byte[]> {
|
|||
if (get(HFileInfo.LASTKEY) != null) {
|
||||
lastKeyCell = new KeyValue.KeyOnlyKeyValue(get(HFileInfo.LASTKEY));
|
||||
}
|
||||
if (get(HFileInfo.KEY_OF_BIGGEST_CELL) != null) {
|
||||
biggestCell = new KeyValue.KeyOnlyKeyValue(get(HFileInfo.KEY_OF_BIGGEST_CELL));
|
||||
lenOfBiggestCell = Bytes.toLong(get(HFileInfo.LEN_OF_BIGGEST_CELL));
|
||||
}
|
||||
avgKeyLen = Bytes.toInt(get(HFileInfo.AVG_KEY_LEN));
|
||||
avgValueLen = Bytes.toInt(get(HFileInfo.AVG_VALUE_LEN));
|
||||
byte[] keyValueFormatVersion = get(HFileWriterImpl.KEY_VALUE_VERSION);
|
||||
|
@ -511,6 +522,14 @@ public class HFileInfo implements SortedMap<byte[], byte[]> {
|
|||
return avgValueLen;
|
||||
}
|
||||
|
||||
public String getKeyOfBiggestCell() {
|
||||
return CellUtil.toString(biggestCell, false);
|
||||
}
|
||||
|
||||
public long getLenOfBiggestCell() {
|
||||
return lenOfBiggestCell;
|
||||
}
|
||||
|
||||
public boolean shouldIncludeMemStoreTS() {
|
||||
return includesMemstoreTS;
|
||||
}
|
||||
|
|
|
@ -34,7 +34,9 @@ import org.apache.hadoop.fs.permission.FsPermission;
|
|||
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.CellComparator;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.KeyValueUtil;
|
||||
import org.apache.hadoop.hbase.MetaCellComparator;
|
||||
import org.apache.hadoop.hbase.PrivateCellUtil;
|
||||
|
@ -93,6 +95,11 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
/** Used for calculating the average value length. */
|
||||
protected long totalValueLength = 0;
|
||||
|
||||
/** Len of the biggest cell. */
|
||||
protected long lenOfBiggestCell = 0;
|
||||
/** Key of the biggest cell. */
|
||||
protected byte[] keyOfBiggestCell;
|
||||
|
||||
/** Total uncompressed bytes, maybe calculate a compression ratio later. */
|
||||
protected long totalUncompressedBytes = 0;
|
||||
|
||||
|
@ -741,7 +748,10 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
|
||||
totalKeyLength += PrivateCellUtil.estimatedSerializedSizeOfKey(cell);
|
||||
totalValueLength += cell.getValueLength();
|
||||
|
||||
if (lenOfBiggestCell < PrivateCellUtil.estimatedSerializedSizeOf(cell)) {
|
||||
lenOfBiggestCell = PrivateCellUtil.estimatedSerializedSizeOf(cell);
|
||||
keyOfBiggestCell = PrivateCellUtil.getCellKeySerializedAsKeyValueKey(cell);
|
||||
}
|
||||
// Are we the first key in this block?
|
||||
if (firstCellInBlock == null) {
|
||||
// If cell is big, block will be closed and this firstCellInBlock reference will only last
|
||||
|
@ -795,6 +805,16 @@ public class HFileWriterImpl implements HFile.Writer {
|
|||
// Average value length.
|
||||
int avgValueLen = entryCount == 0 ? 0 : (int) (totalValueLength / entryCount);
|
||||
fileInfo.append(HFileInfo.AVG_VALUE_LEN, Bytes.toBytes(avgValueLen), false);
|
||||
|
||||
// Biggest cell.
|
||||
if (keyOfBiggestCell != null) {
|
||||
fileInfo.append(HFileInfo.KEY_OF_BIGGEST_CELL, keyOfBiggestCell, false);
|
||||
fileInfo.append(HFileInfo.LEN_OF_BIGGEST_CELL, Bytes.toBytes(lenOfBiggestCell), false);
|
||||
LOG.debug("Len of the biggest cell in {} is {}, key is {}",
|
||||
this.getPath() == null ? "" : this.getPath().toString(), lenOfBiggestCell,
|
||||
CellUtil.toString(new KeyValue.KeyOnlyKeyValue(keyOfBiggestCell), false));
|
||||
}
|
||||
|
||||
if (hFileContext.isIncludesTags()) {
|
||||
// When tags are not being written in this file, MAX_TAGS_LEN is excluded
|
||||
// from the FileInfo
|
||||
|
|
|
@ -23,6 +23,7 @@
|
|||
import="java.util.List"
|
||||
import="org.apache.hadoop.hbase.client.RegionInfoDisplay"
|
||||
import="org.apache.hadoop.hbase.regionserver.HRegionServer"
|
||||
import="org.apache.hadoop.hbase.regionserver.HStoreFile"
|
||||
import="org.apache.hadoop.hbase.regionserver.Region"
|
||||
import="org.apache.hadoop.hbase.regionserver.Store"
|
||||
import="org.apache.hadoop.hbase.regionserver.StoreFile"
|
||||
|
@ -69,12 +70,16 @@
|
|||
<th>Store File</th>
|
||||
<th>Size (MB)</th>
|
||||
<th>Modification time</th>
|
||||
<th>Len Of Biggest Cell</th>
|
||||
<th>Key Of Biggest Cell</th>
|
||||
</tr>
|
||||
<% for(StoreFile sf : storeFiles) { %>
|
||||
<tr>
|
||||
<td><a href="storeFile.jsp?name=<%= sf.getEncodedPath() %>"><%= sf.getPath() %></a></td>
|
||||
<td><%= (int) (rs.getFileSystem().getLength(sf.getPath()) / 1024 / 1024) %></td>
|
||||
<td><%= new Date(sf.getModificationTimestamp()) %></td>
|
||||
<td><%= String.format("%,1d", ((HStoreFile)sf).getFileInfo().getHFileInfo().getLenOfBiggestCell()) %></td>
|
||||
<td><%= ((HStoreFile)sf).getFileInfo().getHFileInfo().getKeyOfBiggestCell() %></td>
|
||||
</tr>
|
||||
<% } %>
|
||||
|
||||
|
|
Loading…
Reference in New Issue