ndimiduk commented on code in PR #6327:
URL: https://github.com/apache/hbase/pull/6327#discussion_r1822248302


##########
hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/row/stats/RowStatisticsImpl.java:
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.coprocessor.example.row.stats;
+
+import java.util.Map;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.RawCellBuilder;
+import 
org.apache.hadoop.hbase.coprocessor.example.row.stats.utils.RowStatisticsUtil;
+import org.apache.hadoop.hbase.regionserver.Shipper;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.GsonUtil;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.gson.Gson;
+import org.apache.hbase.thirdparty.com.google.gson.JsonObject;
+
+/**
+ * Holder for accumulating row statistics in {@link 
RowStatisticsCompactionObserver} Creates various
+ * cell, row, and total stats.
+ */
+@InterfaceAudience.Private
+public class RowStatisticsImpl implements RowStatistics {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RowStatisticsImpl.class);
+  private static final Gson GSON = GsonUtil.createGson().create();
+
+  //
+  // Transient fields which are not included in gson serialization
+  //
+  private final transient long blockSize;
+  private final transient long maxCacheSize;
+  private transient int rowCells;
+  private transient long rowBytes;
+  private transient byte[] largestRow;
+  private transient Cell largestCell;
+  private final transient boolean isMajor;
+  private final transient SizeBucketTracker rowSizeBuckets;
+  private final transient SizeBucketTracker valueSizeBuckets;
+
+  // We don't need to clone anything until shipped() is called on scanner.
+  // To avoid allocations, we keep a reference until that point
+  private transient Cell largestRowRef;
+  private transient Cell largestCellRef;
+  //
+  // Non-transient fields which are included in gson
+  //
+  private final String table;
+  private final String region;
+  private final String columnFamily;
+  private long largestRowBytes;
+  private int largestRowCells;
+  private long largestCellBytes;
+  private int cellsLargerThanOneBlock;
+  private int rowsLargerThanOneBlock;
+  private int cellsLargerThanMaxCacheSize;
+  private int totalDeletes;
+  private int totalCells;
+  private int totalRows;
+  private long totalBytes;
+
+  RowStatisticsImpl(String table, String encodedRegion, String columnFamily, 
long blockSize,
+    long maxCacheSize, boolean isMajor) {
+    this.table = table;
+    this.region = encodedRegion;
+    this.columnFamily = columnFamily;
+    this.blockSize = blockSize;
+    this.maxCacheSize = maxCacheSize;
+    this.isMajor = isMajor;
+    this.rowSizeBuckets = new SizeBucketTracker();
+    this.valueSizeBuckets = new SizeBucketTracker();
+  }
+
+  public void handleRowChanged(Cell lastCell) {
+    if (rowBytes > largestRowBytes) {
+      largestRowRef = lastCell;
+      largestRowBytes = rowBytes;
+      largestRowCells = rowCells;
+    }
+    if (rowBytes > blockSize) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("RowTooLarge: rowBytes={}, blockSize={}, table={}, 
rowKey={}", rowBytes,
+          blockSize, table, Bytes.toStringBinary(lastCell.getRowArray(), 
lastCell.getRowOffset(),
+            lastCell.getRowLength()));
+      }
+      rowsLargerThanOneBlock++;
+    }
+    rowSizeBuckets.add(rowBytes);
+    rowBytes = 0;
+    rowCells = 0;
+    totalRows++;
+  }
+
+  public void consumeCell(Cell cell) {
+    int cellSize = cell.getSerializedSize();
+
+    rowBytes += cellSize;
+    rowCells++;
+
+    boolean tooLarge = false;
+    if (cellSize > maxCacheSize) {
+      cellsLargerThanMaxCacheSize++;
+      tooLarge = true;
+    }
+    if (cellSize > blockSize) {
+      cellsLargerThanOneBlock++;
+      tooLarge = true;
+    }
+
+    if (tooLarge && LOG.isDebugEnabled()) {
+      LOG.debug("CellTooLarge: size={}, blockSize={}, maxCacheSize={}, 
table={}, cell={}", cellSize,
+        blockSize, maxCacheSize, table, CellUtil.toString(cell, false));
+    }
+
+    if (cellSize > largestCellBytes) {
+      largestCellRef = cell;
+      largestCellBytes = cellSize;
+    }
+    valueSizeBuckets.add(cell.getValueLength());
+
+    totalCells++;
+    if (CellUtil.isDelete(cell)) {
+      totalDeletes++;
+    }
+    totalBytes += cellSize;
+  }
+
+  /**
+   * Clone the cell refs so they can be cleaned up by {@link 
Shipper#shipped()}. Doing this lazily
+   * here, rather than eagerly in the above two methods can save us on some 
allocations. We might
+   * change the largestCell/largestRow multiple times between shipped() calls.
+   */
+  public void shipped(RawCellBuilder cellBuilder) {
+    if (largestRowRef != null) {
+      largestRow = CellUtil.cloneRow(largestRowRef);
+      largestRowRef = null;
+    }
+    if (largestCellRef != null) {
+      largestCell = RowStatisticsUtil.cloneWithoutValue(cellBuilder, 
largestCellRef);
+      largestCellRef = null;
+    }
+  }
+
+  public String getTable() {
+    return table;
+  }
+
+  public String getRegion() {
+    return region;
+  }
+
+  public String getColumnFamily() {
+    return columnFamily;
+  }
+
+  public boolean isMajor() {
+    return isMajor;
+  }
+
+  public byte[] getLargestRow() {
+    return largestRow;
+  }
+
+  public String getLargestRowAsString() {
+    return Bytes.toStringBinary(getLargestRow());
+  }
+
+  public long getLargestRowNumBytes() {
+    return largestRowBytes;
+  }
+
+  public int getLargestRowCellsCount() {
+    return largestRowCells;
+  }
+
+  public Cell getLargestCell() {
+    return largestCell;
+  }
+
+  public String getLargestCellAsString() {
+    return CellUtil.toString(getLargestCell(), false);
+  }
+
+  public long getLargestCellNumBytes() {
+    return largestCellBytes;
+  }
+
+  public int getCellsLargerThanOneBlockCount() {
+    return cellsLargerThanOneBlock;
+  }
+
+  public int getRowsLargerThanOneBlockCount() {
+    return rowsLargerThanOneBlock;
+  }
+
+  public int getCellsLargerThanMaxCacheSizeCount() {
+    return cellsLargerThanMaxCacheSize;
+  }
+
+  public int getTotalDeletesCount() {
+    return totalDeletes;
+  }
+
+  public int getTotalCellsCount() {
+    return totalCells;
+  }
+
+  public int getTotalRowsCount() {
+    return totalRows;
+  }
+
+  public long getTotalBytes() {
+    return totalBytes;
+  }
+
+  public Map<String, Long> getRowSizeBuckets() {
+    return rowSizeBuckets.toMap();
+  }
+
+  public Map<String, Long> getValueSizeBuckets() {
+    return valueSizeBuckets.toMap();
+  }
+
+  @Override
+  public String toString() {
+    return ("RowStatistics{" + "largestRow=" + Bytes.toStringBinary(largestRow)

Review Comment:
   Instead of building these by hand, better to use commons-lang3 
ToStringBuilder with its SHORT_PREFIX_STYLE.



##########
hbase-examples/pom.xml:
##########
@@ -49,6 +49,26 @@
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-protocol-shaded</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hbase.thirdparty</groupId>
+      <artifactId>hbase-shaded-miscellaneous</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.lmax</groupId>
+      <artifactId>disruptor</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-1.2-api</artifactId>

Review Comment:
   Sorry, I don't think that we should have a dependency on log4j-api. We use 
slf4j-api as our facad over the logger. A quick Ctl-f on this PR doesn't find 
any reference to "log4j" other than this line. I think it should be removed 
entirely.
   
   Am I wrong about that? Do your tests run with this dependency removed? Does 
the dependency plugin complain about an undeclared dependency?



##########
hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/row/stats/RowStatisticsCompactionObserver.java:
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.coprocessor.example.row.stats;
+
+import static 
org.apache.hadoop.hbase.coprocessor.example.row.stats.utils.RowStatisticsTableUtil.CF;
+import static 
org.apache.hadoop.hbase.coprocessor.example.row.stats.utils.RowStatisticsTableUtil.NAMESPACE;
+import static 
org.apache.hadoop.hbase.coprocessor.example.row.stats.utils.RowStatisticsTableUtil.NAMESPACED_TABLE_NAME;
+import static 
org.apache.hadoop.hbase.coprocessor.example.row.stats.utils.RowStatisticsTableUtil.TABLE_RECORDER_KEY;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.ExtendedCell;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.RawCellBuilder;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import 
org.apache.hadoop.hbase.coprocessor.example.row.stats.recorder.RowStatisticsRecorder;
+import 
org.apache.hadoop.hbase.coprocessor.example.row.stats.recorder.RowStatisticsTableRecorder;
+import 
org.apache.hadoop.hbase.coprocessor.example.row.stats.utils.RowStatisticsUtil;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
+import org.apache.hadoop.hbase.regionserver.Shipper;
+import org.apache.hadoop.hbase.regionserver.Store;
+import 
org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class RowStatisticsCompactionObserver
+  implements RegionCoprocessor, RegionObserver, MasterCoprocessor, 
MasterObserver {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RowStatisticsCompactionObserver.class);
+
+  // From private field BucketAllocator.DEFAULT_BUCKET_SIZES
+  private static final long DEFAULT_MAX_BUCKET_SIZE = 512 * 1024 + 1024;
+  private static final ConcurrentMap<String, Long> TABLE_COUNTERS = new 
ConcurrentHashMap();
+  private static final String ROW_STATISTICS_DROPPED = "rowStatisticsDropped";
+  private static final String ROW_STATISTICS_PUT_FAILED = 
"rowStatisticsPutFailures";
+  private Counter rowStatisticsDropped;
+  private Counter rowStatisticsPutFailed;
+  private long maxCacheSize;
+  private final RowStatisticsRecorder recorder;
+  private RowStatisticsTableRecorder tableRecorder;
+
+  @InterfaceAudience.Private
+  public RowStatisticsCompactionObserver(RowStatisticsRecorder recorder) {
+    this.recorder = recorder;
+    this.tableRecorder = null;
+  }
+
+  public RowStatisticsCompactionObserver() {
+    this(null);
+  }
+
+  @Override
+  public Optional<RegionObserver> getRegionObserver() {
+    return Optional.of(this);
+  }
+
+  @Override
+  public Optional<MasterObserver> getMasterObserver() {
+    return Optional.of(this);
+  }
+
+  @Override
+  public void start(CoprocessorEnvironment e) throws IOException {
+    if (e instanceof RegionCoprocessorEnvironment) {
+      RegionCoprocessorEnvironment regionEnv = (RegionCoprocessorEnvironment) 
e;
+      if (RowStatisticsUtil.isInternalTable(regionEnv)) {
+        return;
+      }
+
+      String[] configuredBuckets =
+        
regionEnv.getConfiguration().getStrings(BlockCacheFactory.BUCKET_CACHE_BUCKETS_KEY);
+      maxCacheSize = DEFAULT_MAX_BUCKET_SIZE;
+      if (configuredBuckets != null && configuredBuckets.length > 0) {
+        String lastBucket = configuredBuckets[configuredBuckets.length - 1];
+        try {
+          maxCacheSize = Integer.parseInt(lastBucket.trim());
+        } catch (NumberFormatException ex) {
+          LOG.warn("Failed to parse {} value {} as int", 
BlockCacheFactory.BUCKET_CACHE_BUCKETS_KEY,
+            lastBucket, ex);
+        }
+      }
+
+      rowStatisticsDropped =
+        
regionEnv.getMetricRegistryForRegionServer().counter(ROW_STATISTICS_DROPPED);
+      rowStatisticsPutFailed =
+        
regionEnv.getMetricRegistryForRegionServer().counter(ROW_STATISTICS_PUT_FAILED);
+
+      TableName tableName = regionEnv.getRegionInfo().getTable();
+      TABLE_COUNTERS.merge(tableName.getNameAsString(), 1L, Long::sum);
+    }
+  }
+
+  @Override
+  public void stop(CoprocessorEnvironment e) throws IOException {
+    if (e instanceof RegionCoprocessorEnvironment) {
+      RegionCoprocessorEnvironment regionEnv = (RegionCoprocessorEnvironment) 
e;
+      if (RowStatisticsUtil.isInternalTable(regionEnv)) {
+        return;
+      }
+      TableName tableName = regionEnv.getRegionInfo().getTable();
+      long tableCount = TABLE_COUNTERS.merge(tableName.getNameAsString(), -1L, 
Long::sum);
+      if (tableCount == 0) {
+        long regionCount = 0;
+        for (long count : TABLE_COUNTERS.values()) {
+          regionCount += count;
+        }
+        if (regionCount == 0) {
+          
regionEnv.getMetricRegistryForRegionServer().remove(ROW_STATISTICS_DROPPED,
+            rowStatisticsDropped);
+          
regionEnv.getMetricRegistryForRegionServer().remove(ROW_STATISTICS_PUT_FAILED,
+            rowStatisticsPutFailed);
+          boolean removed = 
regionEnv.getSharedData().remove(TABLE_RECORDER_KEY, tableRecorder);
+          if (removed) {
+            tableRecorder.close();
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> 
ctx)
+    throws IOException {
+    try (Admin admin = ctx.getEnvironment().getConnection().getAdmin()) {

Review Comment:
   👍 



##########
hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/row/stats/RowStatisticsCompactionObserver.java:
##########
@@ -0,0 +1,321 @@
+/*
+ * 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.coprocessor.example.row.stats;
+
+import static 
org.apache.hadoop.hbase.coprocessor.example.row.stats.utils.RowStatisticsTableUtil.CF;
+import static 
org.apache.hadoop.hbase.coprocessor.example.row.stats.utils.RowStatisticsTableUtil.NAMESPACE;
+import static 
org.apache.hadoop.hbase.coprocessor.example.row.stats.utils.RowStatisticsTableUtil.NAMESPACED_TABLE_NAME;
+import static 
org.apache.hadoop.hbase.coprocessor.example.row.stats.utils.RowStatisticsTableUtil.TABLE_RECORDER_KEY;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.stream.Collectors;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.ExtendedCell;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.RawCellBuilder;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import 
org.apache.hadoop.hbase.coprocessor.example.row.stats.recorder.RowStatisticsRecorder;
+import 
org.apache.hadoop.hbase.coprocessor.example.row.stats.recorder.RowStatisticsTableRecorder;
+import 
org.apache.hadoop.hbase.coprocessor.example.row.stats.utils.RowStatisticsUtil;
+import org.apache.hadoop.hbase.io.hfile.BlockCacheFactory;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.regionserver.InternalScanner;
+import org.apache.hadoop.hbase.regionserver.ScanType;
+import org.apache.hadoop.hbase.regionserver.ScannerContext;
+import org.apache.hadoop.hbase.regionserver.Shipper;
+import org.apache.hadoop.hbase.regionserver.Store;
+import 
org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
+import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+public class RowStatisticsCompactionObserver
+  implements RegionCoprocessor, RegionObserver, MasterCoprocessor, 
MasterObserver {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RowStatisticsCompactionObserver.class);
+
+  // From private field BucketAllocator.DEFAULT_BUCKET_SIZES
+  private static final long DEFAULT_MAX_BUCKET_SIZE = 512 * 1024 + 1024;
+  private static final ConcurrentMap<String, Long> TABLE_COUNTERS = new 
ConcurrentHashMap();
+  private static final String ROW_STATISTICS_DROPPED = "rowStatisticsDropped";
+  private static final String ROW_STATISTICS_PUT_FAILED = 
"rowStatisticsPutFailures";
+  private Counter rowStatisticsDropped;
+  private Counter rowStatisticsPutFailed;
+  private long maxCacheSize;
+  private final RowStatisticsRecorder recorder;
+  private RowStatisticsTableRecorder tableRecorder;
+
+  @InterfaceAudience.Private
+  public RowStatisticsCompactionObserver(RowStatisticsRecorder recorder) {
+    this.recorder = recorder;
+    this.tableRecorder = null;
+  }
+
+  public RowStatisticsCompactionObserver() {
+    this(null);
+  }
+
+  @Override
+  public Optional<RegionObserver> getRegionObserver() {
+    return Optional.of(this);
+  }
+
+  @Override
+  public Optional<MasterObserver> getMasterObserver() {
+    return Optional.of(this);
+  }
+
+  @Override
+  public void start(CoprocessorEnvironment e) throws IOException {
+    if (e instanceof RegionCoprocessorEnvironment) {
+      RegionCoprocessorEnvironment regionEnv = (RegionCoprocessorEnvironment) 
e;
+      if (RowStatisticsUtil.isInternalTable(regionEnv)) {
+        return;
+      }
+
+      String[] configuredBuckets =
+        
regionEnv.getConfiguration().getStrings(BlockCacheFactory.BUCKET_CACHE_BUCKETS_KEY);
+      maxCacheSize = DEFAULT_MAX_BUCKET_SIZE;
+      if (configuredBuckets != null && configuredBuckets.length > 0) {
+        String lastBucket = configuredBuckets[configuredBuckets.length - 1];
+        try {
+          maxCacheSize = Integer.parseInt(lastBucket.trim());
+        } catch (NumberFormatException ex) {
+          LOG.warn("Failed to parse {} value {} as int", 
BlockCacheFactory.BUCKET_CACHE_BUCKETS_KEY,
+            lastBucket, ex);
+        }
+      }
+
+      rowStatisticsDropped =
+        
regionEnv.getMetricRegistryForRegionServer().counter(ROW_STATISTICS_DROPPED);
+      rowStatisticsPutFailed =
+        
regionEnv.getMetricRegistryForRegionServer().counter(ROW_STATISTICS_PUT_FAILED);
+
+      TableName tableName = regionEnv.getRegionInfo().getTable();
+      TABLE_COUNTERS.merge(tableName.getNameAsString(), 1L, Long::sum);
+    }
+  }
+
+  @Override
+  public void stop(CoprocessorEnvironment e) throws IOException {
+    if (e instanceof RegionCoprocessorEnvironment) {
+      RegionCoprocessorEnvironment regionEnv = (RegionCoprocessorEnvironment) 
e;
+      if (RowStatisticsUtil.isInternalTable(regionEnv)) {
+        return;
+      }
+      TableName tableName = regionEnv.getRegionInfo().getTable();
+      long tableCount = TABLE_COUNTERS.merge(tableName.getNameAsString(), -1L, 
Long::sum);
+      if (tableCount == 0) {
+        long regionCount = 0;
+        for (long count : TABLE_COUNTERS.values()) {
+          regionCount += count;
+        }
+        if (regionCount == 0) {
+          
regionEnv.getMetricRegistryForRegionServer().remove(ROW_STATISTICS_DROPPED,
+            rowStatisticsDropped);
+          
regionEnv.getMetricRegistryForRegionServer().remove(ROW_STATISTICS_PUT_FAILED,
+            rowStatisticsPutFailed);
+          boolean removed = 
regionEnv.getSharedData().remove(TABLE_RECORDER_KEY, tableRecorder);
+          if (removed) {
+            tableRecorder.close();
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> 
ctx)
+    throws IOException {
+    try (Admin admin = ctx.getEnvironment().getConnection().getAdmin()) {
+      if (admin.tableExists(NAMESPACED_TABLE_NAME)) {
+        LOG.info("Table {} already exists. Skipping table creation process.",
+          NAMESPACED_TABLE_NAME);
+      } else {
+        boolean shouldCreateNamespace =
+          Arrays.stream(admin.listNamespaces()).filter(namespace -> 
namespace.equals(NAMESPACE))
+            .collect(Collectors.toUnmodifiableSet()).isEmpty();
+        if (shouldCreateNamespace) {
+          NamespaceDescriptor nd = 
NamespaceDescriptor.create(NAMESPACE).build();
+          try {
+            admin.createNamespace(nd);
+          } catch (IOException e) {
+            LOG.error("Failed to create namespace {}", NAMESPACE, e);
+          }
+        }
+        ColumnFamilyDescriptor cfd = 
ColumnFamilyDescriptorBuilder.newBuilder(CF).setMaxVersions(25)

Review Comment:
   I think it's fine.



-- 
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.

To unsubscribe, e-mail: issues-unsubscr...@hbase.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to