PHOENIX-3524 Changing the APPEND_ONLY_SCHEMA property using an ALTER TABLE does not work
Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/0ea97ce6 Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/0ea97ce6 Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/0ea97ce6 Branch: refs/heads/encodecolumns2 Commit: 0ea97ce675bbd01eef2e566aff7f5910faae4369 Parents: e202020 Author: Thomas D'Silva <tdsi...@salesforce.com> Authored: Mon Dec 19 15:43:31 2016 -0800 Committer: Thomas D'Silva <tdsi...@salesforce.com> Committed: Tue Dec 20 09:04:42 2016 -0800 ---------------------------------------------------------------------- .../phoenix/end2end/AlterTableWithViewsIT.java | 97 +++++++++++++++----- .../phoenix/end2end/AppendOnlySchemaIT.java | 9 +- .../phoenix/end2end/ImmutableTablePropIT.java | 42 ++++++++- .../coprocessor/MetaDataEndpointImpl.java | 32 +++++-- .../phoenix/exception/SQLExceptionCode.java | 2 + .../apache/phoenix/jdbc/PhoenixConnection.java | 14 --- .../phoenix/query/ConnectionQueryServices.java | 3 +- .../query/ConnectionQueryServicesImpl.java | 22 +---- .../query/ConnectionlessQueryServicesImpl.java | 16 +--- .../query/DelegateConnectionQueryServices.java | 14 +-- .../apache/phoenix/query/MetaDataMutated.java | 1 - .../apache/phoenix/schema/MetaDataClient.java | 62 ++++++++----- .../apache/phoenix/schema/PMetaDataImpl.java | 24 ----- .../phoenix/schema/PSynchronizedMetaData.java | 17 ---- .../apache/phoenix/schema/TableProperty.java | 90 +++++++++++++++--- 15 files changed, 269 insertions(+), 176 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/phoenix/blob/0ea97ce6/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 91aca4e..52dae00 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 @@ -19,7 +19,6 @@ package org.apache.phoenix.end2end; import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_MUTATE_TABLE; import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB; -import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -33,7 +32,6 @@ import java.sql.ResultSetMetaData; import java.sql.SQLException; import java.util.Arrays; import java.util.Collection; -import java.util.Properties; import org.apache.commons.lang.ArrayUtils; import org.apache.hadoop.hbase.client.HTableInterface; @@ -47,8 +45,6 @@ import org.apache.phoenix.schema.PNameFactory; import org.apache.phoenix.schema.PTable; import org.apache.phoenix.schema.PTableKey; import org.apache.phoenix.schema.PTableType; -import org.apache.phoenix.util.PropertiesUtil; -import org.apache.phoenix.util.SchemaUtil; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -106,37 +102,52 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT { try (Connection conn = DriverManager.getConnection(getUrl()); Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) { String tableName = generateUniqueName(); - String viewOfTable = tableName + "_VIEW"; + String viewOfTable1 = tableName + "_VIEW1"; + String viewOfTable2 = tableName + "_VIEW2"; String ddlFormat = "CREATE TABLE IF NOT EXISTS " + tableName + " (" + " %s ID char(1) NOT NULL," + " COL1 integer NOT NULL," + " COL2 bigint NOT NULL," + " CONSTRAINT NAME_PK PRIMARY KEY (%s ID, COL1, COL2)" - + " ) %s"; + + " ) UPDATE_CACHE_FREQUENCY=15 " + + (isMultiTenant ? ",%s" : "%s"); conn.createStatement().execute(generateDDL(ddlFormat)); - viewConn.createStatement().execute("CREATE VIEW " + viewOfTable + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName); + viewConn.createStatement().execute("CREATE VIEW " + viewOfTable1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName); + viewConn.createStatement().execute("CREATE VIEW " + viewOfTable2 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName); + viewConn.createStatement().execute("ALTER VIEW " + viewOfTable2 + " SET UPDATE_CACHE_FREQUENCY = 5"); PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class); PTable table = phoenixConn.getTable(new PTableKey(null, tableName)); PName tenantId = isMultiTenant ? PNameFactory.newName("tenant1") : null; assertFalse(table.isImmutableRows()); - assertEquals(0, table.getUpdateCacheFrequency()); - PTable viewTable = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable)); - assertFalse(viewTable.isImmutableRows()); - assertEquals(0, viewTable.getUpdateCacheFrequency()); - - conn.createStatement().execute("ALTER TABLE " + tableName + " SET IMMUTABLE_ROWS=true, UPDATE_CACHE_FREQUENCY=123"); + assertEquals(15, table.getUpdateCacheFrequency()); + PTable viewTable1 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable1)); + assertFalse(viewTable1.isImmutableRows()); + assertEquals(15, viewTable1.getUpdateCacheFrequency()); // query the view to force the table cache to be updated - viewConn.createStatement().execute("SELECT * FROM "+viewOfTable); + viewConn.createStatement().execute("SELECT * FROM "+viewOfTable2); + PTable viewTable2 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable2)); + assertFalse(viewTable2.isImmutableRows()); + assertEquals(5, viewTable2.getUpdateCacheFrequency()); + + conn.createStatement().execute("ALTER TABLE " + tableName + " SET IMMUTABLE_ROWS=true, UPDATE_CACHE_FREQUENCY=10"); + // query the views to force the table cache to be updated + viewConn.createStatement().execute("SELECT * FROM "+viewOfTable1); + viewConn.createStatement().execute("SELECT * FROM "+viewOfTable2); phoenixConn = conn.unwrap(PhoenixConnection.class); table = phoenixConn.getTable(new PTableKey(null, tableName)); assertTrue(table.isImmutableRows()); - assertEquals(123, table.getUpdateCacheFrequency()); + assertEquals(10, table.getUpdateCacheFrequency()); - viewTable = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable)); - assertTrue(viewTable.isImmutableRows()); - assertEquals(123, viewTable.getUpdateCacheFrequency()); + viewTable1 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable1)); + assertTrue(viewTable1.isImmutableRows()); + assertEquals(10, viewTable1.getUpdateCacheFrequency()); + + viewTable2 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable2)); + assertTrue(viewTable2.isImmutableRows()); + // update cache frequency is not propagated to the view since it was altered on the view + assertEquals(5, viewTable2.getUpdateCacheFrequency()); } } @@ -731,10 +742,11 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT { String viewOfTable = baseTableName + "_VIEW"; String ddl = "CREATE TABLE " + baseTableName + " (\n" - +"ID VARCHAR(15) NOT NULL,\n" + +"%s ID VARCHAR(15) NOT NULL,\n" + + " COL1 integer NOT NULL," +"CREATED_DATE DATE,\n" - +"CONSTRAINT PK PRIMARY KEY (ID))"; - conn.createStatement().execute(ddl); + +"CONSTRAINT PK PRIMARY KEY (%s ID, COL1)) %s"; + conn.createStatement().execute(generateDDL(ddl)); ddl = "CREATE VIEW " + viewOfTable + " AS SELECT * FROM " + baseTableName; viewConn.createStatement().execute(ddl); @@ -757,7 +769,48 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT { assertEquals(SQLExceptionCode.CANNOT_ALTER_TABLE_PROPERTY_ON_VIEW.getErrorCode(), e.getErrorCode()); } } - + } + + @Test + public void testAlterAppendOnlySchema() throws Exception { + try (Connection conn = DriverManager.getConnection(getUrl()); + Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) { + String baseTableName = "NONTXNTBL_" + generateUniqueName() + (isMultiTenant ? "0":"1"); + String viewOfTable = baseTableName + "_VIEW"; + + String ddl = "CREATE TABLE " + baseTableName + " (\n" + +"%s ID VARCHAR(15) NOT NULL,\n" + + " COL1 integer NOT NULL," + +"CREATED_DATE DATE,\n" + +"CONSTRAINT PK PRIMARY KEY (%s ID, COL1)) %s"; + conn.createStatement().execute(generateDDL(ddl)); + ddl = "CREATE VIEW " + viewOfTable + " AS SELECT * FROM " + baseTableName; + viewConn.createStatement().execute(ddl); + + PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class); + PTable table = phoenixConn.getTable(new PTableKey(null, baseTableName)); + PName tenantId = isMultiTenant ? PNameFactory.newName("tenant1") : null; + assertFalse(table.isAppendOnlySchema()); + PTable viewTable = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable)); + assertFalse(viewTable.isAppendOnlySchema()); + + try { + viewConn.createStatement().execute("ALTER VIEW " + viewOfTable + " SET APPEND_ONLY_SCHEMA = true"); + fail(); + } + catch(SQLException e){ + assertEquals(SQLExceptionCode.CANNOT_ALTER_TABLE_PROPERTY_ON_VIEW.getErrorCode(), e.getErrorCode()); + } + + conn.createStatement().execute("ALTER TABLE " + baseTableName + " SET APPEND_ONLY_SCHEMA = true"); + viewConn.createStatement().execute("SELECT * FROM "+viewOfTable); + + phoenixConn = conn.unwrap(PhoenixConnection.class); + table = phoenixConn.getTable(new PTableKey(null, baseTableName)); + assertTrue(table.isAppendOnlySchema()); + viewTable = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable)); + assertTrue(viewTable.isAppendOnlySchema()); + } } } http://git-wip-us.apache.org/repos/asf/phoenix/blob/0ea97ce6/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java index 8d40094..7ed64ff 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java @@ -133,7 +133,7 @@ public class AppendOnlySchemaIT extends ParallelStatsDisabledIT { // if not verify exists is true one call to add column table with empty mutation list (which does not make a rpc) // else verify no add column calls - verify(connectionQueryServices, notExists ? times(1) : never() ).addColumn(eq(Collections.<Mutation>emptyList()), any(PTable.class), anyMap(), anySetOf(String.class)); + verify(connectionQueryServices, notExists ? times(1) : never() ).addColumn(eq(Collections.<Mutation>emptyList()), any(PTable.class), anyMap(), anySetOf(String.class), anyListOf(PColumn.class)); // upsert one row conn2.createStatement().execute("UPSERT INTO " + viewName + "(hostName, metricVal) VALUES('host2', 2.0)"); @@ -309,10 +309,11 @@ public class AppendOnlySchemaIT extends ParallelStatsDisabledIT { + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1))" + " APPEND_ONLY_SCHEMA = true, UPDATE_CACHE_FREQUENCY=1000"); conn.createStatement().execute( - "create view IF NOT EXISTS " + viewName + " (val1 integer) AS SELECT * FROM " + tableName - + " UPDATE_CACHE_FREQUENCY=1000"); + "create view IF NOT EXISTS " + viewName + " (val1 integer) AS SELECT * FROM " + tableName); PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class); - assertEquals(true, pconn.getTable(new PTableKey(pconn.getTenantId(), viewName)).isAppendOnlySchema()); + PTable view = pconn.getTable(new PTableKey(pconn.getTenantId(), viewName)); + assertEquals(true, view.isAppendOnlySchema()); + assertEquals(1000, view.getUpdateCacheFrequency()); } } http://git-wip-us.apache.org/repos/asf/phoenix/blob/0ea97ce6/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropIT.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropIT.java index 693fb81..78be892 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropIT.java @@ -18,14 +18,18 @@ package org.apache.phoenix.end2end; import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import java.sql.Connection; import java.sql.DriverManager; +import java.sql.SQLException; import java.sql.Statement; import java.util.Properties; +import org.apache.phoenix.exception.SQLExceptionCode; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.schema.PTable; import org.apache.phoenix.schema.PTableKey; @@ -69,13 +73,13 @@ public class ImmutableTablePropIT extends ParallelStatsDisabledIT { String mutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName()); try (Connection conn = DriverManager.getConnection(getUrl(), props);) { Statement stmt = conn.createStatement(); - // create table with immutable_table property set to true + // create table with immutable table property set to true String ddl = "CREATE TABLE " + immutableDataTableFullName + " (a_string varchar not null, col1 integer" + " CONSTRAINT pk PRIMARY KEY (a_string)) IMMUTABLE_ROWS=true"; stmt.execute(ddl); - // create table with immutable_table property set to false + // create table with immutable table property set to false ddl = "CREATE TABLE " + mutableDataTableFullName + " (a_string varchar not null, col1 integer" + " CONSTRAINT pk PRIMARY KEY (a_string)) IMMUTABLE_ROWS=false"; @@ -89,4 +93,38 @@ public class ImmutableTablePropIT extends ParallelStatsDisabledIT { } } + @Test + public void testImmutableKeywordAndProperty() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String immutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName()); + String mutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName()); + try (Connection conn = DriverManager.getConnection(getUrl(), props);) { + Statement stmt = conn.createStatement(); + try { + // create immutable table with immutable table property set to true + String ddl = "CREATE IMMUTABLE TABLE " + immutableDataTableFullName + + " (a_string varchar not null, col1 integer" + + " CONSTRAINT pk PRIMARY KEY (a_string)) IMMUTABLE_ROWS=true"; + stmt.execute(ddl); + fail(); + } + catch (SQLException e) { + assertEquals(SQLExceptionCode.IMMUTABLE_TABLE_PROPERTY_INVALID.getErrorCode(), e.getErrorCode()); + } + + try { + // create immutable table with immutable table property set to false + String ddl = "CREATE IMMUTABLE TABLE " + mutableDataTableFullName + + " (a_string varchar not null, col1 integer" + + " CONSTRAINT pk PRIMARY KEY (a_string)) IMMUTABLE_ROWS=false"; + stmt.execute(ddl); + fail(); + } + catch (SQLException e) { + assertEquals(SQLExceptionCode.IMMUTABLE_TABLE_PROPERTY_INVALID.getErrorCode(), e.getErrorCode()); + } + + } + } + } http://git-wip-us.apache.org/repos/asf/phoenix/blob/0ea97ce6/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java index 9ade0f5..8afdec4 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java @@ -87,6 +87,7 @@ import java.util.Collections; import java.util.Comparator; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.NavigableMap; import java.util.Properties; @@ -226,6 +227,7 @@ import org.slf4j.LoggerFactory; import com.google.common.cache.Cache; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.protobuf.ByteString; import com.google.protobuf.RpcCallback; import com.google.protobuf.RpcController; @@ -1899,8 +1901,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso byte[] tenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX]; byte[] schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX]; byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX]; + byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName); try { - byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName); HRegion region = env.getRegion(); MetaDataMutationResult result = checkTableKeyInRegion(key, region); if (result != null) { @@ -1995,7 +1997,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso // mutateRowsWithLocks call long currentTime = MetaDataUtil.getClientTimeStamp(tableMetadata); // if the update mutation caused tables to be deleted just return the result which will contain the table to be deleted - return result != null ? result : new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, currentTime, null); + if (result !=null) { + return result; + } else { + table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP); + return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, currentTime, table); + } } finally { region.releaseRowLocks(locks); } @@ -2154,7 +2161,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso List<ImmutableBytesPtr> invalidateList, long clientTimeStamp, TableViewFinderResult childViewsResult, HRegion region, List<RowLock> locks) throws IOException, SQLException { List<PutWithOrdinalPosition> columnPutsForBaseTable = Lists.newArrayListWithExpectedSize(tableMetadata.size()); - List<Cell> tablePropertyCells = Lists.newArrayListWithExpectedSize(tableMetadata.size()); + Map<TableProperty, Cell> tablePropertyCellMap = Maps.newHashMapWithExpectedSize(tableMetadata.size()); // Isolate the puts relevant to adding columns. Also figure out what kind of columns are being added. for (Mutation m : tableMetadata) { if (m instanceof Put) { @@ -2178,7 +2185,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso Cell tablePropCell = CellUtil.createCell(cell.getRow(), CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell), cell.getTimestamp(), cell.getTypeByte(), CellUtil.cloneValue(cell)); - tablePropertyCells.add(tablePropCell); + tablePropertyCellMap.put(tableProp, tablePropCell); } } } @@ -2353,12 +2360,19 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso deltaNumPkColsSoFar); // set table properties in child view - if (!tablePropertyCells.isEmpty()) { + if (!tablePropertyCellMap.isEmpty()) { Put viewHeaderRowPut = new Put(viewKey, clientTimeStamp); - for (Cell tablePropertyCell : tablePropertyCells) { - viewHeaderRowPut.add(CellUtil.createCell(viewKey, CellUtil.cloneFamily(tablePropertyCell), - CellUtil.cloneQualifier(tablePropertyCell), clientTimeStamp, tablePropertyCell.getTypeByte(), - CellUtil.cloneValue(tablePropertyCell))); + for (TableProperty tableProp : TableProperty.values()) { + Cell tablePropertyCell = tablePropertyCellMap.get(tableProp); + if ( tablePropertyCell != null) { + // set this table property on the view if it is not mutable on a view (which means the property is always the same as the base table) + // or if it is mutable on a view and the property value is the same as the base table property (which means it wasn't changed on the view) + if (!tableProp.isMutableOnView() || tableProp.getPTableValue(view).equals(tableProp.getPTableValue(basePhysicalTable))) { + viewHeaderRowPut.add(CellUtil.createCell(viewKey, CellUtil.cloneFamily(tablePropertyCell), + CellUtil.cloneQualifier(tablePropertyCell), clientTimeStamp, tablePropertyCell.getTypeByte(), + CellUtil.cloneValue(tablePropertyCell))); + } + } } byte[] viewSequencePtr = new byte[PLong.INSTANCE.getByteSize()]; PLong.INSTANCE.getCodec().encodeLong(view.getSequenceNumber() + 1, viewSequencePtr, 0); http://git-wip-us.apache.org/repos/asf/phoenix/blob/0ea97ce6/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java index 20f3974..fb4e3c3 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java @@ -366,6 +366,8 @@ public enum SQLExceptionCode { CANNOT_ALTER_IMMUTABLE_ROWS_PROPERTY(1133, "XCL33", "IMMUTABLE_ROWS property can be changed only if the table storage scheme is ONE_CELL_PER_KEYVALUE_COLUMN"), CANNOT_ALTER_TABLE_PROPERTY_ON_VIEW(1134, "XCL34", "Altering this table property on a view is not allowed"), + + IMMUTABLE_TABLE_PROPERTY_INVALID(1135, "XCL35", "IMMUTABLE table property cannot be used with CREATE IMMUTABLE TABLE statement "), /** * Implementation defined class. Phoenix internal error. (errorcode 20, sqlstate INT). http://git-wip-us.apache.org/repos/asf/phoenix/blob/0ea97ce6/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java index 7cdab20..f465176 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java @@ -919,20 +919,6 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea } @Override - public void addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, - long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, - boolean isTransactional, long updateCacheFrequency, boolean isNamespaceMapped, long resolvedTime) - throws SQLException { - metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, - isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency, isNamespaceMapped, - resolvedTime); - // Cascade through to connectionQueryServices too - getQueryServices().addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, - isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency, isNamespaceMapped, - resolvedTime); - } - - @Override public void removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException { metaData.removeTable(tenantId, tableName, parentTableName, tableTimeStamp); //Cascade through to connectionQueryServices too http://git-wip-us.apache.org/repos/asf/phoenix/blob/0ea97ce6/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java index 51716d0..38580e4 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java @@ -37,6 +37,7 @@ import org.apache.phoenix.execute.MutationState; import org.apache.phoenix.hbase.index.util.KeyValueBuilder; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.parse.PFunction; +import org.apache.phoenix.schema.PColumn; import org.apache.phoenix.schema.PName; import org.apache.phoenix.schema.PTable; import org.apache.phoenix.schema.PTableType; @@ -83,7 +84,7 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated boolean isNamespaceMapped, boolean allocateIndexId) throws SQLException; public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType, boolean cascade) throws SQLException; public MetaDataMutationResult dropFunction(List<Mutation> tableMetadata, boolean ifExists) throws SQLException; - public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded) throws SQLException; + public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<PColumn> columns) throws SQLException; public MetaDataMutationResult dropColumn(List<Mutation> tableMetadata, PTableType tableType) throws SQLException; public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata, String parentTableName) throws SQLException; public MutationState updateData(MutationPlan plan) throws SQLException; http://git-wip-us.apache.org/repos/asf/phoenix/blob/0ea97ce6/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 c8cd04d..da43af7 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 @@ -641,26 +641,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement } @Override - public void addColumn(final PName tenantId, final String tableName, final List<PColumn> columns, - final long tableTimeStamp, final long tableSeqNum, final boolean isImmutableRows, - final boolean isWalDisabled, final boolean isMultitenant, final boolean storeNulls, - final boolean isTransactional, final long updateCacheFrequency, final boolean isNamespaceMapped, - final long resolvedTime) throws SQLException { - metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() { - @Override - public void mutate(PMetaData metaData) throws SQLException { - try { - metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, - isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional, - updateCacheFrequency, isNamespaceMapped, resolvedTime); - } catch (TableNotFoundException e) { - // The DROP TABLE may have been processed first, so just ignore. - } - } - }); - } - - @Override public void removeTable(PName tenantId, final String tableName, String parentTableName, long tableTimeStamp) throws SQLException { synchronized (latestMetaDataLock) { throwConnectionClosedIfNullMetaData(); @@ -1707,7 +1687,7 @@ 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 { + public MetaDataMutationResult addColumn(final List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> stmtProperties, Set<String> colFamiliesForPColumnsToBeAdded, List<PColumn> columns) throws SQLException { List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size()); Map<String, Object> tableProps = new HashMap<String, Object>(); Set<HTableDescriptor> tableDescriptors = Collections.emptySet(); http://git-wip-us.apache.org/repos/asf/phoenix/blob/0ea97ce6/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java index 6398a23..042d7a6 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java @@ -180,16 +180,6 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple } @Override - public void addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, - long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, - boolean isTransactional, long updateCacheFrequency, boolean isNamespaceMapped, long resolvedTime) - throws SQLException { - metaData.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, - isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency, isNamespaceMapped, - resolvedTime); - } - - @Override public void removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException { metaData.removeTable(tenantId, tableName, parentTableName, tableTimeStamp); @@ -271,8 +261,10 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple } @Override - public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded) throws SQLException { - return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, null); + public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<PColumn> columnsToBeAdded) throws SQLException { + List<PColumn> columns = Lists.newArrayList(table.getColumns()); + columns.addAll(columnsToBeAdded); + return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, PTableImpl.makePTable(table, columns)); } @Override http://git-wip-us.apache.org/repos/asf/phoenix/blob/0ea97ce6/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java index 685e583..4d77267 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java @@ -87,16 +87,6 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple } @Override - public void addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, - long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, - boolean isTransactional, long updateCacheFrequency, boolean isNamespaceMapped, long resolvedTime) - throws SQLException { - getDelegate().addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, isImmutableRows, - isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency, isNamespaceMapped, - resolvedTime); - } - - @Override public void removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException { getDelegate().removeTable(tenantId, tableName, parentTableName, tableTimeStamp); @@ -132,8 +122,8 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple } @Override - public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded) throws SQLException { - return getDelegate().addColumn(tableMetaData, table, properties, colFamiliesForPColumnsToBeAdded); + public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<PColumn> columns) throws SQLException { + return getDelegate().addColumn(tableMetaData, table, properties, colFamiliesForPColumnsToBeAdded, columns); } http://git-wip-us.apache.org/repos/asf/phoenix/blob/0ea97ce6/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java b/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java index 0b6a644..e7ce65c 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/MetaDataMutated.java @@ -38,7 +38,6 @@ public interface MetaDataMutated { void addTable(PTable table, long resolvedTime) throws SQLException; void updateResolvedTimestamp(PTable table, long resolvedTimestamp) throws SQLException; void removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException; - void addColumn(PName tenantId, String tableName, List<PColumn> columns, long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, boolean isTransactional, long updateCacheFrequency, boolean isNamespaceMapped, long resolvedTime) throws SQLException; void removeColumn(PName tenantId, String tableName, List<PColumn> columnsToRemove, long tableTimeStamp, long tableSeqNum, long resolvedTime) throws SQLException; void addFunction(PFunction function) throws SQLException; void removeFunction(PName tenantId, String function, long functionTimeStamp) throws SQLException; http://git-wip-us.apache.org/repos/asf/phoenix/blob/0ea97ce6/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 0803e89..7f183e9 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 @@ -889,19 +889,20 @@ public class MetaDataClient { populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps); boolean isAppendOnlySchema = false; + long updateCacheFrequency = 0; if (parent==null) { Boolean appendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps); if (appendOnlySchemaProp != null) { isAppendOnlySchema = appendOnlySchemaProp; } + Long updateCacheFrequencyProp = (Long) TableProperty.UPDATE_CACHE_FREQUENCY.getValue(tableProps); + if (updateCacheFrequencyProp != null) { + updateCacheFrequency = updateCacheFrequencyProp; + } } else { isAppendOnlySchema = parent.isAppendOnlySchema(); - } - long updateCacheFrequency = 0; - Long updateCacheFrequencyProp = (Long) TableProperty.UPDATE_CACHE_FREQUENCY.getValue(tableProps); - if (updateCacheFrequencyProp != null) { - updateCacheFrequency = updateCacheFrequencyProp; + updateCacheFrequency = parent.getUpdateCacheFrequency(); } // updateCacheFrequency cannot be set to ALWAYS if isAppendOnlySchema is true if (isAppendOnlySchema && updateCacheFrequency==0) { @@ -909,6 +910,12 @@ public class MetaDataClient { .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName()) .build().buildException(); } + Boolean immutableProp = (Boolean) TableProperty.IMMUTABLE_ROWS.getValue(tableProps); + if (statement.immutableRows()!=null && immutableProp!=null) { + throw new SQLExceptionInfo.Builder(SQLExceptionCode.IMMUTABLE_TABLE_PROPERTY_INVALID) + .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName()) + .build().buildException(); + } PTable table = null; // if the APPEND_ONLY_SCHEMA attribute is true first check if the table is present in the cache @@ -1660,6 +1667,7 @@ public class MetaDataClient { storeNulls = parent.getStoreNulls(); isImmutableRows = parent.isImmutableRows(); isAppendOnlySchema = parent.isAppendOnlySchema(); + // Index on view // TODO: Can we support a multi-tenant index directly on a multi-tenant // table instead of only a view? We don't have anywhere to put the link @@ -1733,7 +1741,6 @@ public class MetaDataClient { isImmutableRows = isImmutableRowsProp; } } - if (tableType == PTableType.TABLE) { Boolean isAppendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps); isAppendOnlySchema = isAppendOnlySchemaProp!=null ? isAppendOnlySchemaProp : false; @@ -1917,6 +1924,10 @@ public class MetaDataClient { saltBucketNum = parent.getBucketNum(); isAppendOnlySchema = parent.isAppendOnlySchema(); isImmutableRows = parent.isImmutableRows(); + if (updateCacheFrequencyProp == null) { + // set to the parent value if the property is not set on the view + updateCacheFrequency = parent.getUpdateCacheFrequency(); + } disableWAL = (disableWALProp == null ? parent.isWALDisabled() : disableWALProp); defaultFamilyName = parent.getDefaultFamilyName() == null ? null : parent.getDefaultFamilyName().getString(); List<PColumn> allColumns = parent.getColumns(); @@ -2647,12 +2658,12 @@ public class MetaDataClient { } private long incrementTableSeqNum(PTable table, PTableType expectedType, int columnCountDelta, Boolean isTransactional, Long updateCacheFrequency) throws SQLException { - return incrementTableSeqNum(table, expectedType, columnCountDelta, isTransactional, updateCacheFrequency, null, null, null, null, -1L); + return incrementTableSeqNum(table, expectedType, columnCountDelta, isTransactional, updateCacheFrequency, null, null, null, null, -1L, null); } private long incrementTableSeqNum(PTable table, PTableType expectedType, int columnCountDelta, Boolean isTransactional, Long updateCacheFrequency, Boolean isImmutableRows, Boolean disableWAL, - Boolean isMultiTenant, Boolean storeNulls, Long guidePostWidth) + Boolean isMultiTenant, Boolean storeNulls, Long guidePostWidth, Boolean appendOnlySchema) throws SQLException { String schemaName = table.getSchemaName().getString(); String tableName = table.getTableName().getString(); @@ -2693,6 +2704,9 @@ public class MetaDataClient { if (guidePostWidth == null || guidePostWidth >= 0) { mutateLongProperty(tenantId, schemaName, tableName, GUIDE_POSTS_WIDTH, guidePostWidth); } + if (appendOnlySchema !=null) { + mutateBooleanProperty(tenantId, schemaName, tableName, APPEND_ONLY_SCHEMA, appendOnlySchema); + } return seqNum; } @@ -2757,6 +2771,7 @@ public class MetaDataClient { Boolean storeNullsProp = null; Boolean isTransactionalProp = null; Long updateCacheFrequencyProp = null; + Boolean appendOnlySchemaProp = null; Long guidePostWidth = -1L; Map<String, List<Pair<String, Object>>> properties = new HashMap<>(stmtProperties.size()); @@ -2818,6 +2833,8 @@ public class MetaDataClient { updateCacheFrequencyProp = (Long)value; } else if (propName.equals(GUIDE_POSTS_WIDTH)) { guidePostWidth = (Long)value; + } else if (propName.equals(APPEND_ONLY_SCHEMA)) { + appendOnlySchemaProp = (Boolean) value; } } // if removeTableProps is true only add the property if it is not a HTable or Phoenix Table property @@ -2884,6 +2901,14 @@ public class MetaDataClient { changingPhoenixTableProperty = true; } } + Boolean appendOnlySchema = null; + if (appendOnlySchemaProp !=null) { + if (appendOnlySchemaProp != table.isAppendOnlySchema()) { + appendOnlySchema = appendOnlySchemaProp; + changingPhoenixTableProperty = true; + } + } + if (guidePostWidth == null || guidePostWidth >= 0) { changingPhoenixTableProperty = true; } @@ -3031,7 +3056,7 @@ public class MetaDataClient { long seqNum = table.getSequenceNumber(); if (changingPhoenixTableProperty || columnDefs.size() > 0) { seqNum = incrementTableSeqNum(table, tableType, columnDefs.size(), isTransactional, updateCacheFrequency, isImmutableRows, - disableWAL, multiTenant, storeNulls, guidePostWidth); + disableWAL, multiTenant, storeNulls, guidePostWidth, appendOnlySchema); tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond()); connection.rollback(); } @@ -3059,7 +3084,7 @@ public class MetaDataClient { } } - MetaDataMutationResult result = connection.getQueryServices().addColumn(tableMetaData, table, properties, colFamiliesForPColumnsToBeAdded); + MetaDataMutationResult result = connection.getQueryServices().addColumn(tableMetaData, table, properties, colFamiliesForPColumnsToBeAdded, columns); try { MutationCode code = processMutationResult(schemaName, tableName, result); if (code == MutationCode.COLUMN_ALREADY_EXISTS) { @@ -3076,20 +3101,7 @@ public class MetaDataClient { String fullTableName = SchemaUtil.getTableName(schemaName, tableName); long resolvedTimeStamp = TransactionUtil.getResolvedTime(connection, result); if (table.getIndexes().isEmpty() || (numPkColumnsAdded==0 && !nonTxToTx)) { - connection.addColumn( - tenantId, - fullTableName, - columns, - result.getMutationTime(), - seqNum, - isImmutableRows == null ? table.isImmutableRows() : isImmutableRows, - disableWAL == null ? table.isWALDisabled() : disableWAL, - multiTenant == null ? table.isMultiTenant() : multiTenant, - storeNulls == null ? table.getStoreNulls() : storeNulls, - isTransactional == null ? table.isTransactional() : isTransactional, - updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency, - table.isNamespaceMapped(), - resolvedTimeStamp); + connection.addTable(result.getTable(), resolvedTimeStamp); } else if (updateCacheFrequency != null) { // Force removal from cache as the update cache frequency has changed // Note that clients outside this JVM won't be affected. @@ -3334,7 +3346,7 @@ public class MetaDataClient { Collections.<Mutation>singletonList(new Put(SchemaUtil.getTableKey (tenantIdBytes, tableContainingColumnToDrop.getSchemaName().getBytes(), tableContainingColumnToDrop.getTableName().getBytes()))), - tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF))); + tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)), Collections.<PColumn>emptyList()); } } http://git-wip-us.apache.org/repos/asf/phoenix/blob/0ea97ce6/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java index 064137e..6ca38cc 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java @@ -144,30 +144,6 @@ public class PMetaDataImpl implements PMetaData { } @Override - public void addColumn(PName tenantId, String tableName, List<PColumn> columnsToAdd, long tableTimeStamp, - long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, boolean isMultitenant, boolean storeNulls, - boolean isTransactional, long updateCacheFrequency, boolean isNamespaceMapped, long resolvedTime) - throws SQLException { - PTableRef oldTableRef = metaData.get(new PTableKey(tenantId, tableName)); - if (oldTableRef == null) { - return; - } - List<PColumn> oldColumns = PTableImpl.getColumnsToClone(oldTableRef.getTable()); - List<PColumn> newColumns; - if (columnsToAdd.isEmpty()) { - newColumns = oldColumns; - } else { - newColumns = Lists.newArrayListWithExpectedSize(oldColumns.size() + columnsToAdd.size()); - newColumns.addAll(oldColumns); - newColumns.addAll(columnsToAdd); - } - PTable newTable = PTableImpl.makePTable(oldTableRef.getTable(), tableTimeStamp, tableSeqNum, newColumns, - isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional, updateCacheFrequency, - isNamespaceMapped); - addTable(newTable, resolvedTime); - } - - @Override public void removeTable(PName tenantId, String tableName, String parentTableName, long tableTimeStamp) throws SQLException { PTableRef parentTableRef = null; PTableKey key = new PTableKey(tenantId, tableName); http://git-wip-us.apache.org/repos/asf/phoenix/blob/0ea97ce6/phoenix-core/src/main/java/org/apache/phoenix/schema/PSynchronizedMetaData.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PSynchronizedMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PSynchronizedMetaData.java index af4bc60..e4ccf8b 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PSynchronizedMetaData.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PSynchronizedMetaData.java @@ -161,23 +161,6 @@ public class PSynchronizedMetaData implements PMetaData { } @Override - public void addColumn(PName tenantId, String tableName, List<PColumn> columns, - long tableTimeStamp, long tableSeqNum, boolean isImmutableRows, boolean isWalDisabled, - boolean isMultitenant, boolean storeNulls, boolean isTransactional, - long updateCacheFrequency, boolean isNamespaceMapped, long resolvedTime) - throws SQLException { - readWriteLock.writeLock().lock(); - try { - delegate.addColumn(tenantId, tableName, columns, tableTimeStamp, tableSeqNum, - isImmutableRows, isWalDisabled, isMultitenant, storeNulls, isTransactional, - updateCacheFrequency, isNamespaceMapped, resolvedTime); - } - finally { - readWriteLock.writeLock().unlock(); - } - } - - @Override public PSchema getSchema(PTableKey key) throws SchemaNotFoundException { readWriteLock.readLock().lock(); try { http://git-wip-us.apache.org/repos/asf/phoenix/blob/0ea97ce6/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java index 69cc7ff..9962859 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java @@ -35,23 +35,63 @@ import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; import org.apache.phoenix.util.SchemaUtil; public enum TableProperty { - + @Deprecated // use the IMMUTABLE keyword while creating the table - IMMUTABLE_ROWS(PhoenixDatabaseMetaData.IMMUTABLE_ROWS, true, true, false), + IMMUTABLE_ROWS(PhoenixDatabaseMetaData.IMMUTABLE_ROWS, true, true, false) { + @Override + public Object getPTableValue(PTable table) { + return table.isImmutableRows(); + } + }, - MULTI_TENANT(PhoenixDatabaseMetaData.MULTI_TENANT, true, false, false), + MULTI_TENANT(PhoenixDatabaseMetaData.MULTI_TENANT, true, false, false) { + @Override + public Object getPTableValue(PTable table) { + return table.isMultiTenant(); + } + }, - DISABLE_WAL(PhoenixDatabaseMetaData.DISABLE_WAL, true, false, false), + DISABLE_WAL(PhoenixDatabaseMetaData.DISABLE_WAL, true, false, false) { + @Override + public Object getPTableValue(PTable table) { + return table.isWALDisabled(); + } + }, - SALT_BUCKETS(PhoenixDatabaseMetaData.SALT_BUCKETS, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, false, SALT_ONLY_ON_CREATE_TABLE, false, false), + SALT_BUCKETS(PhoenixDatabaseMetaData.SALT_BUCKETS, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, false, SALT_ONLY_ON_CREATE_TABLE, false, false) { + @Override + public Object getPTableValue(PTable table) { + return table.getBucketNum(); + } + }, - DEFAULT_COLUMN_FAMILY(DEFAULT_COLUMN_FAMILY_NAME, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, false, DEFAULT_COLUMN_FAMILY_ONLY_ON_CREATE_TABLE, false, false), + DEFAULT_COLUMN_FAMILY(DEFAULT_COLUMN_FAMILY_NAME, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, false, DEFAULT_COLUMN_FAMILY_ONLY_ON_CREATE_TABLE, false, false) { + @Override + public Object getPTableValue(PTable table) { + return table.getDefaultFamilyName(); + } + }, - TTL(HColumnDescriptor.TTL, COLUMN_FAMILY_NOT_ALLOWED_FOR_TTL, true, CANNOT_ALTER_PROPERTY, false, false), + TTL(HColumnDescriptor.TTL, COLUMN_FAMILY_NOT_ALLOWED_FOR_TTL, true, CANNOT_ALTER_PROPERTY, false, false) { + @Override + public Object getPTableValue(PTable table) { + return null; + } + }, - STORE_NULLS(PhoenixDatabaseMetaData.STORE_NULLS, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, false, false), + STORE_NULLS(PhoenixDatabaseMetaData.STORE_NULLS, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, false, false) { + @Override + public Object getPTableValue(PTable table) { + return table.getStoreNulls(); + } + }, - TRANSACTIONAL(PhoenixDatabaseMetaData.TRANSACTIONAL, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, false, false), + TRANSACTIONAL(PhoenixDatabaseMetaData.TRANSACTIONAL, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, false, false) { + @Override + public Object getPTableValue(PTable table) { + return table.isTransactional(); + } + }, UPDATE_CACHE_FREQUENCY(PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY, true, true, true) { @Override @@ -67,26 +107,46 @@ public enum TableProperty { return value == null ? null : ((Number) value).longValue(); } return value; - } + } + + @Override + public Object getPTableValue(PTable table) { + return table.getUpdateCacheFrequency(); + } }, AUTO_PARTITION_SEQ(PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, false, false, false) { @Override public Object getValue(Object value) { return value == null ? null : SchemaUtil.normalizeIdentifier(value.toString()); + } + + @Override + public Object getPTableValue(PTable table) { + return table.getAutoPartitionSeqName(); } }, - APPEND_ONLY_SCHEMA(PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, true, false), + APPEND_ONLY_SCHEMA(PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, true, false) { + @Override + public Object getPTableValue(PTable table) { + return table.isAppendOnlySchema(); + } + }, GUIDE_POSTS_WIDTH(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH, true, false, false) { @Override public Object getValue(Object value) { return value == null ? null : ((Number) value).longValue(); + } + + @Override + public Object getPTableValue(PTable table) { + return null; } }, ; - + private final String propertyName; private final SQLExceptionCode colFamSpecifiedException; private final boolean isMutable; // whether or not a property can be changed through statements like ALTER TABLE. @@ -174,4 +234,10 @@ public enum TableProperty { return isMutable; } + public boolean isMutableOnView() { + return isMutableOnView; + } + + abstract public Object getPTableValue(PTable table); + }