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


##########
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:
   Is this necessary? At least, should be qualified as `<scope>` `test` or 
`runtime` ?



##########
hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/row/stats/RowStatistics.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public interface RowStatistics {

Review Comment:
   A general comment on this interface -- many of these methods are returning a 
"count" value but they don't indicate that in the method name. I expect a 
method like `getFooBytes` to return a `byte[]` (I think that this is common in 
the hbase code base ; we deal a lot with the `byte[]` type).



##########
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)
+          .setTimeToLive((int) Duration.ofDays(7).toSeconds()).build();
+        TableDescriptor td =
+          
TableDescriptorBuilder.newBuilder(NAMESPACED_TABLE_NAME).setColumnFamily(cfd).build();
+        LOG.info("Creating table {}", NAMESPACED_TABLE_NAME);
+        try {
+          admin.createTable(td);
+        } catch (IOException e) {
+          LOG.error("Failed to create table {}", NAMESPACED_TABLE_NAME, e);
+        }
+      }
+    } catch (IOException e) {
+      LOG.error("Failed to get Connection or Admin. Cannot determine if table 
{} exists.",
+        NAMESPACED_TABLE_NAME, e);
+    }
+  }
+
+  @Override
+  public InternalScanner 
preCompact(ObserverContext<RegionCoprocessorEnvironment> context,
+    Store store, InternalScanner scanner, ScanType scanType, 
CompactionLifeCycleTracker tracker,
+    CompactionRequest request) {
+    if (RowStatisticsUtil.isInternalTable(store.getTableName())) {
+      LOG.debug("Region {} belongs to an internal table {}, so no row 
statistics will be recorded",

Review Comment:
   I think you can skip this log message. Internal tables are expected, so this 
will just add extra logging that doesn't really help an operator to understand 
what's happening in the system.



##########
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)
+          .setTimeToLive((int) Duration.ofDays(7).toSeconds()).build();
+        TableDescriptor td =
+          
TableDescriptorBuilder.newBuilder(NAMESPACED_TABLE_NAME).setColumnFamily(cfd).build();
+        LOG.info("Creating table {}", NAMESPACED_TABLE_NAME);
+        try {
+          admin.createTable(td);
+        } catch (IOException e) {
+          LOG.error("Failed to create table {}", NAMESPACED_TABLE_NAME, e);
+        }
+      }
+    } catch (IOException e) {
+      LOG.error("Failed to get Connection or Admin. Cannot determine if table 
{} exists.",
+        NAMESPACED_TABLE_NAME, e);
+    }
+  }
+
+  @Override
+  public InternalScanner 
preCompact(ObserverContext<RegionCoprocessorEnvironment> context,
+    Store store, InternalScanner scanner, ScanType scanType, 
CompactionLifeCycleTracker tracker,
+    CompactionRequest request) {
+    if (RowStatisticsUtil.isInternalTable(store.getTableName())) {
+      LOG.debug("Region {} belongs to an internal table {}, so no row 
statistics will be recorded",
+        store.getRegionInfo().getRegionNameAsString(), 
store.getTableName().getNameAsString());
+      return scanner;
+    }
+    int blocksize = store.getColumnFamilyDescriptor().getBlocksize();
+    boolean isMajor = request.isMajor();
+    RowStatisticsImpl stats;
+    if (isMajor) {

Review Comment:
   nit: collapse this if/else block into a single constructor call and pass in 
`isMajor` as the final parameter.



##########
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)
+          .setTimeToLive((int) Duration.ofDays(7).toSeconds()).build();
+        TableDescriptor td =
+          
TableDescriptorBuilder.newBuilder(NAMESPACED_TABLE_NAME).setColumnFamily(cfd).build();
+        LOG.info("Creating table {}", NAMESPACED_TABLE_NAME);
+        try {
+          admin.createTable(td);
+        } catch (IOException e) {
+          LOG.error("Failed to create table {}", NAMESPACED_TABLE_NAME, e);
+        }
+      }
+    } catch (IOException e) {
+      LOG.error("Failed to get Connection or Admin. Cannot determine if table 
{} exists.",
+        NAMESPACED_TABLE_NAME, e);
+    }
+  }
+
+  @Override
+  public InternalScanner 
preCompact(ObserverContext<RegionCoprocessorEnvironment> context,
+    Store store, InternalScanner scanner, ScanType scanType, 
CompactionLifeCycleTracker tracker,
+    CompactionRequest request) {
+    if (RowStatisticsUtil.isInternalTable(store.getTableName())) {
+      LOG.debug("Region {} belongs to an internal table {}, so no row 
statistics will be recorded",
+        store.getRegionInfo().getRegionNameAsString(), 
store.getTableName().getNameAsString());
+      return scanner;
+    }
+    int blocksize = store.getColumnFamilyDescriptor().getBlocksize();
+    boolean isMajor = request.isMajor();
+    RowStatisticsImpl stats;
+    if (isMajor) {
+      stats = new RowStatisticsImpl(store.getTableName().getNameAsString(),
+        store.getRegionInfo().getEncodedName(), store.getColumnFamilyName(), 
blocksize,
+        maxCacheSize, true);
+    } else {
+      stats = new RowStatisticsImpl(store.getTableName().getNameAsString(),
+        store.getRegionInfo().getEncodedName(), store.getColumnFamilyName(), 
blocksize,
+        maxCacheSize, false);
+    }
+    return new RowStatisticsScanner(scanner, isMajor, stats, 
context.getEnvironment());
+  }
+
+  private class RowStatisticsScanner implements InternalScanner, Shipper {
+
+    private final InternalScanner scanner;
+    private final Shipper shipper;
+    private final boolean isMajor;
+    private final RowStatisticsImpl rowStatistics;
+    private final RegionCoprocessorEnvironment regionEnv;
+    private RawCellBuilder cellBuilder;
+    private Cell lastCell;
+
+    public RowStatisticsScanner(InternalScanner scanner, boolean isMajor,
+      RowStatisticsImpl rowStatistics, RegionCoprocessorEnvironment regionEnv) 
{
+      this.scanner = scanner;
+      if (scanner instanceof Shipper) {
+        this.shipper = (Shipper) scanner;
+      } else {
+        this.shipper = null;
+      }
+      this.isMajor = isMajor;
+      this.rowStatistics = rowStatistics;
+      this.regionEnv = regionEnv;
+      this.cellBuilder = regionEnv.getCellBuilder();
+    }
+
+    @Override
+    public boolean next(List<? super ExtendedCell> list, ScannerContext 
scannerContext)
+      throws IOException {
+      boolean ret = scanner.next(list, scannerContext);
+
+      if (list.isEmpty()) {
+        return ret;
+      }
+
+      // each next() call returns at most 1 row (maybe less for large rows)
+      // so we just need to check if the first cell has changed rows
+      ExtendedCell first = (ExtendedCell) list.get(0);
+      if (rowChanged(first)) {
+        rowStatistics.handleRowChanged(lastCell);
+      }
+
+      for (int i = 0; i < list.size(); i++) {
+        ExtendedCell cell = (ExtendedCell) list.get(i);
+        rowStatistics.consumeCell(cell);
+        lastCell = cell;
+      }
+
+      return ret;
+    }
+
+    @Override
+    public boolean next(List<? super ExtendedCell> result) throws IOException {

Review Comment:
   Why does this version of the method not perform any accounting?



##########
hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/row/stats/utils/RowStatisticsUtil.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.utils;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.RawCellBuilder;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class RowStatisticsUtil {
+
+  public static Cell cloneWithoutValue(RawCellBuilder cellBuilder, Cell cell) {
+    return cellBuilder.clear().setRow(cell.getRowArray(), cell.getRowOffset(), 
cell.getRowLength())
+      .setFamily(cell.getFamilyArray(), cell.getFamilyOffset(), 
cell.getFamilyLength())
+      .setQualifier(cell.getQualifierArray(), cell.getQualifierOffset(), 
cell.getQualifierLength())
+      .setTimestamp(cell.getTimestamp()).setType(cell.getType()).build();
+  }
+
+  public static boolean isInternalTable(RegionCoprocessorEnvironment 
environment) {
+    return isInternalTable(environment.getRegionInfo().getTable());
+  }
+
+  public static boolean isInternalTable(TableName tableName) {

Review Comment:
   Does `TableName#isSystemTable()` already do what you need?



##########
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) {

Review Comment:
   nit on style throughout: exit early instead of ever-increasing to the 
indention depth.



##########
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();

Review Comment:
   Why not use `TableName` as the map key?



##########
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();

Review Comment:
   I think the better approach is to close when `tableRecorder != null` .



##########
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,

Review Comment:
   Why do you need to remove metrics? Is this some kind of coprocessor hygiene 
best practice?



##########
hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/row/stats/RowStatisticsImpl.java:
##########
@@ -0,0 +1,272 @@
+/*
+ * 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

Review Comment:
   Is lazy clone safe when off-heap memory is being used? If the cell us 
referenced from a netty buffer, we will hold onto that slab until the 
compaction completes. I'm not sure how reference counting works here, if we 
need to explicitly increment a counter in order to prevent a use-after-free 
segfault.



##########
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:
   why 25 versions?



##########
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)
+          .setTimeToLive((int) Duration.ofDays(7).toSeconds()).build();
+        TableDescriptor td =
+          
TableDescriptorBuilder.newBuilder(NAMESPACED_TABLE_NAME).setColumnFamily(cfd).build();
+        LOG.info("Creating table {}", NAMESPACED_TABLE_NAME);
+        try {
+          admin.createTable(td);
+        } catch (IOException e) {
+          LOG.error("Failed to create table {}", NAMESPACED_TABLE_NAME, e);
+        }
+      }
+    } catch (IOException e) {
+      LOG.error("Failed to get Connection or Admin. Cannot determine if table 
{} exists.",
+        NAMESPACED_TABLE_NAME, e);
+    }
+  }
+
+  @Override
+  public InternalScanner 
preCompact(ObserverContext<RegionCoprocessorEnvironment> context,
+    Store store, InternalScanner scanner, ScanType scanType, 
CompactionLifeCycleTracker tracker,
+    CompactionRequest request) {
+    if (RowStatisticsUtil.isInternalTable(store.getTableName())) {
+      LOG.debug("Region {} belongs to an internal table {}, so no row 
statistics will be recorded",
+        store.getRegionInfo().getRegionNameAsString(), 
store.getTableName().getNameAsString());
+      return scanner;
+    }
+    int blocksize = store.getColumnFamilyDescriptor().getBlocksize();
+    boolean isMajor = request.isMajor();
+    RowStatisticsImpl stats;
+    if (isMajor) {
+      stats = new RowStatisticsImpl(store.getTableName().getNameAsString(),
+        store.getRegionInfo().getEncodedName(), store.getColumnFamilyName(), 
blocksize,
+        maxCacheSize, true);
+    } else {
+      stats = new RowStatisticsImpl(store.getTableName().getNameAsString(),
+        store.getRegionInfo().getEncodedName(), store.getColumnFamilyName(), 
blocksize,
+        maxCacheSize, false);
+    }
+    return new RowStatisticsScanner(scanner, isMajor, stats, 
context.getEnvironment());
+  }
+
+  private class RowStatisticsScanner implements InternalScanner, Shipper {

Review Comment:
   Can this be a `static` class? 



##########
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:
   Is there a race between master+coprocessor startup and a region server 
trying to write stats?



##########
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)
+          .setTimeToLive((int) Duration.ofDays(7).toSeconds()).build();
+        TableDescriptor td =
+          
TableDescriptorBuilder.newBuilder(NAMESPACED_TABLE_NAME).setColumnFamily(cfd).build();
+        LOG.info("Creating table {}", NAMESPACED_TABLE_NAME);
+        try {
+          admin.createTable(td);
+        } catch (IOException e) {
+          LOG.error("Failed to create table {}", NAMESPACED_TABLE_NAME, e);
+        }
+      }
+    } catch (IOException e) {
+      LOG.error("Failed to get Connection or Admin. Cannot determine if table 
{} exists.",
+        NAMESPACED_TABLE_NAME, e);
+    }
+  }
+
+  @Override
+  public InternalScanner 
preCompact(ObserverContext<RegionCoprocessorEnvironment> context,
+    Store store, InternalScanner scanner, ScanType scanType, 
CompactionLifeCycleTracker tracker,
+    CompactionRequest request) {
+    if (RowStatisticsUtil.isInternalTable(store.getTableName())) {
+      LOG.debug("Region {} belongs to an internal table {}, so no row 
statistics will be recorded",
+        store.getRegionInfo().getRegionNameAsString(), 
store.getTableName().getNameAsString());
+      return scanner;
+    }
+    int blocksize = store.getColumnFamilyDescriptor().getBlocksize();
+    boolean isMajor = request.isMajor();
+    RowStatisticsImpl stats;
+    if (isMajor) {
+      stats = new RowStatisticsImpl(store.getTableName().getNameAsString(),
+        store.getRegionInfo().getEncodedName(), store.getColumnFamilyName(), 
blocksize,
+        maxCacheSize, true);
+    } else {
+      stats = new RowStatisticsImpl(store.getTableName().getNameAsString(),
+        store.getRegionInfo().getEncodedName(), store.getColumnFamilyName(), 
blocksize,
+        maxCacheSize, false);
+    }
+    return new RowStatisticsScanner(scanner, isMajor, stats, 
context.getEnvironment());
+  }
+
+  private class RowStatisticsScanner implements InternalScanner, Shipper {
+
+    private final InternalScanner scanner;
+    private final Shipper shipper;
+    private final boolean isMajor;
+    private final RowStatisticsImpl rowStatistics;
+    private final RegionCoprocessorEnvironment regionEnv;
+    private RawCellBuilder cellBuilder;
+    private Cell lastCell;
+
+    public RowStatisticsScanner(InternalScanner scanner, boolean isMajor,
+      RowStatisticsImpl rowStatistics, RegionCoprocessorEnvironment regionEnv) 
{
+      this.scanner = scanner;
+      if (scanner instanceof Shipper) {
+        this.shipper = (Shipper) scanner;
+      } else {
+        this.shipper = null;
+      }
+      this.isMajor = isMajor;

Review Comment:
   nit: trim off one constructor argument and read `isMajor` from the 
`rowStatistics` value.



##########
hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/row/stats/recorder/RowStatisticsTableRecorder.java:
##########
@@ -0,0 +1,187 @@
+/*
+ * 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.recorder;
+
+import static 
org.apache.hadoop.hbase.coprocessor.example.row.stats.utils.RowStatisticsConfigurationUtil.getInt;
+import static 
org.apache.hadoop.hbase.coprocessor.example.row.stats.utils.RowStatisticsConfigurationUtil.getLong;
+import static 
org.apache.hadoop.hbase.coprocessor.example.row.stats.utils.RowStatisticsTableUtil.NAMESPACED_TABLE_NAME;
+
+import com.lmax.disruptor.BlockingWaitStrategy;
+import com.lmax.disruptor.RingBuffer;
+import com.lmax.disruptor.TimeoutException;
+import com.lmax.disruptor.dsl.Disruptor;
+import com.lmax.disruptor.dsl.ProducerType;
+import java.io.IOException;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.BufferedMutatorParams;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionConfiguration;
+import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
+import org.apache.hadoop.hbase.coprocessor.example.row.stats.RowStatisticsImpl;
+import 
org.apache.hadoop.hbase.coprocessor.example.row.stats.ringbuffer.RowStatisticsDisruptorExceptionHandler;
+import 
org.apache.hadoop.hbase.coprocessor.example.row.stats.ringbuffer.RowStatisticsEventHandler;
+import 
org.apache.hadoop.hbase.coprocessor.example.row.stats.ringbuffer.RowStatisticsRingBufferEnvelope;
+import 
org.apache.hadoop.hbase.coprocessor.example.row.stats.ringbuffer.RowStatisticsRingBufferPayload;
+import org.apache.hadoop.hbase.metrics.Counter;
+import org.apache.hadoop.hbase.util.Threads;
+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;
+
+@InterfaceAudience.Private
+public class RowStatisticsTableRecorder implements RowStatisticsRecorder {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(RowStatisticsTableRecorder.class);
+  // Must be multiple of 2. Should be greater than num regions/RS
+  private static final int DEFAULT_EVENT_COUNT = 1024;
+  private static final long DISRUPTOR_SHUTDOWN_TIMEOUT_MS = 60_0000L;
+  private final BufferedMutator bufferedMutator;
+  private final Counter rowStatisticsDropped;
+  private final Disruptor<RowStatisticsRingBufferEnvelope> disruptor;
+  private final RingBuffer<RowStatisticsRingBufferEnvelope> ringBuffer;
+  private final AtomicBoolean closed;
+
+  /*
+   * This constructor is ONLY for testing. Use 
TableRecorder#forClusterConnection if you want to
+   * instantiate a TableRecorder object.
+   */
+  private RowStatisticsTableRecorder(BufferedMutator bufferedMutator,
+    Disruptor<RowStatisticsRingBufferEnvelope> disruptor, Counter 
rowStatisticsDropped) {
+    this.bufferedMutator = bufferedMutator;
+    this.disruptor = disruptor;
+    this.ringBuffer = disruptor.getRingBuffer();
+    this.rowStatisticsDropped = rowStatisticsDropped;
+    this.closed = new AtomicBoolean(false);
+  }
+
+  public static RowStatisticsTableRecorder forClusterConnection(Connection 
clusterConnection,
+    Counter rowStatisticsDropped, Counter rowStatisticsPutFailed) {
+    BufferedMutator bufferedMutator =
+      initializeBufferedMutator(clusterConnection, rowStatisticsPutFailed);
+    if (bufferedMutator == null) {
+      return null;
+    }
+
+    Disruptor<RowStatisticsRingBufferEnvelope> disruptor =
+      initializeDisruptor(bufferedMutator, rowStatisticsPutFailed);
+    disruptor.start();
+
+    return new RowStatisticsTableRecorder(bufferedMutator, disruptor, 
rowStatisticsDropped);
+  }
+
+  @Override
+  public void record(RowStatisticsImpl rowStatistics, boolean isMajor,
+    Optional<byte[]> fullRegionName) {
+    if (!closed.get()) {
+      if (
+        !ringBuffer.tryPublishEvent((envelope, seqId) -> envelope
+          .load(new RowStatisticsRingBufferPayload(rowStatistics, isMajor, 
fullRegionName.get())))
+      ) {
+        rowStatisticsDropped.increment();
+        LOG.error("Failed to load row statistics for region={} into the ring 
buffer",
+          rowStatistics.getRegion());
+      }
+    } else {
+      rowStatisticsDropped.increment();
+      LOG.error("TableRecorder is closed. Will not record row statistics for 
region={}",
+        rowStatistics.getRegion());
+    }
+  }
+
+  public void close() throws IOException {
+    if (!closed.compareAndSet(false, true)) {
+      return;
+    }
+    try {
+      disruptor.shutdown(DISRUPTOR_SHUTDOWN_TIMEOUT_MS, TimeUnit.MILLISECONDS);
+    } catch (TimeoutException e) {
+      LOG.warn(
+        "Disruptor shutdown timed out after {} ms. Forcing halt. Some row 
statistics may be lost",
+        DISRUPTOR_SHUTDOWN_TIMEOUT_MS);
+      disruptor.halt();
+      disruptor.shutdown();
+    }
+    bufferedMutator.close();
+  }
+
+  private static BufferedMutator initializeBufferedMutator(Connection conn,
+    Counter rowStatisticsPutFailed) {
+    Configuration conf = conn.getConfiguration();
+    TableRecorderExceptionListener exceptionListener =
+      new TableRecorderExceptionListener(rowStatisticsPutFailed);
+    BufferedMutatorParams params = new 
BufferedMutatorParams(NAMESPACED_TABLE_NAME)
+      .rpcTimeout(getInt(conf, HConstants.HBASE_RPC_TIMEOUT_KEY, 15_000))
+      .operationTimeout(getInt(conf, 
HConstants.HBASE_CLIENT_OPERATION_TIMEOUT, 30_000))
+      .setWriteBufferPeriodicFlushTimeoutMs(
+        getLong(conf, 
ConnectionConfiguration.WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS, 60_000L))
+      .writeBufferSize(getLong(conf, 
ConnectionConfiguration.WRITE_BUFFER_SIZE_KEY,
+        ConnectionConfiguration.WRITE_BUFFER_SIZE_DEFAULT))
+      .listener(exceptionListener);
+    BufferedMutator bufferedMutator = null;
+    try {
+      bufferedMutator = conn.getBufferedMutator(params);
+    } catch (IOException e) {
+      LOG.error(
+        "This should NEVER print! 
ConnectionImplementation#getBufferedMutator(BufferedMutatorParams bmp) does NOT 
raise IOExceptions",
+        e);
+    }
+    return bufferedMutator;
+  }
+
+  private static Disruptor<RowStatisticsRingBufferEnvelope>
+    initializeDisruptor(BufferedMutator bufferedMutator, Counter 
rowStatisticsPutFailures) {
+    Disruptor<RowStatisticsRingBufferEnvelope> disruptor =
+      new Disruptor<>(RowStatisticsRingBufferEnvelope::new, 
DEFAULT_EVENT_COUNT,
+        new 
ThreadFactoryBuilder().setNameFormat("rowstats.append-pool-%d").setDaemon(true)
+          
.setUncaughtExceptionHandler(Threads.LOGGING_EXCEPTION_HANDLER).build(),
+        ProducerType.MULTI, new BlockingWaitStrategy());
+    disruptor.setDefaultExceptionHandler(new 
RowStatisticsDisruptorExceptionHandler());
+    RowStatisticsEventHandler rowStatisticsEventHandler =
+      new RowStatisticsEventHandler(bufferedMutator, rowStatisticsPutFailures);
+    disruptor.handleEventsWith(new RowStatisticsEventHandler[] { 
rowStatisticsEventHandler });
+    return disruptor;
+  }
+
+  protected static class TableRecorderExceptionListener
+    implements BufferedMutator.ExceptionListener {
+
+    private final Counter rowStatisticsPutFailures;
+
+    TableRecorderExceptionListener(Counter counter) {
+      this.rowStatisticsPutFailures = counter;
+    }
+
+    public void onException(RetriesExhaustedWithDetailsException exception,
+      BufferedMutator mutator) {
+      long failedPuts = mutator.getWriteBufferSize();
+      for (int i = 0; i < failedPuts; i++) {
+        rowStatisticsPutFailures.increment();

Review Comment:
   I think that there's an `Counter#increment(long)` that you can use instead 
of a for-loop.



##########
hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/row/stats/utils/RowStatisticsConfigurationUtil.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.utils;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.yetus.audience.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class RowStatisticsConfigurationUtil {
+
+  private static final String ROW_STATISTICS_PREFIX = 
"hubspot.row.statistics.";

Review Comment:
   Please choose a more appropriate configuration prefix other than "hubspot".



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