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


##########
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:
   If the RegionServer tries to record row statistics before the 
HMasterObserver coprocessor can create the internal table, the [RegionServer 
will drop the statistics for this region's compaction 
event](https://github.com/apache/hbase/blob/a32c93444ac6c97971ed29b57b11c5e2f334253b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/row/stats/ringbuffer/RowStatisticsEventHandler.java#L57-L60).
 The RegionServer will attempt to record the region's updated row statistics on 
the next one.
   
   **More details:**
   - We [wrap the InternalScanner of the preCompact hook with our own 
RowStatisticsInternalScanner](https://github.com/apache/hbase/blob/a32c93444ac6c97971ed29b57b11c5e2f334253b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/row/stats/RowStatisticsCompactionObserver.java#L221).
 
   - The 
[RowStatisticsInternalScanner](https://github.com/apache/hbase/blob/a32c93444ac6c97971ed29b57b11c5e2f334253b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/row/stats/RowStatisticsCompactionObserver.java#L234-L235)
 records row statistics in the internal row statistics table. It does this by 
instantiating a new (or retrieving an existing) RowStatisticsTableRecorder 
object and passing the region's row statistics to it. 
   - The RowStatisticsTableRecorder::record method uses a [disruptor to pass 
row statistics between the data collection and persistence stages of the 
system](https://github.com/apache/hbase/blob/a32c93444ac6c97971ed29b57b11c5e2f334253b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/row/stats/recorder/RowStatisticsTableRecorder.java#L99-L100).
   - In the data persistence layer of the system, [we drop row statistics when 
we cannot publish them to the internal 
table](https://github.com/apache/hbase/blob/a32c93444ac6c97971ed29b57b11c5e2f334253b/hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/row/stats/ringbuffer/RowStatisticsEventHandler.java#L57-L60)



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