Return-Path: X-Original-To: apmail-phoenix-commits-archive@minotaur.apache.org Delivered-To: apmail-phoenix-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 02FDA1043F for ; Wed, 5 Mar 2014 22:57:38 +0000 (UTC) Received: (qmail 90094 invoked by uid 500); 5 Mar 2014 22:56:22 -0000 Delivered-To: apmail-phoenix-commits-archive@phoenix.apache.org Received: (qmail 89230 invoked by uid 500); 5 Mar 2014 22:55:52 -0000 Mailing-List: contact commits-help@phoenix.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@phoenix.incubator.apache.org Delivered-To: mailing list commits@phoenix.incubator.apache.org Received: (qmail 88523 invoked by uid 99); 5 Mar 2014 22:55:34 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 05 Mar 2014 22:55:34 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO mail.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with SMTP; Wed, 05 Mar 2014 22:55:30 +0000 Received: (qmail 86563 invoked by uid 99); 5 Mar 2014 22:54:48 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 05 Mar 2014 22:54:48 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 5DCE4937DE3; Wed, 5 Mar 2014 22:54:47 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jeffreyz@apache.org To: commits@phoenix.incubator.apache.org Date: Wed, 05 Mar 2014 22:54:56 -0000 Message-Id: <35484b4b56b347b5ba25f83d48310f12@git.apache.org> In-Reply-To: <6712aa6a6ff5432cb9414eaff469087b@git.apache.org> References: <6712aa6a6ff5432cb9414eaff469087b@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [10/50] [abbrv] git commit: PHOENIX-97 Sort columns by COLUMN_NAME for metaData.getPrimaryKey() as dictated by spec (JamesTaylor) X-Virus-Checked: Checked by ClamAV on apache.org PHOENIX-97 Sort columns by COLUMN_NAME for metaData.getPrimaryKey() as dictated by spec (JamesTaylor) Project: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/commit/70a6ea3d Tree: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/tree/70a6ea3d Diff: http://git-wip-us.apache.org/repos/asf/incubator-phoenix/diff/70a6ea3d Branch: refs/heads/4.0 Commit: 70a6ea3d89236a8cc4b0dbccc07d1e08336b0b63 Parents: 6142ff4 Author: James Taylor Authored: Sun Mar 2 16:01:55 2014 -0800 Committer: James Taylor Committed: Sun Mar 2 16:01:55 2014 -0800 ---------------------------------------------------------------------- .../phoenix/jdbc/PhoenixDatabaseMetaData.java | 91 ++------------------ .../apache/phoenix/query/QueryConstants.java | 2 + .../apache/phoenix/schema/MetaDataClient.java | 66 +++++++++----- .../org/apache/phoenix/util/SchemaUtil.java | 16 +++- .../apache/phoenix/end2end/AlterTableTest.java | 23 ++++- .../end2end/QueryDatabaseMetaDataTest.java | 18 ++-- .../end2end/TenantSpecificTablesDDLTest.java | 10 +-- 7 files changed, 104 insertions(+), 122 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/70a6ea3d/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java index 4f2ee29..4d3f083 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java @@ -23,14 +23,11 @@ import java.sql.ResultSet; import java.sql.RowIdLifetime; import java.sql.SQLException; import java.sql.Statement; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.List; import org.apache.commons.lang.StringEscapeUtils; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.Bytes; import org.apache.phoenix.compile.ColumnProjector; import org.apache.phoenix.compile.ExpressionProjector; @@ -38,13 +35,11 @@ import org.apache.phoenix.compile.RowProjector; import org.apache.phoenix.coprocessor.MetaDataProtocol; import org.apache.phoenix.exception.SQLExceptionCode; import org.apache.phoenix.exception.SQLExceptionInfo; -import org.apache.phoenix.expression.BaseTerminalExpression; import org.apache.phoenix.expression.RowKeyColumnExpression; import org.apache.phoenix.expression.function.IndexStateNameFunction; import org.apache.phoenix.expression.function.SQLTableTypeFunction; import org.apache.phoenix.expression.function.SQLViewTypeFunction; import org.apache.phoenix.expression.function.SqlTypeNameFunction; -import org.apache.phoenix.iterate.DelegateResultIterator; import org.apache.phoenix.iterate.MaterializedResultIterator; import org.apache.phoenix.iterate.ResultIterator; import org.apache.phoenix.parse.HintNode.Hint; @@ -653,99 +648,27 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData, org.apache.pho if (table == null || table.length() == 0) { return emptyResultSet; } - final int keySeqPosition = 4; - final int pkNamePosition = 5; StringBuilder buf = new StringBuilder("select /*+" + Hint.NO_INTRA_REGION_PARALLELIZATION + "*/\n" + TENANT_ID + " " + TABLE_CAT + "," + // use catalog for tenant_id TABLE_SCHEM + "," + TABLE_NAME + " ," + COLUMN_NAME + "," + - "null as " + KEY_SEQ + "," + - "PK_NAME" + "," + + KEY_SEQ + "," + + PK_NAME + "," + "CASE WHEN " + SORT_ORDER + " = " + (SortOrder.DESC.getSystemValue()) + " THEN 'D' ELSE 'A' END ASC_OR_DESC," + DATA_TYPE + "," + // include type info, though not in spec SqlTypeNameFunction.NAME + "(" + DATA_TYPE + ") AS " + TYPE_NAME + "," + - COLUMN_SIZE + + COLUMN_SIZE + "," + + VIEW_CONSTANT + " from " + SYSTEM_CATALOG + " " + SYSTEM_CATALOG_ALIAS + " where "); buf.append(TABLE_SCHEM + (schema == null || schema.length() == 0 ? " is null" : " = '" + escapePattern(schema) + "'" )); buf.append(" and " + TABLE_NAME + " = '" + escapePattern(table) + "'" ); + buf.append(" and " + COLUMN_NAME + " is not null"); buf.append(" and " + COLUMN_FAMILY + " is null"); addTenantIdFilter(buf, catalog); - buf.append(" order by " + TENANT_ID + "," + TABLE_SCHEM + "," + TABLE_NAME + " ," + ORDINAL_POSITION); - // Dynamically replaces the KEY_SEQ with an expression that gets incremented after each next call. - Statement stmt = connection.createStatement(new PhoenixStatementFactory() { - - @Override - public PhoenixStatement newStatement(PhoenixConnection connection) { - final byte[] unsetValue = new byte[0]; - final ImmutableBytesWritable pkNamePtr = new ImmutableBytesWritable(unsetValue); - final byte[] rowNumberHolder = new byte[PDataType.INTEGER.getByteSize()]; - return new PhoenixStatement(connection) { - @Override - protected PhoenixResultSet newResultSet(ResultIterator iterator, RowProjector projector) throws SQLException { - List columns = new ArrayList(projector.getColumnProjectors()); - ColumnProjector column = columns.get(keySeqPosition); - - columns.set(keySeqPosition, new ExpressionProjector(column.getName(), column.getTableName(), - new BaseTerminalExpression() { - @Override - public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) { - ptr.set(rowNumberHolder); - return true; - } - - @Override - public PDataType getDataType() { - return PDataType.INTEGER; - } - }, - column.isCaseSensitive()) - ); - column = columns.get(pkNamePosition); - columns.set(pkNamePosition, new ExpressionProjector(column.getName(), column.getTableName(), - new BaseTerminalExpression() { - @Override - public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) { - if (pkNamePtr.get() == unsetValue) { - boolean b = tuple.getValue(TABLE_FAMILY_BYTES, PK_NAME_BYTES, pkNamePtr); - if (!b) { - pkNamePtr.set(ByteUtil.EMPTY_BYTE_ARRAY); - } - } - ptr.set(pkNamePtr.get(),pkNamePtr.getOffset(),pkNamePtr.getLength()); - return true; - } - - @Override - public PDataType getDataType() { - return PDataType.VARCHAR; - } - }, - column.isCaseSensitive()) - ); - final RowProjector newProjector = new RowProjector(columns, projector.getEstimatedRowByteSize(), projector.isProjectEmptyKeyValue()); - ResultIterator delegate = new DelegateResultIterator(iterator) { - private int rowCount = 0; - - @Override - public Tuple next() throws SQLException { - // Ignore first row, since it's the table row - PDataType.INTEGER.toBytes(rowCount++, rowNumberHolder, 0); - return super.next(); - } - }; - return new PhoenixResultSet(delegate, newProjector, this); - } - - }; - } - - }); - ResultSet rs = stmt.executeQuery(buf.toString()); - if (rs.next()) { // Skip table row - we just use that to get the PK_NAME - rs.getString(pkNamePosition+1); // Hack to cause the statement to cache this value - } + buf.append(" order by " + TENANT_ID + "," + TABLE_SCHEM + "," + TABLE_NAME + " ," + COLUMN_NAME); + ResultSet rs = connection.createStatement().executeQuery(buf.toString()); return rs; } http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/70a6ea3d/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java index 1fad973..766ff67 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java @@ -38,6 +38,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INCREMENT_BY; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_AUTOINCREMENT; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NULLABLE; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.KEY_SEQ; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MULTI_TENANT; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NULLABLE; @@ -200,6 +201,7 @@ public interface QueryConstants { SORT_ORDER + " INTEGER," + ARRAY_SIZE + " INTEGER,\n" + VIEW_CONSTANT + " VARBINARY,\n" + + KEY_SEQ + " SMALLINT,\n" + // Link metadata (only set on rows linking table to index or view) LINK_TYPE + " UNSIGNED_TINYINT,\n" + "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + "," http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/70a6ea3d/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 ab420be..7e74289 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 @@ -23,6 +23,7 @@ import static com.google.common.collect.Sets.newLinkedHashSetWithExpectedSize; import static org.apache.phoenix.exception.SQLExceptionCode.INSUFFICIENT_MULTI_TENANT_COLUMNS; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_FAMILY; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TABLE_NAME; @@ -32,6 +33,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAM import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DISABLE_WAL; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.KEY_SEQ; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MULTI_TENANT; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NULLABLE; @@ -39,14 +41,13 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ORDINAL_POSITION; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PK_NAME; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SORT_ORDER; -import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_FAMILY; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SEQ_NUM; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID; -import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA; -import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT; @@ -218,8 +219,10 @@ public class MetaDataClient { SORT_ORDER + "," + DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs ARRAY_SIZE + "," + - VIEW_CONSTANT + - ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; + VIEW_CONSTANT + "," + + PK_NAME + "," + // write this both in the column and table rows for access by metadata APIs + KEY_SEQ + + ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)"; private static final String UPDATE_COLUMN_POSITION = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " + TENANT_ID + "," + @@ -326,7 +329,7 @@ public class MetaDataClient { } - private void addColumnMutation(String schemaName, String tableName, PColumn column, PreparedStatement colUpsert, String parentTableName) throws SQLException { + private void addColumnMutation(String schemaName, String tableName, PColumn column, PreparedStatement colUpsert, String parentTableName, String pkName, Short keySeq) throws SQLException { colUpsert.setString(1, connection.getTenantId() == null ? null : connection.getTenantId().getString()); colUpsert.setString(2, schemaName); colUpsert.setString(3, tableName); @@ -353,6 +356,12 @@ public class MetaDataClient { colUpsert.setInt(13, column.getArraySize()); } colUpsert.setBytes(14, column.getViewConstant()); + colUpsert.setString(15, pkName); + if (keySeq == null) { + colUpsert.setNull(16, Types.SMALLINT); + } else { + colUpsert.setShort(16, keySeq); + } colUpsert.execute(); } @@ -1052,6 +1061,7 @@ public class MetaDataClient { } } + short nextKeySeq = 0; for (int i = 0; i < columns.size(); i++) { PColumn column = columns.get(i); int columnPosition = column.getPosition(); @@ -1065,7 +1075,8 @@ public class MetaDataClient { } }); } - addColumnMutation(schemaName, tableName, column, colUpsert, parentTableName); + Short keySeq = SchemaUtil.isPKColumn(column) ? ++nextKeySeq : null; + addColumnMutation(schemaName, tableName, column, colUpsert, parentTableName, pkName, keySeq); } tableMetaData.addAll(connection.getMutationState().toMutations().next().getSecond()); @@ -1476,6 +1487,7 @@ public class MetaDataClient { throw new SQLExceptionInfo.Builder(SQLExceptionCode.SET_UNSUPPORTED_PROP_ON_ALTER_TABLE) .setTableName(table.getName().getString()).build().buildException(); } + short nextKeySeq = SchemaUtil.getMaxKeySeq(table); for( ColumnDef colDef : columnDefs) { if (colDef != null && !colDef.isNull()) { if(colDef.isPK()) { @@ -1489,26 +1501,38 @@ public class MetaDataClient { throwIfAlteringViewPK(colDef, table); PColumn column = newColumn(position++, colDef, PrimaryKeyConstraint.EMPTY, table.getDefaultFamilyName() == null ? null : table.getDefaultFamilyName().getString()); columns.add(column); - addColumnMutation(schemaName, tableName, column, colUpsert, null); + String pkName = null; + Short keySeq = null; // TODO: support setting properties on other families? - if (column.getFamilyName() != null) { - families.add(new Pair>(column.getFamilyName().getBytes(),statement.getProps())); - } else { // If adding to primary key, then add the same column to all indexes on the table + if (column.getFamilyName() == null) { isAddingPKColumn = true; - for (PTable index : table.getIndexes()) { - int indexColPosition = index.getColumns().size(); - PDataType indexColDataType = IndexUtil.getIndexColumnDataType(column); - ColumnName indexColName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(column)); - ColumnDef indexColDef = FACTORY.columnDef(indexColName, indexColDataType.getSqlTypeName(), column.isNullable(), column.getMaxLength(), column.getScale(), true, column.getSortOrder()); - PColumn indexColumn = newColumn(indexColPosition, indexColDef, PrimaryKeyConstraint.EMPTY, index.getDefaultFamilyName() == null ? null : index.getDefaultFamilyName().getString()); - addColumnMutation(schemaName, index.getTableName().getString(), indexColumn, colUpsert, index.getParentTableName().getString()); + pkName = table.getPKName() == null ? null : table.getPKName().getString(); + keySeq = ++nextKeySeq; + } else { + families.add(new Pair>(column.getFamilyName().getBytes(),statement.getProps())); + } + addColumnMutation(schemaName, tableName, column, colUpsert, null, pkName, keySeq); + } + // Add any new PK columns to end of index PK + if (isAddingPKColumn) { + for (PTable index : table.getIndexes()) { + short nextIndexKeySeq = SchemaUtil.getMaxKeySeq(index); + int indexPosition = index.getColumns().size(); + for (ColumnDef colDef : columnDefs) { + if (colDef.isPK()) { + PDataType indexColDataType = IndexUtil.getIndexColumnDataType(colDef.isNull(), colDef.getDataType()); + ColumnName indexColName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(null, colDef.getColumnDefName().getColumnName())); + ColumnDef indexColDef = FACTORY.columnDef(indexColName, indexColDataType.getSqlTypeName(), colDef.isNull(), colDef.getMaxLength(), colDef.getScale(), true, colDef.getSortOrder()); + PColumn indexColumn = newColumn(indexPosition++, indexColDef, PrimaryKeyConstraint.EMPTY, null); + addColumnMutation(schemaName, index.getTableName().getString(), indexColumn, colUpsert, index.getParentTableName().getString(), index.getPKName() == null ? null : index.getPKName().getString(), ++nextIndexKeySeq); + } } } - - tableMetaData.addAll(connection.getMutationState().toMutations().next().getSecond()); - connection.rollback(); } + + tableMetaData.addAll(connection.getMutationState().toMutations().next().getSecond()); + connection.rollback(); } else { // Only support setting IMMUTABLE_ROWS=true and DISABLE_WAL=true on ALTER TABLE SET command // TODO: support setting HBase table properties too http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/70a6ea3d/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java index b30a2b9..aaf3840 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java @@ -38,7 +38,6 @@ import org.apache.phoenix.query.KeyRange; import org.apache.phoenix.query.QueryConstants; import org.apache.phoenix.schema.AmbiguousColumnException; import org.apache.phoenix.schema.ColumnFamilyNotFoundException; -import org.apache.phoenix.schema.SortOrder; import org.apache.phoenix.schema.ColumnNotFoundException; import org.apache.phoenix.schema.PColumn; import org.apache.phoenix.schema.PColumnFamily; @@ -50,6 +49,7 @@ import org.apache.phoenix.schema.PTable; import org.apache.phoenix.schema.RowKeySchema; import org.apache.phoenix.schema.RowKeySchema.RowKeySchemaBuilder; import org.apache.phoenix.schema.SaltingUtil; +import org.apache.phoenix.schema.SortOrder; import org.apache.phoenix.schema.ValueSchema.Field; @@ -540,4 +540,18 @@ public class SchemaUtil { } return maxKeyLength; } + + public static short getMaxKeySeq(PTable table) { + int offset = 0; + if (table.getBucketNum() != null) { + offset++; + } + if (table.isMultiTenant()) { + offset++; + } + if (table.getViewIndexId() != null) { + offset++; + } + return (short)(table.getPKColumns().size() - offset); + } } http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/70a6ea3d/phoenix-core/src/test/java/org/apache/phoenix/end2end/AlterTableTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/end2end/AlterTableTest.java b/phoenix-core/src/test/java/org/apache/phoenix/end2end/AlterTableTest.java index da16ef3..1a39c31 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/end2end/AlterTableTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/end2end/AlterTableTest.java @@ -44,6 +44,7 @@ import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.query.QueryConstants; import org.apache.phoenix.schema.PTableKey; import org.apache.phoenix.schema.TableNotFoundException; +import org.apache.phoenix.util.IndexUtil; import org.apache.phoenix.util.SchemaUtil; import org.junit.Test; @@ -345,8 +346,26 @@ public class AlterTableTest extends BaseHBaseManagedTimeTest { conn.commit(); assertIndexExists(conn,true); - conn.createStatement().execute("ALTER TABLE " + DATA_TABLE_FULL_NAME + " ADD k2 DECIMAL PRIMARY KEY"); - // TODO: verify metadata of index + conn.createStatement().execute("ALTER TABLE " + DATA_TABLE_FULL_NAME + " ADD v3 VARCHAR, k2 DECIMAL PRIMARY KEY"); + rs = conn.getMetaData().getPrimaryKeys("", SCHEMA_NAME, DATA_TABLE_NAME); + assertTrue(rs.next()); + assertEquals("K",rs.getString("COLUMN_NAME")); + assertEquals(1, rs.getShort("KEY_SEQ")); + assertTrue(rs.next()); + assertEquals("K2",rs.getString("COLUMN_NAME")); + assertEquals(2, rs.getShort("KEY_SEQ")); + + rs = conn.getMetaData().getPrimaryKeys("", SCHEMA_NAME, INDEX_TABLE_NAME); + assertTrue(rs.next()); + assertEquals(QueryConstants.DEFAULT_COLUMN_FAMILY + IndexUtil.INDEX_COLUMN_NAME_SEP + "V1",rs.getString("COLUMN_NAME")); + assertEquals(1, rs.getShort("KEY_SEQ")); + assertTrue(rs.next()); + assertEquals(IndexUtil.INDEX_COLUMN_NAME_SEP + "K",rs.getString("COLUMN_NAME")); + assertEquals(2, rs.getShort("KEY_SEQ")); + assertTrue(rs.next()); + assertEquals(IndexUtil.INDEX_COLUMN_NAME_SEP + "K2",rs.getString("COLUMN_NAME")); + assertEquals(3, rs.getShort("KEY_SEQ")); + assertIndexExists(conn,true); query = "SELECT * FROM " + DATA_TABLE_FULL_NAME; http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/70a6ea3d/phoenix-core/src/test/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataTest.java b/phoenix-core/src/test/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataTest.java index 9c40d9a..dde53bf 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataTest.java @@ -18,8 +18,8 @@ package org.apache.phoenix.end2end; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA; -import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SEQUENCE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SEQUENCE; import static org.apache.phoenix.util.TestUtil.ATABLE_NAME; import static org.apache.phoenix.util.TestUtil.ATABLE_SCHEMA_NAME; import static org.apache.phoenix.util.TestUtil.BTABLE_NAME; @@ -367,10 +367,10 @@ public class QueryDatabaseMetaDataTest extends BaseClientManagedTimeTest { assertEquals(CUSTOM_ENTITY_DATA_SCHEMA_NAME, rs.getString("TABLE_SCHEM")); assertEquals(CUSTOM_ENTITY_DATA_NAME, rs.getString("TABLE_NAME")); assertEquals(null, rs.getString("TABLE_CAT")); - assertEquals(SchemaUtil.normalizeIdentifier("organization_id"), rs.getString("COLUMN_NAME")); - assertEquals(1, rs.getInt("KEY_SEQ")); - assertEquals(SchemaUtil.normalizeIdentifier("pk"), rs.getString("PK_NAME")); // TODO: this is on the table row - + assertEquals(SchemaUtil.normalizeIdentifier("custom_entity_data_id"), rs.getString("COLUMN_NAME")); + assertEquals(3, rs.getInt("KEY_SEQ")); + assertEquals(SchemaUtil.normalizeIdentifier("pk"), rs.getString("PK_NAME")); + assertTrue(rs.next()); assertEquals(CUSTOM_ENTITY_DATA_SCHEMA_NAME, rs.getString("TABLE_SCHEM")); assertEquals(CUSTOM_ENTITY_DATA_NAME, rs.getString("TABLE_NAME")); @@ -383,10 +383,10 @@ public class QueryDatabaseMetaDataTest extends BaseClientManagedTimeTest { assertEquals(CUSTOM_ENTITY_DATA_SCHEMA_NAME, rs.getString("TABLE_SCHEM")); assertEquals(CUSTOM_ENTITY_DATA_NAME, rs.getString("TABLE_NAME")); assertEquals(null, rs.getString("TABLE_CAT")); - assertEquals(SchemaUtil.normalizeIdentifier("custom_entity_data_id"), rs.getString("COLUMN_NAME")); - assertEquals(3, rs.getInt("KEY_SEQ")); - assertEquals(SchemaUtil.normalizeIdentifier("pk"), rs.getString("PK_NAME")); - + assertEquals(SchemaUtil.normalizeIdentifier("organization_id"), rs.getString("COLUMN_NAME")); + assertEquals(1, rs.getInt("KEY_SEQ")); + assertEquals(SchemaUtil.normalizeIdentifier("pk"), rs.getString("PK_NAME")); // TODO: this is on the table row + assertFalse(rs.next()); rs = dbmd.getColumns("", CUSTOM_ENTITY_DATA_SCHEMA_NAME, CUSTOM_ENTITY_DATA_NAME, null); http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/70a6ea3d/phoenix-core/src/test/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLTest.java b/phoenix-core/src/test/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLTest.java index b0be8ca..1d98d8f 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLTest.java @@ -23,8 +23,8 @@ import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_MODIFY_VIEW_P import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_MUTATE_TABLE; import static org.apache.phoenix.exception.SQLExceptionCode.TABLE_UNDEFINED; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA; -import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SEQUENCE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SEQUENCE; import static org.apache.phoenix.schema.PTableType.SYSTEM; import static org.apache.phoenix.schema.PTableType.TABLE; import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES; @@ -373,7 +373,7 @@ public class TenantSpecificTablesDDLTest extends BaseTenantSpecificTablesTest { // null means across all tenant_ids rs = meta.getSuperTables(null, null, StringUtil.escapeLike(TENANT_TABLE_NAME)); assertTrue(rs.next()); - assertEquals(TENANT_ID, rs.getString(PhoenixDatabaseMetaData.COLUMN_FAMILY)); + assertEquals(TENANT_ID, rs.getString(PhoenixDatabaseMetaData.TABLE_CAT)); assertEquals(TENANT_TABLE_NAME, rs.getString(PhoenixDatabaseMetaData.TABLE_NAME)); assertEquals(PARENT_TABLE_NAME, rs.getString(PhoenixDatabaseMetaData.SUPERTABLE_NAME)); assertFalse(rs.next()); @@ -383,18 +383,18 @@ public class TenantSpecificTablesDDLTest extends BaseTenantSpecificTablesTest { conn = DriverManager.getConnection(getUrl()); rs = conn.getMetaData().getSuperTables(TENANT_ID, null, null); assertTrue(rs.next()); - assertEquals(TENANT_ID, rs.getString(PhoenixDatabaseMetaData.COLUMN_FAMILY)); + assertEquals(TENANT_ID, rs.getString(PhoenixDatabaseMetaData.TABLE_CAT)); assertEquals(TENANT_TABLE_NAME, rs.getString(PhoenixDatabaseMetaData.TABLE_NAME)); assertEquals(PARENT_TABLE_NAME, rs.getString(PhoenixDatabaseMetaData.SUPERTABLE_NAME)); assertTrue(rs.next()); - assertEquals(TENANT_ID, rs.getString(PhoenixDatabaseMetaData.COLUMN_FAMILY)); + assertEquals(TENANT_ID, rs.getString(PhoenixDatabaseMetaData.TABLE_CAT)); assertEquals(TENANT_TABLE_NAME_NO_TENANT_TYPE_ID, rs.getString(PhoenixDatabaseMetaData.TABLE_NAME)); assertEquals(PARENT_TABLE_NAME_NO_TENANT_TYPE_ID, rs.getString(PhoenixDatabaseMetaData.SUPERTABLE_NAME)); assertFalse(rs.next()); rs = conn.getMetaData().getCatalogs(); assertTrue(rs.next()); - assertEquals(TENANT_ID, rs.getString(PhoenixDatabaseMetaData.COLUMN_FAMILY)); + assertEquals(TENANT_ID, rs.getString(PhoenixDatabaseMetaData.TABLE_CAT)); assertFalse(rs.next()); } finally {