dbwong commented on a change in pull request #759:
URL: https://github.com/apache/phoenix/pull/759#discussion_r411697149



##########
File path: 
phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
##########
@@ -9,32 +9,9 @@
  */
 package org.apache.phoenix.monitoring;
 
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_FAILED_QUERY_COUNTER;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_BYTES_REGION_SERVER_RESULTS;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_MILLS_BETWEEN_NEXTS;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_RPC_CALLS;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HBASE_COUNT_SCANNED_REGIONS;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HCONNECTIONS_COUNTER;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_MUTATION_BATCH_FAILED_COUNT;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_MUTATION_BATCH_SIZE;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_MUTATION_BYTES;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_MUTATION_COMMIT_TIME;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_MUTATION_SQL_COUNTER;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_NUM_PARALLEL_SCANS;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_OPEN_PHOENIX_CONNECTIONS;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_PHOENIX_CONNECTIONS_ATTEMPTED_COUNTER;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_PHOENIX_CONNECTIONS_THROTTLED_COUNTER;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_SERVICES_COUNTER;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_TIME;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_TIMEOUT_COUNTER;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_REJECTED_TASK_COUNTER;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_SCAN_BYTES;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_SELECT_SQL_COUNTER;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_SPOOL_FILE_COUNTER;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_TASK_END_TO_END_TIME;
-import static 
org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_TASK_EXECUTION_TIME;
-import static org.apache.phoenix.monitoring.MetricType.MEMORY_CHUNK_BYTES;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.*;

Review comment:
       Phoenix Style Guidelines recommends not using include *.

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
##########
@@ -1367,7 +1368,14 @@ private static Expression 
getFirstPKColumnExpression(PTable table) throws SQLExc
     public static Collection<GlobalMetric> getGlobalPhoenixClientMetrics() {
         return GlobalClientMetrics.getMetrics();
     }
-    
+
+   public static Map<String,Long>getTableLevelMetrics(){
+        return PhoenixTableRegistry.getInstance().getTableLevelMetrics();
+   }
+
+   public static void resetPhoenixTableLevelMetrics(){

Review comment:
       Public facing api please add javadoc describing the function.

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
##########
@@ -1367,7 +1368,14 @@ private static Expression 
getFirstPKColumnExpression(PTable table) throws SQLExc
     public static Collection<GlobalMetric> getGlobalPhoenixClientMetrics() {
         return GlobalClientMetrics.getMetrics();
     }
-    
+
+   public static Map<String,Long>getTableLevelMetrics(){

Review comment:
       Public facing api please add javadoc describing the function.

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/monitoring/PhoenixTableRegistry.java
##########
@@ -0,0 +1,87 @@
+package org.apache.phoenix.monitoring;

Review comment:
       Need apache header.

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/iterate/TableSnapshotResultIterator.java
##########
@@ -128,7 +129,7 @@ public boolean initSnapshotScanner() throws SQLException {
         RegionInfo hri = regions.get(this.currentRegion);
         this.scanIterator =
             new ScanningResultIterator(new SnapshotScanner(configuration, fs, 
restoreDir, htd, hri, scan),
-                scan, scanMetricsHolder);
+                scan, scanMetricsHolder,hri.getTable().toString());

Review comment:
       I don't think we should metric on snapshots.  

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/monitoring/TableLevelMetricRegistry.java
##########
@@ -0,0 +1,124 @@
+package org.apache.phoenix.monitoring;

Review comment:
       Need apache header.
   
   

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/monitoring/TableLevelMetricRegistry.java
##########
@@ -0,0 +1,124 @@
+package org.apache.phoenix.monitoring;
+
+import org.apache.hadoop.hbase.metrics.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.phoenix.monitoring.MetricType.*;
+
+public class TableLevelMetricRegistry{
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(TableLevelMetricRegistry.class);
+    private final String tableName;
+    private GlobalMetric selectSqlCounter;
+    private GlobalMetric mutationBatchFailedSize;
+    private GlobalMetric mutationBatchSize;
+    private GlobalMetric mutationBytes;
+    private GlobalMetric mutationSqlCounter;
+    private GlobalMetric queryFailedCounter;
+    private GlobalMetric queryTimeoutCounter;
+    private GlobalMetric numParalleScans;
+    private GlobalMetric numScanBytes;
+    private GlobalMetric taskExecutedCounter;
+    private Map<MetricType,GlobalMetric>metricRegister;
+
+    public TableLevelMetricRegistry(final String tableName){
+        this.tableName = tableName;
+        this.metricRegister = new HashMap<>();
+        for(MetricType type : MetricType.values()){

Review comment:
       I prefer a different approach to this loop, final private array of 
metric types we care about, then iterate over that array.  What do you think?

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/monitoring/TableLevelMetricRegistry.java
##########
@@ -0,0 +1,124 @@
+package org.apache.phoenix.monitoring;
+
+import org.apache.hadoop.hbase.metrics.*;

Review comment:
       Same * info as above.

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
##########
@@ -677,6 +680,10 @@ public boolean isGlobalMetricsEnabled() {
         return config.getBoolean(GLOBAL_METRICS_ENABLED, 
DEFAULT_IS_GLOBAL_METRICS_ENABLED);
     }
 
+    public boolean isTableLevelMetricsEnabled() {
+        return config.getBoolean(TABLE_LEVEL_METRICS_ENABLED, 
DEFAULT_IS_GLOBAL_METRICS_ENABLED);

Review comment:
       This should be DEFAULT_IS_TABLE_LEVEL_METRICS_ENABLED

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/monitoring/PhoenixTableRegistry.java
##########
@@ -0,0 +1,87 @@
+package org.apache.phoenix.monitoring;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.metrics.MetricRegistries;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+
+/**
+ * Central place where we keep track of all the Table Level phoenix metrics.
+ * Registers as new MetricSource with Hbase Registry.
+ * Register each tableMetrics and store the instance of it associated with 
TableName in a map
+ */
+
+public class PhoenixTableRegistry  {
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(PhoenixTableRegistry.class);
+    private static final boolean isTableLevelMetricsEnabled = 
QueryServicesOptions.withDefaults().isTableLevelMetricsEnabled();
+
+    private static 
ConcurrentMap<String,TableLevelMetricRegistry>tablePhoenixMapping;
+
+    static MetricRegistry metricRegistry;
+    static {
+        if(isTableLevelMetricsEnabled) {
+            metricRegistry = createMetricRegistry();
+            
GlobalMetricRegistriesAdapter.getInstance().registerMetricRegistry(metricRegistry);
+            tablePhoenixMapping = new ConcurrentHashMap<>();
+        }
+    }
+
+    private static MetricRegistry createMetricRegistry() {
+        LOGGER.info("Creating Metric Registry for Phoenix Table Level 
Metrics");
+        MetricRegistryInfo registryInfo = new 
MetricRegistryInfo("PHOENIX-TableLevel", "Phoenix Client Metrics",

Review comment:
       Consdier constants?

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/monitoring/PhoenixTableRegistry.java
##########
@@ -0,0 +1,87 @@
+package org.apache.phoenix.monitoring;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.metrics.MetricRegistries;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+
+/**
+ * Central place where we keep track of all the Table Level phoenix metrics.
+ * Registers as new MetricSource with Hbase Registry.
+ * Register each tableMetrics and store the instance of it associated with 
TableName in a map
+ */
+
+public class PhoenixTableRegistry  {
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(PhoenixTableRegistry.class);
+    private static final boolean isTableLevelMetricsEnabled = 
QueryServicesOptions.withDefaults().isTableLevelMetricsEnabled();
+
+    private static 
ConcurrentMap<String,TableLevelMetricRegistry>tablePhoenixMapping;
+
+    static MetricRegistry metricRegistry;
+    static {
+        if(isTableLevelMetricsEnabled) {
+            metricRegistry = createMetricRegistry();
+            
GlobalMetricRegistriesAdapter.getInstance().registerMetricRegistry(metricRegistry);
+            tablePhoenixMapping = new ConcurrentHashMap<>();
+        }
+    }
+
+    private static MetricRegistry createMetricRegistry() {
+        LOGGER.info("Creating Metric Registry for Phoenix Table Level 
Metrics");
+        MetricRegistryInfo registryInfo = new 
MetricRegistryInfo("PHOENIX-TableLevel", "Phoenix Client Metrics",
+                "phoenixTableLevel", "Phoenix,sub=CLIENT", true);
+        return MetricRegistries.global().create(registryInfo);
+    }
+
+    private static PhoenixTableRegistry instance = new PhoenixTableRegistry();
+    public static PhoenixTableRegistry getInstance(){
+        return instance;
+    }
+
+    public void  addOrCreateTable(String tableName, MetricType type, long 
value){

Review comment:
       Name and parameters makes this a bit unclear to me?

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/monitoring/TableLevelMetricRegistry.java
##########
@@ -0,0 +1,124 @@
+package org.apache.phoenix.monitoring;
+
+import org.apache.hadoop.hbase.metrics.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.phoenix.monitoring.MetricType.*;
+
+public class TableLevelMetricRegistry{
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(TableLevelMetricRegistry.class);
+    private final String tableName;
+    private GlobalMetric selectSqlCounter;
+    private GlobalMetric mutationBatchFailedSize;
+    private GlobalMetric mutationBatchSize;
+    private GlobalMetric mutationBytes;
+    private GlobalMetric mutationSqlCounter;
+    private GlobalMetric queryFailedCounter;
+    private GlobalMetric queryTimeoutCounter;
+    private GlobalMetric numParalleScans;
+    private GlobalMetric numScanBytes;
+    private GlobalMetric taskExecutedCounter;
+    private Map<MetricType,GlobalMetric>metricRegister;
+
+    public TableLevelMetricRegistry(final String tableName){
+        this.tableName = tableName;
+        this.metricRegister = new HashMap<>();
+        for(MetricType type : MetricType.values()){
+            switch(type) {
+                case SELECT_SQL_COUNTER:
+                    selectSqlCounter = new 
GlobalMetricImpl(SELECT_SQL_COUNTER);
+                    
this.metricRegister.put(SELECT_SQL_COUNTER,selectSqlCounter);
+                    break;
+                case MUTATION_BATCH_FAILED_SIZE:
+                    this.mutationBatchFailedSize = new 
GlobalMetricImpl(MUTATION_BATCH_FAILED_SIZE);
+                    
this.metricRegister.put(MUTATION_BATCH_FAILED_SIZE,mutationBatchFailedSize);
+                    break;
+                case MUTATION_BATCH_SIZE:
+                    this.mutationBatchSize = new 
GlobalMetricImpl(MUTATION_BATCH_SIZE);
+                    
this.metricRegister.put(MUTATION_BATCH_SIZE,mutationBatchSize);
+                    break;
+                case MUTATION_BYTES:
+                    this.mutationBytes = new GlobalMetricImpl(MUTATION_BYTES);
+                    this.metricRegister.put(MUTATION_BYTES,mutationBytes);
+                    break;
+                case MUTATION_SQL_COUNTER:
+                    this.mutationSqlCounter = new 
GlobalMetricImpl(MUTATION_SQL_COUNTER);
+                    
this.metricRegister.put(MUTATION_SQL_COUNTER,mutationSqlCounter);
+                    break;
+                case QUERY_FAILED_COUNTER:
+                    this.queryFailedCounter = new 
GlobalMetricImpl(QUERY_FAILED_COUNTER);
+                    
this.metricRegister.put(QUERY_FAILED_COUNTER,queryFailedCounter);
+                    break;
+                case QUERY_TIMEOUT_COUNTER:
+                    this.queryTimeoutCounter = new 
GlobalMetricImpl(QUERY_TIMEOUT_COUNTER);
+                    
this.metricRegister.put(QUERY_TIMEOUT_COUNTER,queryTimeoutCounter);
+                    break;
+                case NUM_PARALLEL_SCANS:
+                    this.numParalleScans = new 
GlobalMetricImpl(NUM_PARALLEL_SCANS);
+                    
this.metricRegister.put(NUM_PARALLEL_SCANS,numParalleScans);
+                    break;
+                case SCAN_BYTES:
+                    this.numScanBytes = new GlobalMetricImpl(SCAN_BYTES);
+                    this.metricRegister.put(SCAN_BYTES,numScanBytes);
+                case TASK_EXECUTED_COUNTER:
+                    this.taskExecutedCounter = new 
GlobalMetricImpl(TASK_EXECUTED_COUNTER);
+                    
this.metricRegister.put(TASK_EXECUTED_COUNTER,taskExecutedCounter);
+                default:
+                    break;
+            }
+        }
+    }
+
+    static class PhoenixGlobalMetricGauge implements Gauge<Long> {

Review comment:
       Name is confusing is this thought out or simply copied from global 
metrics, and please refactor out to extract the class and share it.  Some other 
pieces of these functions can also be extracted out and shared i think?

##########
File path: 
phoenix-core/src/main/java/org/apache/phoenix/monitoring/PhoenixTableRegistry.java
##########
@@ -0,0 +1,87 @@
+package org.apache.phoenix.monitoring;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hbase.metrics.MetricRegistries;
+import org.apache.hadoop.hbase.metrics.MetricRegistry;
+import org.apache.hadoop.hbase.metrics.MetricRegistryInfo;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+
+/**
+ * Central place where we keep track of all the Table Level phoenix metrics.
+ * Registers as new MetricSource with Hbase Registry.
+ * Register each tableMetrics and store the instance of it associated with 
TableName in a map
+ */
+
+public class PhoenixTableRegistry  {
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(PhoenixTableRegistry.class);
+    private static final boolean isTableLevelMetricsEnabled = 
QueryServicesOptions.withDefaults().isTableLevelMetricsEnabled();
+
+    private static 
ConcurrentMap<String,TableLevelMetricRegistry>tablePhoenixMapping;
+
+    static MetricRegistry metricRegistry;
+    static {
+        if(isTableLevelMetricsEnabled) {
+            metricRegistry = createMetricRegistry();
+            
GlobalMetricRegistriesAdapter.getInstance().registerMetricRegistry(metricRegistry);
+            tablePhoenixMapping = new ConcurrentHashMap<>();
+        }
+    }
+
+    private static MetricRegistry createMetricRegistry() {
+        LOGGER.info("Creating Metric Registry for Phoenix Table Level 
Metrics");
+        MetricRegistryInfo registryInfo = new 
MetricRegistryInfo("PHOENIX-TableLevel", "Phoenix Client Metrics",
+                "phoenixTableLevel", "Phoenix,sub=CLIENT", true);
+        return MetricRegistries.global().create(registryInfo);
+    }
+
+    private static PhoenixTableRegistry instance = new PhoenixTableRegistry();
+    public static PhoenixTableRegistry getInstance(){
+        return instance;
+    }
+
+    public void  addOrCreateTable(String tableName, MetricType type, long 
value){
+        if(isTableLevelMetricsEnabled) {
+            TableLevelMetricRegistry tInstance = 
tablePhoenixMapping.get(tableName);
+            if (tInstance == null) {
+                tInstance = new TableLevelMetricRegistry(tableName);
+                tInstance.registerMetrics(metricRegistry);
+                tablePhoenixMapping.put(tableName, tInstance);
+            }
+            tInstance.changeMetricValue(type, value);
+        }
+    }
+
+    //will be enabled in code flow in sub-sequent Iterations.
+    public void deleteTable(String tableName){
+        if(isTableLevelMetricsEnabled) {
+            if(tablePhoenixMapping.containsKey(tableName)){
+                TableLevelMetricRegistry tInstance = 
tablePhoenixMapping.get(tableName);
+                tInstance.removeMetricRegistry(metricRegistry);
+                tablePhoenixMapping.remove(tableName);
+            }
+        }
+    }
+
+    @VisibleForTesting
+    public Map<String,Long> getTableLevelMetrics(){
+        Map<String,Long>tableMap = new HashMap<>();
+        ConcurrentMap<String,TableLevelMetricRegistry>map = 
tablePhoenixMapping;

Review comment:
       nit: this line seems wasted?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to