You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2015/08/23 19:16:00 UTC

[35/40] phoenix git commit: PHOENIX-2177 Adding a column to the view shouldn't call admin.modifyTable() for the base table

PHOENIX-2177 Adding a column to the view shouldn't call admin.modifyTable() for the base table


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

Branch: refs/heads/calcite
Commit: 8a1566c03ad24b2ffa5515b55918d9dc4f03efac
Parents: 5c04843
Author: Samarth <sa...@salesforce.com>
Authored: Thu Aug 13 12:24:50 2015 -0700
Committer: Samarth <sa...@salesforce.com>
Committed: Thu Aug 13 12:25:48 2015 -0700

----------------------------------------------------------------------
 .../phoenix/end2end/AlterTableWithViewsIT.java  | 40 ++++++++++++++++++++
 .../query/ConnectionQueryServicesImpl.java      | 28 +++++++++++---
 .../org/apache/phoenix/query/QueryServices.java |  1 +
 .../phoenix/query/QueryServicesOptions.java     |  4 +-
 4 files changed, 66 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8a1566c0/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
index 22abd38..77738be 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
@@ -22,6 +22,7 @@ import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COU
 import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -35,6 +36,8 @@ import java.sql.Statement;
 import java.util.List;
 import java.util.Properties;
 
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -48,6 +51,7 @@ import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
+import org.junit.Assert;
 import org.junit.Test;
 
 import com.google.common.base.Objects;
@@ -1056,6 +1060,42 @@ public class AlterTableWithViewsIT extends BaseHBaseManagedTimeIT {
         }
     }
     
+    @Test
+    public void testAlteringViewConditionallyModifiesHTableMetadata() throws Exception {
+        String baseTable = "testAlteringViewConditionallyModifiesBaseTable".toUpperCase();
+        String view1 = "view1".toUpperCase();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR, V3 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true ";
+            conn.createStatement().execute(baseTableDDL);
+            HTableDescriptor tableDesc1 = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin().getTableDescriptor(Bytes.toBytes(baseTable)); 
+            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+                String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
+                tenant1Conn.createStatement().execute(view1DDL);
+                // This should not modify the base table
+                String alterView = "ALTER VIEW " + view1 + " ADD NEWCOL1 VARCHAR";
+                tenant1Conn.createStatement().execute(alterView);
+                HTableDescriptor tableDesc2 = tenant1Conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin().getTableDescriptor(Bytes.toBytes(baseTable));
+                assertEquals(tableDesc1, tableDesc2);
+                
+                // Add a new column family that doesn't already exist in the base table
+                alterView = "ALTER VIEW " + view1 + " ADD CF.NEWCOL2 VARCHAR";
+                tenant1Conn.createStatement().execute(alterView);
+                
+                // Verify that the column family now shows up in the base table descriptor
+                tableDesc2 = tenant1Conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin().getTableDescriptor(Bytes.toBytes(baseTable));
+                assertFalse(tableDesc2.equals(tableDesc1));
+                assertNotNull(tableDesc2.getFamily(Bytes.toBytes("CF")));
+                
+                // Add a column with an existing column family. This shouldn't modify the base table.
+                alterView = "ALTER VIEW " + view1 + " ADD CF.NEWCOL3 VARCHAR";
+                tenant1Conn.createStatement().execute(alterView);
+                HTableDescriptor tableDesc3 = tenant1Conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin().getTableDescriptor(Bytes.toBytes(baseTable));
+                assertTrue(tableDesc3.equals(tableDesc2));
+                assertNotNull(tableDesc3.getFamily(Bytes.toBytes("CF")));
+            }
+        }
+    }
+    
     private static long getTableSequenceNumber(PhoenixConnection conn, String tableName) throws SQLException {
         PTable table = conn.getMetaDataCache().getTable(new PTableKey(conn.getTenantId(), SchemaUtil.normalizeIdentifier(tableName)));
         return table.getSequenceNumber();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8a1566c0/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index dede98d..22a6720 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -27,7 +27,6 @@ import static org.apache.phoenix.util.UpgradeUtil.upgradeTo4_5_0;
 
 import java.io.IOException;
 import java.sql.SQLException;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -1504,14 +1503,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     
     @Override
     public MetaDataMutationResult addColumn(final List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> stmtProperties, Set<String> colFamiliesForPColumnsToBeAdded) throws SQLException {
-        List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
         Map<String, Object> tableProps = new HashMap<String, Object>();
-        HTableDescriptor tableDescriptor = separateAndValidateProperties(table, stmtProperties, colFamiliesForPColumnsToBeAdded, families, tableProps);
+        HTableDescriptor tableDescriptor = separateAndValidateProperties(table, stmtProperties, colFamiliesForPColumnsToBeAdded, tableProps);
         SQLException sqlE = null;
         if (tableDescriptor != null) {
             try {
-                boolean pollingNotNeeded = (!tableProps.isEmpty() && families.isEmpty() && colFamiliesForPColumnsToBeAdded.isEmpty());
-                modifyTable(table.getPhysicalName().getBytes(), tableDescriptor, !pollingNotNeeded);
+                boolean modifyHTable = true;
+                if (table.getType() == PTableType.VIEW) {
+                    boolean canViewsAddNewCF = props.getBoolean(QueryServices.ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE,
+                            QueryServicesOptions.DEFAULT_ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE);
+                    // When adding a column to a view, base physical table should only be modified when new column families are being added.  
+                    modifyHTable = canViewsAddNewCF && !existingColumnFamiliesForBaseTable(table.getPhysicalName()).containsAll(colFamiliesForPColumnsToBeAdded);
+                }
+                boolean pollingNotNeeded = (!tableProps.isEmpty() && !existingColumnFamilies(table).containsAll(colFamiliesForPColumnsToBeAdded));
+                if (modifyHTable) {
+                    modifyTable(table.getPhysicalName().getBytes(), tableDescriptor, !pollingNotNeeded);
+                }
             } catch (IOException e) {
                 sqlE = ServerUtil.parseServerException(e);
             } catch (InterruptedException e) {
@@ -1590,7 +1597,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return result;
     }
     
-    private HTableDescriptor separateAndValidateProperties(PTable table, Map<String, List<Pair<String, Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<Pair<byte[], Map<String, Object>>> families, Map<String, Object> tableProps) throws SQLException {
+    private HTableDescriptor separateAndValidateProperties(PTable table, Map<String, List<Pair<String, Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, Map<String, Object> tableProps) throws SQLException {
         Map<String, Map<String, Object>> stmtFamiliesPropsMap = new HashMap<>(properties.size());
         Map<String,Object> commonFamilyProps = new HashMap<>();
         boolean addingColumns = colFamiliesForPColumnsToBeAdded != null && colFamiliesForPColumnsToBeAdded.size() > 0;
@@ -1763,6 +1770,15 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return !isHColumnProperty(propName) && !TableProperty.isPhoenixTableProperty(propName);
     } 
     
+    private HashSet<String> existingColumnFamiliesForBaseTable(PName baseTableName) throws TableNotFoundException {
+        synchronized (latestMetaDataLock) {
+            throwConnectionClosedIfNullMetaData();
+            PTable table = latestMetaData.getTable(new PTableKey(null, baseTableName.getString()));
+            latestMetaDataLock.notifyAll();
+            return existingColumnFamilies(table);
+        }
+    }
+    
     private HashSet<String> existingColumnFamilies(PTable table) {
         List<PColumnFamily> cfs = table.getColumnFamilies();
         HashSet<String> cfNames = new HashSet<>(cfs.size());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8a1566c0/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 62b080c..080586d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -168,6 +168,7 @@ public interface QueryServices extends SQLCloseable {
     public static final String FORCE_ROW_KEY_ORDER_ATTRIB = "phoenix.query.force.rowkeyorder";
     public static final String ALLOW_USER_DEFINED_FUNCTIONS_ATTRIB = "phoenix.functions.allowUserDefinedFunctions";
     public static final String COLLECT_REQUEST_LEVEL_METRICS = "phoenix.query.request.metrics.enabled";
+    public static final String ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE = "phoenix.view.allowNewColumnFamily";
     
 
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8a1566c0/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index ea81cf5..4b8f092 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.query;
 
 import static org.apache.phoenix.query.QueryServices.ALLOW_ONLINE_TABLE_SCHEMA_UPDATE;
+import static org.apache.phoenix.query.QueryServices.ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE;
 import static org.apache.phoenix.query.QueryServices.CALL_QUEUE_PRODUCER_ATTRIB_NAME;
 import static org.apache.phoenix.query.QueryServices.CALL_QUEUE_ROUND_ROBIN_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.COLLECT_REQUEST_LEVEL_METRICS;
@@ -74,7 +75,6 @@ import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.ipc.controller.ClientRpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
-import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.trace.util.Tracing;
 import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -198,6 +198,7 @@ public class QueryServicesOptions {
     public static final boolean DEFAULT_FORCE_ROW_KEY_ORDER = false;
     public static final boolean DEFAULT_ALLOW_USER_DEFINED_FUNCTIONS = false;
     public static final boolean DEFAULT_REQUEST_LEVEL_METRICS_ENABLED = false;
+    public static final boolean DEFAULT_ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE = true;
 
     private final Configuration config;
 
@@ -255,6 +256,7 @@ public class QueryServicesOptions {
             .setIfUnset(USE_BYTE_BASED_REGEX_ATTRIB, DEFAULT_USE_BYTE_BASED_REGEX)
             .setIfUnset(FORCE_ROW_KEY_ORDER_ATTRIB, DEFAULT_FORCE_ROW_KEY_ORDER)
             .setIfUnset(COLLECT_REQUEST_LEVEL_METRICS, DEFAULT_REQUEST_LEVEL_METRICS_ENABLED)
+            .setIfUnset(ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE, DEFAULT_ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE);
             ;
         // HBase sets this to 1, so we reset it to something more appropriate.
         // Hopefully HBase will change this, because we can't know if a user set