You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by an...@apache.org on 2016/07/12 12:53:46 UTC

phoenix git commit: PHOENIX-3063 MutableIndexFailureIT is getting hanged in 4.x-HBase-0.98 branch

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 04df7bca0 -> 45f0004e0


PHOENIX-3063 MutableIndexFailureIT is getting hanged in 4.x-HBase-0.98 branch


Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/45f0004e
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/45f0004e
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/45f0004e

Branch: refs/heads/4.x-HBase-0.98
Commit: 45f0004e07ca7892dcb3dc757359671f2c141261
Parents: 04df7bc
Author: Ankit Singhal <an...@gmail.com>
Authored: Tue Jul 12 18:20:44 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Tue Jul 12 18:20:44 2016 +0530

----------------------------------------------------------------------
 .../apache/phoenix/compile/FromCompiler.java    |  8 ++--
 .../phoenix/iterate/BaseResultIterators.java    |  5 ++-
 .../apache/phoenix/schema/MetaDataClient.java   | 47 +++++++++++---------
 .../org/apache/phoenix/schema/PTableImpl.java   |  6 +--
 4 files changed, 38 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/45f0004e/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
index 46e7d14..8e4d9aa 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -185,7 +185,7 @@ public class FromCompiler {
                     if (htable != null) Closeables.closeQuietly(htable);
                 }
                 tableNode = NamedTableNode.create(null, baseTable, statement.getColumnDefs());
-                return new SingleTableColumnResolver(connection, tableNode, e.getTimeStamp(), new HashMap<String, UDFParseNode>(1));
+                return new SingleTableColumnResolver(connection, tableNode, e.getTimeStamp(), new HashMap<String, UDFParseNode>(1), false);
             }
             throw e;
         }
@@ -332,7 +332,7 @@ public class FromCompiler {
     	private final String alias;
         private final List<PSchema> schemas;
 
-       public SingleTableColumnResolver(PhoenixConnection connection, NamedTableNode table, long timeStamp, Map<String, UDFParseNode> udfParseNodes) throws SQLException  {
+       public SingleTableColumnResolver(PhoenixConnection connection, NamedTableNode table, long timeStamp, Map<String, UDFParseNode> udfParseNodes, boolean isNamespaceMapped) throws SQLException  {
            super(connection, 0, false, udfParseNodes);
            List<PColumnFamily> families = Lists.newArrayListWithExpectedSize(table.getDynamicColumns().size());
            for (ColumnDef def : table.getDynamicColumns()) {
@@ -345,8 +345,8 @@ public class FromCompiler {
             if (connection.getSchema() != null) {
                 schema = schema != null ? schema : connection.getSchema();
             }
-            PTable theTable = new PTableImpl(connection.getTenantId(), schema, table.getName().getTableName(),
-                    scn == null ? HConstants.LATEST_TIMESTAMP : scn, families);
+           PTable theTable = new PTableImpl(connection.getTenantId(), schema, table.getName().getTableName(),
+                    scn == null ? HConstants.LATEST_TIMESTAMP : scn, families, isNamespaceMapped);
            theTable = this.addDynamicColumns(table.getDynamicColumns(), theTable);
            alias = null;
            tableRefs = ImmutableList.of(new TableRef(alias, theTable, timeStamp, !table.getDynamicColumns().isEmpty()));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/45f0004e/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
index 6f053bf..4a797b8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
@@ -80,6 +80,7 @@ import org.apache.phoenix.schema.PColumnFamily;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTable.ViewType;
+import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.StaleRegionBoundaryCacheException;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.stats.GuidePostsInfo;
@@ -359,7 +360,9 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         if (null == currentSCN) {
           currentSCN = HConstants.LATEST_TIMESTAMP;
         }
-        tableStats = useStats() ? context.getConnection().getQueryServices().getTableStats(physicalTableName, currentSCN) : PTableStats.EMPTY_STATS;
+        tableStats = useStats() && table.getType() != PTableType.SYSTEM
+                ? context.getConnection().getQueryServices().getTableStats(physicalTableName, currentSCN)
+                : PTableStats.EMPTY_STATS;
         // Used to tie all the scans together during logging
         scanId = UUID.randomUUID().toString();
         

http://git-wip-us.apache.org/repos/asf/phoenix/blob/45f0004e/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index d9cd666..7d2de29 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -123,6 +123,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
@@ -2519,7 +2520,6 @@ public class MetaDataClient {
                 Delete linkDelete = new Delete(linkKey, clientTimeStamp);
                 tableMetaData.add(linkDelete);
             }
-
             MetaDataMutationResult result = connection.getQueryServices().dropTable(tableMetaData, tableType, cascade);
             MutationCode code = result.getMutationCode();
             PTable table = result.getTable();
@@ -2560,24 +2560,29 @@ public class MetaDataClient {
                                     hasLocalIndexTable = true;
                                 } 
                             }
-                            hasViewIndexTable = true; // As there is no way to know whether table has views or not so
-                                                      // ensuring we delete sequence and cached object during drop
+                            hasViewIndexTable = true;// keeping always true for deletion of stats if view index present
+                                                     // or not
+                            MetaDataUtil.deleteViewIndexSequences(connection, table.getPhysicalName(),
+                                    table.isNamespaceMapped());
+                            byte[] viewIndexPhysicalName = MetaDataUtil
+                                    .getViewIndexPhysicalName(table.getPhysicalName().getBytes());
+                            if (!dropMetaData) {
+                                // we need to drop rows only when actually view index exists
+                                try (HBaseAdmin admin = connection.getQueryServices().getAdmin()) {
+                                    hasViewIndexTable = admin.tableExists(viewIndexPhysicalName);
+                                } catch (IOException e1) {
+                                    // absorbing as it is not critical check
+                                }
+                            }
                         }
                         if (tableType == PTableType.TABLE
-                                && (table.isMultiTenant() || hasViewIndexTable || hasLocalIndexTable)) {
-    
-                            MetaDataUtil.deleteViewIndexSequences(connection, table.getPhysicalName(), table.isNamespaceMapped());
+                                && (table.isMultiTenant() || hasViewIndexTable)) {
                             if (hasViewIndexTable) {
-                                String viewIndexSchemaName = null;
-                                String viewIndexTableName = null;
-                                if (schemaName != null) {
-                                    viewIndexSchemaName = MetaDataUtil.getViewIndexTableName(schemaName);
-                                    viewIndexTableName = tableName;
-                                } else {
-                                    viewIndexTableName = MetaDataUtil.getViewIndexTableName(tableName);
-                                }
-                                PTable viewIndexTable = new PTableImpl(null, viewIndexSchemaName, viewIndexTableName, ts,
-                                        table.getColumnFamilies());
+                                byte[] viewIndexPhysicalName = MetaDataUtil.getViewIndexPhysicalName(table.getPhysicalName().getBytes());
+                                PTable viewIndexTable = new PTableImpl(null,
+                                        SchemaUtil.getSchemaNameFromFullName(viewIndexPhysicalName),
+                                        SchemaUtil.getTableNameFromFullName(viewIndexPhysicalName), ts,
+                                        table.getColumnFamilies(),table.isNamespaceMapped());
                                 tableRefs.add(new TableRef(null, viewIndexTable, ts, false));
                             }
                         }
@@ -3141,10 +3146,12 @@ public class MetaDataClient {
                         if (!connection.getQueryServices().getProps().getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA)) {
                             Long scn = connection.getSCN();
                             long ts = (scn == null ? result.getMutationTime() : scn);
-                            String viewIndexSchemaName = MetaDataUtil.getViewIndexSchemaName(schemaName);
-                            String viewIndexTableName = MetaDataUtil.getViewIndexTableName(tableName);
-                            PTable viewIndexTable = new PTableImpl(null, viewIndexSchemaName, viewIndexTableName, ts,
-                                    table.getColumnFamilies());
+                            byte[] viewIndexPhysicalName = MetaDataUtil
+                                    .getViewIndexPhysicalName(table.getPhysicalName().getBytes());
+                            PTable viewIndexTable = new PTableImpl(null,
+                                    SchemaUtil.getSchemaNameFromFullName(viewIndexPhysicalName),
+                                    SchemaUtil.getTableNameFromFullName(viewIndexPhysicalName), ts,
+                                    table.getColumnFamilies(), table.isNamespaceMapped());
                             List<TableRef> tableRefs = Collections.singletonList(new TableRef(null, viewIndexTable, ts, false));
                             MutationPlan plan = new PostDDLCompiler(connection).compile(tableRefs, null, null,
                                     Collections.<PColumn> emptyList(), ts);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/45f0004e/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 76bc0d2..ec09992 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -58,6 +58,7 @@ import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.SizedUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TrustedByteArrayOutputStream;
+import org.apache.tephra.TxConstants;
 
 import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
@@ -69,8 +70,6 @@ import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
-import org.apache.tephra.TxConstants;
-
 /**
  *
  * Base class for PTable implementors.  Provides abstraction for
@@ -138,7 +137,7 @@ public class PTableImpl implements PTable {
         this.rowKeySchema = RowKeySchema.EMPTY_SCHEMA;
     }
 
-    public PTableImpl(PName tenantId, String schemaName, String tableName, long timestamp, List<PColumnFamily> families) { // For base table of mapped VIEW
+    public PTableImpl(PName tenantId, String schemaName, String tableName, long timestamp, List<PColumnFamily> families, boolean isNamespaceMapped) { // For base table of mapped VIEW
         Preconditions.checkArgument(tenantId==null || tenantId.getBytes().length > 0); // tenantId should be null or not empty
         this.tenantId = tenantId;
         this.name = PNameFactory.newName(SchemaUtil.getTableName(schemaName, tableName));
@@ -159,6 +158,7 @@ public class PTableImpl implements PTable {
         }
         this.families = families;
         this.physicalNames = Collections.emptyList();
+        this.isNamespaceMapped = isNamespaceMapped;
     }
     
     public PTableImpl(PName tenantId, PName schemaName, PName tableName, long timestamp, List<PColumnFamily> families,