You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2017/11/09 20:58:19 UTC

[11/20] phoenix git commit: PHOENIX-4332 Indexes should inherit guide post width of the base data table

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59e49f78/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
index d82c66f..4e4978c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
@@ -18,6 +18,8 @@
 package org.apache.phoenix.schema.stats;
 
 import java.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -45,15 +47,22 @@ import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
 import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Maps;
 
 /**
@@ -75,6 +84,7 @@ class DefaultStatisticsCollector implements StatisticsCollector {
     private ImmutableBytesWritable currentRow;
     private final long clientTimeStamp;
     private final String tableName;
+    private final boolean isViewIndexTable;
 
     DefaultStatisticsCollector(RegionCoprocessorEnvironment env, String tableName, long clientTimeStamp, byte[] family,
             byte[] gp_width_bytes, byte[] gp_per_region_bytes) throws IOException {
@@ -95,6 +105,9 @@ class DefaultStatisticsCollector implements StatisticsCollector {
         // since there's no row representing those in SYSTEM.CATALOG.
         if (MetaDataUtil.isViewIndex(tableName)) {
             pName = MetaDataUtil.getViewIndexUserTableName(tableName);
+            isViewIndexTable = true;
+        } else {
+            isViewIndexTable = false;
         }
         ptableKey = SchemaUtil.getTableKeyFromFullName(pName);
         this.clientTimeStamp = clientTimeStamp;
@@ -109,7 +122,7 @@ class DefaultStatisticsCollector implements StatisticsCollector {
         }
     }
     
-    private void initGuidepostDepth() throws IOException {
+    private void initGuidepostDepth() throws IOException, ClassNotFoundException, SQLException {
         // First check is if guidepost info set on statement itself
         if (guidePostPerRegionBytes != null || guidePostWidthBytes != null) {
             int guidepostPerRegion = 0;
@@ -135,6 +148,38 @@ class DefaultStatisticsCollector implements StatisticsCollector {
                 if (!result.isEmpty()) {
                     Cell cell = result.listCells().get(0);
                     guidepostWidth = PLong.INSTANCE.getCodec().decodeLong(cell.getValueArray(), cell.getValueOffset(), SortOrder.getDefault());
+                } else if (!isViewIndexTable) {
+                    /*
+                     * The table we are collecting stats for is potentially a base table, or local
+                     * index or a global index. For view indexes, we rely on the the guide post
+                     * width column in the parent data table's metadata which we already tried
+                     * retrieving above.
+                     */
+                    try (Connection conn =
+                            QueryUtil.getConnectionOnServer(env.getConfiguration())) {
+                        PTable table = PhoenixRuntime.getTable(conn, tableName);
+                        if (table.getType() == PTableType.INDEX
+                                && table.getIndexType() == IndexType.GLOBAL) {
+                            /*
+                             * For global indexes, we need to get the parentName first and then
+                             * fetch guide post width configured for the parent table.
+                             */
+                            PName parentName = table.getParentName();
+                            byte[] parentKey =
+                                    SchemaUtil.getTableKeyFromFullName(parentName.getString());
+                            get = new Get(parentKey);
+                            get.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                                PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH_BYTES);
+                            result = htable.get(get);
+                            if (!result.isEmpty()) {
+                                Cell cell = result.listCells().get(0);
+                                guidepostWidth =
+                                        PLong.INSTANCE.getCodec().decodeLong(cell.getValueArray(),
+                                            cell.getValueOffset(), SortOrder.getDefault());
+                            }
+                        }
+                    }
+
                 }
             } finally {
                 if (htable != null) {
@@ -318,7 +363,11 @@ class DefaultStatisticsCollector implements StatisticsCollector {
 
     @Override
     public void init() throws IOException {
-        initGuidepostDepth();
+        try {
+            initGuidepostDepth();
+        } catch (ClassNotFoundException | SQLException e) {
+            throw new IOException("Unable to initialize the guide post depth", e);
+        }
         this.statsWriter = StatisticsWriter.newWriter(env, tableName, clientTimeStamp, guidePostDepth);
     }
 
@@ -331,4 +380,9 @@ class DefaultStatisticsCollector implements StatisticsCollector {
         return null;
     }
 
+    @VisibleForTesting // Don't call this method anywhere else
+    public long getGuidePostDepth() {
+        return guidePostDepth;
+    }
+
 }