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 2015/02/18 04:37:40 UTC

phoenix git commit: PHOENIX-1669 Views are not always properly updated with physical table indexes when cached

Repository: phoenix
Updated Branches:
  refs/heads/master 2e5a63089 -> c633151da


PHOENIX-1669 Views are not always properly updated with physical table indexes when cached


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

Branch: refs/heads/master
Commit: c633151dafcaf703c8daa5f92dbaddd7eeb89891
Parents: 2e5a630
Author: James Taylor <jt...@salesforce.com>
Authored: Tue Feb 17 19:37:02 2015 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Feb 17 19:37:02 2015 -0800

----------------------------------------------------------------------
 .../end2end/index/IndexExpressionIT.java        |  3 --
 .../apache/phoenix/schema/MetaDataClient.java   | 39 ++++++++++++--------
 .../org/apache/phoenix/schema/PTableImpl.java   |  2 +-
 3 files changed, 24 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c633151d/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
index 5c51bda..1e3733b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
@@ -1220,8 +1220,6 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
         assertEquals(1, rs.getLong(1));
         assertFalse(rs.next());
         
-        conn.createStatement().execute("CREATE INDEX vi1 on v(k2)");
-
         //i2 should be used since it contains s3||'_'||s4 i
         String query = "SELECT s2||'_'||s3 FROM v WHERE k2=1 AND (s2||'_'||s3)='abc_cab'";
         rs = conn.createStatement(  ).executeQuery("EXPLAIN " + query);
@@ -1235,7 +1233,6 @@ public class IndexExpressionIT extends BaseHBaseManagedTimeIT {
         assertFalse(rs.next());
         
         conn.createStatement().execute("ALTER VIEW v DROP COLUMN s4");
-        conn.createStatement().execute("CREATE INDEX vi2 on v(k2)");
         //i2 cannot be used since s4 has been dropped from the view, so i1 will be used 
         rs = conn.createStatement().executeQuery("EXPLAIN " + query);
         queryPlan = QueryUtil.getExplainPlan(rs);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c633151d/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 64e62f5..831616b 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
@@ -360,8 +360,7 @@ public class MetaDataClient {
                 // Otherwise, a tenant would be required to create a VIEW first
                 // which is not really necessary unless you want to filter or add
                 // columns
-                addIndexesFromPhysicalTable(result);
-                connection.addTable(result.getTable());
+                addTableToCache(result);
                 return result;
             } else {
                 // if (result.getMutationCode() == MutationCode.NEWER_TABLE_FOUND) {
@@ -369,16 +368,20 @@ public class MetaDataClient {
                 // Since we disallow creation or modification of a table earlier than the latest
                 // timestamp, we can handle this such that we don't ask the
                 // server again.
-                // If table was not found at the current time stamp and we have one cached, remove it.
-                // Otherwise, we're up to date, so there's nothing to do.
                 if (table != null) {
+                    // Ensures that table in result is set to table found in our cache.
                     result.setTable(table);
                     if (code == MutationCode.TABLE_ALREADY_EXISTS) {
+                        // Although this table is up-to-date, the parent table may not be.
+                        // In this case, we update the parent table which may in turn pull
+                        // in indexes to add to this table.
                         if (addIndexesFromPhysicalTable(result)) {
                             connection.addTable(result.getTable());
                         }
                         return result;
                     }
+                    // If table was not found at the current time stamp and we have one cached, remove it.
+                    // Otherwise, we're up to date, so there's nothing to do.
                     if (code == MutationCode.TABLE_NOT_FOUND && tryCount + 1 == maxTryCount) {
                         connection.removeTable(tenantId, fullTableName, table.getParentName() == null ? null : table.getParentName().getString(), table.getTimeStamp());
                     }
@@ -1683,7 +1686,7 @@ public class MetaDataClient {
             MutationCode code = result.getMutationCode();
             switch(code) {
             case TABLE_ALREADY_EXISTS:
-                connection.addTable(result.getTable());
+                addTableToCache(result);
                 if (!statement.ifNotExists()) {
                     throw new TableAlreadyExistsException(schemaName, tableName, result.getTable());
                 }
@@ -1698,7 +1701,7 @@ public class MetaDataClient {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
                     .setSchemaName(schemaName).setTableName(tableName).build().buildException();
             case CONCURRENT_TABLE_MUTATION:
-                connection.addTable(result.getTable());
+                addTableToCache(result);
                 throw new ConcurrentTableMutationException(schemaName, tableName);
             default:
                 PName newSchemaName = PNameFactory.newName(schemaName);
@@ -1708,12 +1711,8 @@ public class MetaDataClient {
                         dataTableName == null ? null : newSchemaName, dataTableName == null ? null : PNameFactory.newName(dataTableName), Collections.<PTable>emptyList(), isImmutableRows,
                         physicalNames, defaultFamilyName == null ? null : PNameFactory.newName(defaultFamilyName), viewStatement, Boolean.TRUE.equals(disableWAL), multiTenant, storeNulls, viewType,
                         indexId, indexType);
-                connection.addTable(table);
-                if (tableType == PTableType.VIEW) {
-                    // Set wasUpdated to true to force attempt to add
-                    // indexes from physical table to view.
-                    addIndexesFromPhysicalTable(new MetaDataMutationResult(code, result.getMutationTime(), table, true));
-                }
+                result = new MetaDataMutationResult(code, result.getMutationTime(), table, true);
+                addTableToCache(result);
                 return table;
             }
         } finally {
@@ -1958,12 +1957,13 @@ public class MetaDataClient {
         case COLUMN_NOT_FOUND:
             break;
         case CONCURRENT_TABLE_MUTATION:
-            connection.addTable(result.getTable());
+            addTableToCache(result);
             if (logger.isDebugEnabled()) {
                 logger.debug(LogUtil.addCustomAnnotations("CONCURRENT_TABLE_MUTATION for table " + SchemaUtil.getTableName(schemaName, tableName), connection));
             }
             throw new ConcurrentTableMutationException(schemaName, tableName);
         case NEWER_TABLE_FOUND:
+            // TODO: update cache?
 //            if (result.getTable() != null) {
 //                connection.addTable(result.getTable());
 //            }
@@ -2254,7 +2254,7 @@ public class MetaDataClient {
                 try {
                     MutationCode code = processMutationResult(schemaName, tableName, result);
                     if (code == MutationCode.COLUMN_ALREADY_EXISTS) {
-                        connection.addTable(result.getTable());
+                        addTableToCache(result);
                         if (!statement.ifNotExists()) {
                             throw new ColumnAlreadyExistsException(schemaName, tableName, SchemaUtil.findExistingColumn(result.getTable(), columns));
                         }
@@ -2493,7 +2493,7 @@ public class MetaDataClient {
                 try {
                     MutationCode code = processMutationResult(schemaName, tableName, result);
                     if (code == MutationCode.COLUMN_NOT_FOUND) {
-                        connection.addTable(result.getTable());
+                        addTableToCache(result);
                         if (!statement.ifExists()) {
                             throw new ColumnNotFoundException(schemaName, tableName, Bytes.toString(result.getFamilyName()), Bytes.toString(result.getColumnName()));
                         }
@@ -2603,7 +2603,7 @@ public class MetaDataClient {
             }
             if (code == MutationCode.TABLE_ALREADY_EXISTS) {
                 if (result.getTable() != null) { // To accommodate connection-less update of index state
-                    connection.addTable(result.getTable());
+                    addTableToCache(result);
                 }
             }
             if (newIndexState == PIndexState.BUILDING) {
@@ -2633,6 +2633,13 @@ public class MetaDataClient {
         }
     }
 
+    private PTable addTableToCache(MetaDataMutationResult result) throws SQLException {
+        addIndexesFromPhysicalTable(result);
+        PTable table = result.getTable();
+        connection.addTable(table);
+        return table;
+    }
+    
     private void throwIfAlteringViewPK(ColumnDef col, PTable table) throws SQLException {
         if (col != null && col.isPK() && table.getType() == PTableType.VIEW) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MODIFY_VIEW_PK)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c633151d/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 08f74b7..e14565d 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
@@ -190,7 +190,7 @@ public class PTableImpl implements PTable {
     public static PTableImpl makePTable(PTable table, long timeStamp, List<PTable> indexes, PName parentSchemaName, String viewStatement) throws SQLException {
         return new PTableImpl(
                 table.getTenantId(), table.getSchemaName(), table.getTableName(), table.getType(), table.getIndexState(), timeStamp,
-                table.getSequenceNumber() + 1, table.getPKName(), table.getBucketNum(), getColumnsToClone(table), parentSchemaName, table.getParentTableName(),
+                table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table), parentSchemaName, table.getParentTableName(),
                 indexes, table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), viewStatement,
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(), table.getTableStats());
     }