PHOENIX-4996: Refactor PTableImpl to use Builder Pattern

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

Branch: refs/heads/master
Commit: 11cc13b043c9d73c49134f27aef5e4c05dc6f30a
Parents: 6053ee6
Author: Chinmay Kulkarni <chinmayskulka...@gmail.com>
Authored: Wed Oct 24 17:56:22 2018 -0700
Committer: Thomas D'Silva <tdsi...@apache.org>
Committed: Tue Nov 6 15:18:45 2018 -0800

----------------------------------------------------------------------
 .../apache/phoenix/compile/DeleteCompiler.java  |    6 +-
 .../apache/phoenix/compile/FromCompiler.java    |   66 +-
 .../apache/phoenix/compile/JoinCompiler.java    |   52 +-
 .../compile/TupleProjectionCompiler.java        |   60 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   41 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   12 +-
 .../coprocessor/MetaDataEndpointImpl.java       |   96 +-
 .../UngroupedAggregateRegionObserver.java       |    6 +-
 .../coprocessor/WhereConstantParser.java        |    3 +-
 .../query/ConnectionlessQueryServicesImpl.java  |    9 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  215 ++-
 .../apache/phoenix/schema/PMetaDataImpl.java    |   28 +-
 .../org/apache/phoenix/schema/PTableImpl.java   | 1259 +++++++++++-------
 .../org/apache/phoenix/schema/TableRef.java     |   17 +-
 .../phoenix/execute/CorrelatePlanTest.java      |   32 +-
 .../execute/LiteralResultIteratorPlanTest.java  |   33 +-
 16 files changed, 1302 insertions(+), 633 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/11cc13b0/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java 
b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
index 14ec45d..51366c0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@ -27,7 +27,6 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.Cell;
@@ -90,7 +89,6 @@ import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.transaction.PhoenixTransactionProvider.Feature;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.IndexUtil;
-import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.ScanUtil;
 
 import com.google.common.base.Preconditions;
@@ -616,7 +614,9 @@ public class DeleteCompiler {
                     }
                 });
             }
-            PTable projectedTable = PTableImpl.makePTable(table, 
PTableType.PROJECTED, adjustedProjectedColumns);
+            PTable projectedTable = PTableImpl.builderWithColumns(table, 
adjustedProjectedColumns)
+                    .setType(PTableType.PROJECTED)
+                    .build();
             final TableRef projectedTableRef = new TableRef(projectedTable, 
targetTableRef.getLowerBoundTimeStamp(), targetTableRef.getTimeStamp());
 
             QueryPlan bestPlanToBe = dataPlan;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11cc13b0/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java 
b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
index 80648a3..d0a49cc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -32,8 +32,6 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
-import org.apache.phoenix.exception.SQLExceptionCode;
-import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.parse.AliasedNode;
@@ -82,6 +80,7 @@ import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ProjectedColumn;
+import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.SchemaNotFoundException;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableNotFoundException;
@@ -284,7 +283,8 @@ public class FromCompiler {
                 column.getTimestamp());
             projectedColumns.add(projectedColumn);
         }
-        PTable t = PTableImpl.makePTable(table, projectedColumns);
+        PTable t = PTableImpl.builderWithColumns(table, projectedColumns)
+                .build();
         return new SingleTableColumnResolver(connection, new 
TableRef(tableRef.getTableAlias(), t, tableRef.getLowerBoundTimeStamp(), 
tableRef.hasDynamicCols()));
     }
     
@@ -369,10 +369,30 @@ public class FromCompiler {
             if (connection.getSchema() != null) {
                 schema = schema != null ? schema : connection.getSchema();
             }
+
             // Storage scheme and encoding scheme don't matter here since the 
PTable is being used only for the purposes of create table.
             // The actual values of these two will be determined by the 
metadata client.
-            PTable theTable = new PTableImpl(connection.getTenantId(), schema, 
table.getName().getTableName(),
-                    scn == null ? HConstants.LATEST_TIMESTAMP : scn, families, 
isNamespaceMapped);
+            PName tenantId = connection.getTenantId();
+            PTableImpl.checkTenantId(tenantId);
+            String tableName = table.getName().getTableName();
+            PName name = PNameFactory.newName(SchemaUtil.getTableName(schema, 
tableName));
+            PTable theTable = new PTableImpl.Builder()
+                    .setTenantId(tenantId)
+                    .setName(name)
+                    .setKey(new PTableKey(tenantId, name.getString()))
+                    .setSchemaName(PNameFactory.newName(schema))
+                    .setTableName(PNameFactory.newName(tableName))
+                    .setType(PTableType.VIEW)
+                    .setViewType(PTable.ViewType.MAPPED)
+                    .setTimeStamp(scn == null ? HConstants.LATEST_TIMESTAMP : 
scn)
+                    .setPkColumns(Collections.emptyList())
+                    .setAllColumns(Collections.emptyList())
+                    .setRowKeySchema(RowKeySchema.EMPTY_SCHEMA)
+                    .setIndexes(Collections.emptyList())
+                    .setFamilyAttributes(families)
+                    .setPhysicalNames(Collections.emptyList())
+                    .setNamespaceMapped(isNamespaceMapped)
+                    .build();
             theTable = this.addDynamicColumns(table.getDynamicColumns(), 
theTable);
             alias = null;
             tableRefs = ImmutableList.of(new TableRef(alias, theTable, 
timeStamp, !table.getDynamicColumns().isEmpty()));
@@ -706,7 +726,7 @@ public class FromCompiler {
             if (!dynColumns.isEmpty()) {
                 List<PColumn> allcolumns = new ArrayList<PColumn>();
                 List<PColumn> existingColumns = theTable.getColumns();
-                // Need to skip the salting column, as it's added in the 
makePTable call below
+                // Need to skip the salting column, as it's handled in the 
PTable builder call below
                 allcolumns.addAll(theTable.getBucketNum() == null ? 
existingColumns : existingColumns.subList(1, existingColumns.size()));
                 // Position still based on with the salting columns
                 int position = existingColumns.size();
@@ -724,7 +744,8 @@ public class FromCompiler {
                         HConstants.LATEST_TIMESTAMP));
                     position++;
                 }
-                theTable = PTableImpl.makePTable(theTable, allcolumns);
+                theTable = PTableImpl.builderWithColumns(theTable, allcolumns)
+                        .build();
             }
             return theTable;
         }
@@ -830,11 +851,32 @@ public class FromCompiler {
                     HConstants.LATEST_TIMESTAMP);
                 columns.add(column);
             }
-            PTable t = PTableImpl.makePTable(null, PName.EMPTY_NAME, 
PName.EMPTY_NAME, PTableType.SUBQUERY, null,
-                    MetaDataProtocol.MIN_TABLE_TIMESTAMP, 
PTable.INITIAL_SEQ_NUM, null, null, columns, null, null,
-                    Collections.<PTable> emptyList(), false, 
Collections.<PName> emptyList(), null, null, false, false,
-                    false, null, null, null, null, false, null, 0, 0L, 
SchemaUtil
-                            .isNamespaceMappingEnabled(PTableType.SUBQUERY, 
connection.getQueryServices().getProps()), null, false, 
ImmutableStorageScheme.ONE_CELL_PER_COLUMN, 
QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, 
PTable.EncodedCQCounter.NULL_COUNTER, true);
+            PTable t = new PTableImpl.Builder()
+                    .setType(PTableType.SUBQUERY)
+                    .setTimeStamp(MetaDataProtocol.MIN_TABLE_TIMESTAMP)
+                    .setIndexDisableTimestamp(0L)
+                    .setSequenceNumber(PTable.INITIAL_SEQ_NUM)
+                    .setImmutableRows(false)
+                    .setDisableWAL(false)
+                    .setMultiTenant(false)
+                    .setStoreNulls(false)
+                    .setUpdateCacheFrequency(0)
+                    
.setNamespaceMapped(SchemaUtil.isNamespaceMappingEnabled(PTableType.SUBQUERY,
+                            connection.getQueryServices().getProps()))
+                    .setAppendOnlySchema(false)
+                    
.setImmutableStorageScheme(ImmutableStorageScheme.ONE_CELL_PER_COLUMN)
+                    
.setQualifierEncodingScheme(QualifierEncodingScheme.NON_ENCODED_QUALIFIERS)
+                    
.setBaseColumnCount(QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT)
+                    .setEncodedCQCounter(PTable.EncodedCQCounter.NULL_COUNTER)
+                    .setUseStatsForParallelization(true)
+                    .setExcludedColumns(ImmutableList.of())
+                    .setSchemaName(PName.EMPTY_NAME)
+                    .setTableName(PName.EMPTY_NAME)
+                    .setRowKeyOrderOptimizable(false)
+                    .setIndexes(Collections.emptyList())
+                    .setPhysicalNames(ImmutableList.of())
+                    .setColumns(columns)
+                    .build();
 
             String alias = subselectNode.getAlias();
             TableRef tableRef = new TableRef(alias, t, 
MetaDataProtocol.MIN_TABLE_TIMESTAMP, false);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11cc13b0/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java 
b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
index 880fa72..9560f08 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.compile;
 
+import static 
org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
 import static 
org.apache.phoenix.schema.PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN;
 import static 
org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
 
@@ -32,6 +33,7 @@ import java.util.ListIterator;
 import java.util.Map;
 import java.util.Set;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -71,7 +73,6 @@ import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.LocalIndexDataColumnRef;
 import org.apache.phoenix.schema.MetaDataEntityNotFoundException;
 import org.apache.phoenix.schema.PColumn;
-import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
@@ -1270,17 +1271,44 @@ public class JoinCompiler {
         if (left.getBucketNum() != null) {
             merged.remove(0);
         }
-        return PTableImpl.makePTable(left.getTenantId(), left.getSchemaName(),
-                
PNameFactory.newName(SchemaUtil.getTableName(left.getName().getString(), 
right.getName().getString())),
-                left.getType(), left.getIndexState(), left.getTimeStamp(), 
left.getSequenceNumber(), left.getPKName(),
-                left.getBucketNum(), merged, left.getParentSchemaName(), 
left.getParentTableName(), left.getIndexes(),
-                left.isImmutableRows(), Collections.<PName> emptyList(), null, 
null, PTable.DEFAULT_DISABLE_WAL,
-                left.isMultiTenant(), left.getStoreNulls(), 
left.getViewType(), left.getViewIndexType(), left.getViewIndexId(),
-                left.getIndexType(), left.rowKeyOrderOptimizable(), 
left.getTransactionProvider(),
-                left.getUpdateCacheFrequency(), 
left.getIndexDisableTimestamp(), left.isNamespaceMapped(), 
-                left.getAutoPartitionSeqName(), left.isAppendOnlySchema(), 
ONE_CELL_PER_COLUMN, NON_ENCODED_QUALIFIERS, 
PTable.EncodedCQCounter.NULL_COUNTER, left.useStatsForParallelization());
+        return new PTableImpl.Builder()
+                .setType(left.getType())
+                .setState(left.getIndexState())
+                .setTimeStamp(left.getTimeStamp())
+                .setIndexDisableTimestamp(left.getIndexDisableTimestamp())
+                .setSequenceNumber(left.getSequenceNumber())
+                .setImmutableRows(left.isImmutableRows())
+                .setDisableWAL(PTable.DEFAULT_DISABLE_WAL)
+                .setMultiTenant(left.isMultiTenant())
+                .setStoreNulls(left.getStoreNulls())
+                .setViewType(left.getViewType())
+                .setViewIndexType(left.getViewIndexType())
+                .setViewIndexId(left.getViewIndexId())
+                .setIndexType(left.getIndexType())
+                .setTransactionProvider(left.getTransactionProvider())
+                .setUpdateCacheFrequency(left.getUpdateCacheFrequency())
+                .setNamespaceMapped(left.isNamespaceMapped())
+                .setAutoPartitionSeqName(left.getAutoPartitionSeqName())
+                .setAppendOnlySchema(left.isAppendOnlySchema())
+                .setImmutableStorageScheme(ONE_CELL_PER_COLUMN)
+                .setQualifierEncodingScheme(NON_ENCODED_QUALIFIERS)
+                .setBaseColumnCount(BASE_TABLE_BASE_COLUMN_COUNT)
+                .setEncodedCQCounter(PTable.EncodedCQCounter.NULL_COUNTER)
+                
.setUseStatsForParallelization(left.useStatsForParallelization())
+                .setExcludedColumns(ImmutableList.of())
+                .setTenantId(left.getTenantId())
+                .setSchemaName(left.getSchemaName())
+                
.setTableName(PNameFactory.newName(SchemaUtil.getTableName(left.getName().getString(),
+                        right.getName().getString())))
+                .setPkName(left.getPKName())
+                .setRowKeyOrderOptimizable(left.rowKeyOrderOptimizable())
+                .setBucketNum(left.getBucketNum())
+                .setIndexes(left.getIndexes() == null ? 
Collections.emptyList() : left.getIndexes())
+                .setParentSchemaName(left.getParentSchemaName())
+                .setParentTableName(left.getParentTableName())
+                .setPhysicalNames(ImmutableList.of())
+                .setColumns(merged)
+                .build();
     }
 
 }
-
-

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11cc13b0/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
index dc85bd3..159e617 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.compile;
 import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_FAMILY;
+import static 
org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
+import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 
 import java.sql.SQLException;
 import java.util.ArrayList;
@@ -26,6 +28,7 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.phoenix.parse.AliasedNode;
 import org.apache.phoenix.parse.ColumnParseNode;
 import org.apache.phoenix.parse.FamilyWildcardParseNode;
@@ -160,15 +163,13 @@ public class TupleProjectionCompiler {
                     sourceColumnRef.getColumn().isNullable(), sourceColumnRef, 
sourceColumnRef.getColumn().getColumnQualifierBytes());
             projectedColumns.add(column);
         }
-        
-        return PTableImpl.makePTable(table.getTenantId(), 
table.getSchemaName(), table.getTableName(),
-                PTableType.PROJECTED, table.getIndexState(), 
table.getTimeStamp(), table.getSequenceNumber(),
-                table.getPKName(), table.getBucketNum(), projectedColumns, 
table.getParentSchemaName(),
-                table.getParentTableName(), table.getIndexes(), 
table.isImmutableRows(), Collections.<PName> emptyList(),
-                table.getDefaultFamilyName(), table.getViewStatement(), 
table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), 
table.getViewType(),
-                table.getViewIndexType(), table.getViewIndexId(),
-                table.getIndexType(), table.rowKeyOrderOptimizable(), 
table.getTransactionProvider(), table.getUpdateCacheFrequency(), 
-                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), 
table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), 
table.getImmutableStorageScheme(), table.getEncodingScheme(), 
table.getEncodedCQCounter(), table.useStatsForParallelization());
+        return PTableImpl.builderWithColumns(table, projectedColumns)
+                .setType(PTableType.PROJECTED)
+                .setBaseColumnCount(BASE_TABLE_BASE_COLUMN_COUNT)
+                .setExcludedColumns(ImmutableList.of())
+                .setPhysicalNames(ImmutableList.of())
+                .setColumns(projectedColumns)
+                .build();
     }
     
     public static PTable createProjectedTable(TableRef tableRef, 
List<ColumnRef> sourceColumnRefs, boolean retainPKColumns) throws SQLException {
@@ -192,14 +193,39 @@ public class TupleProjectionCompiler {
         if (EncodedColumnsUtil.usesEncodedColumnNames(table)) {
             cqCounter = EncodedCQCounter.copy(table.getEncodedCQCounter());
         }
-        
-        return PTableImpl.makePTable(table.getTenantId(), 
PROJECTED_TABLE_SCHEMA, table.getName(), PTableType.PROJECTED,
-                null, table.getTimeStamp(), table.getSequenceNumber(), 
table.getPKName(),
-                table.getBucketNum(), projectedColumns, null, null,
-                Collections.<PTable> emptyList(), table.isImmutableRows(), 
Collections.<PName> emptyList(), null, null,
-                table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(),
-                table.getViewIndexType(), table.getViewIndexId(), null, 
table.rowKeyOrderOptimizable(), table.getTransactionProvider(),
-                table.getUpdateCacheFrequency(), 
table.getIndexDisableTimestamp(), table.isNamespaceMapped(), 
table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), 
table.getImmutableStorageScheme(), table.getEncodingScheme(), cqCounter, 
table.useStatsForParallelization());
+        return new PTableImpl.Builder()
+                .setType(PTableType.PROJECTED)
+                .setTimeStamp(table.getTimeStamp())
+                .setIndexDisableTimestamp(table.getIndexDisableTimestamp())
+                .setSequenceNumber(table.getSequenceNumber())
+                .setImmutableRows(table.isImmutableRows())
+                .setDisableWAL(table.isWALDisabled())
+                .setMultiTenant(table.isMultiTenant())
+                .setStoreNulls(table.getStoreNulls())
+                .setViewType(table.getViewType())
+                .setViewIndexType(table.getViewIndexType())
+                .setViewIndexId(table.getViewIndexId())
+                .setTransactionProvider(table.getTransactionProvider())
+                .setUpdateCacheFrequency(table.getUpdateCacheFrequency())
+                .setNamespaceMapped(table.isNamespaceMapped())
+                .setAutoPartitionSeqName(table.getAutoPartitionSeqName())
+                .setAppendOnlySchema(table.isAppendOnlySchema())
+                .setImmutableStorageScheme(table.getImmutableStorageScheme())
+                .setQualifierEncodingScheme(table.getEncodingScheme())
+                .setBaseColumnCount(BASE_TABLE_BASE_COLUMN_COUNT)
+                .setEncodedCQCounter(cqCounter)
+                
.setUseStatsForParallelization(table.useStatsForParallelization())
+                .setExcludedColumns(ImmutableList.of())
+                .setTenantId(table.getTenantId())
+                .setSchemaName(PROJECTED_TABLE_SCHEMA)
+                .setTableName(table.getTableName())
+                .setPkName(table.getPKName())
+                .setRowKeyOrderOptimizable(table.rowKeyOrderOptimizable())
+                .setBucketNum(table.getBucketNum())
+                .setIndexes(Collections.emptyList())
+                .setPhysicalNames(ImmutableList.of())
+                .setColumns(projectedColumns)
+                .build();
     }
 
     // For extracting column references from single select statement

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11cc13b0/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java 
b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
index c0c63eb..99a531c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
@@ -17,10 +17,14 @@
  */
 package org.apache.phoenix.compile;
 
+import static 
org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
+
 import java.sql.SQLException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
@@ -96,15 +100,34 @@ public class UnionCompiler {
             projectedColumns.add(projectedColumn);
         }
         Long scn = statement.getConnection().getSCN();
-        PTable tempTable = 
PTableImpl.makePTable(statement.getConnection().getTenantId(),
-            UNION_SCHEMA_NAME, UNION_TABLE_NAME, PTableType.SUBQUERY, null,
-            HConstants.LATEST_TIMESTAMP, scn == null ? 
HConstants.LATEST_TIMESTAMP : scn,
-            null, null, projectedColumns, null, null, null, true, null, null, 
null, true,
-            true, true, null, null, null, null, false, null, 0, 0L,
-            SchemaUtil.isNamespaceMappingEnabled(PTableType.SUBQUERY,
-                statement.getConnection().getQueryServices().getProps()), 
null, false, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, 
QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, 
PTable.EncodedCQCounter.NULL_COUNTER, true);
-        TableRef tableRef = new TableRef(null, tempTable, 0, false);
-        return tableRef;
+        PTable tempTable = new PTableImpl.Builder()
+                .setType(PTableType.SUBQUERY)
+                .setTimeStamp(HConstants.LATEST_TIMESTAMP)
+                .setIndexDisableTimestamp(0L)
+                .setSequenceNumber(scn == null ? HConstants.LATEST_TIMESTAMP : 
scn)
+                .setImmutableRows(true)
+                .setDisableWAL(true)
+                .setMultiTenant(true)
+                .setStoreNulls(true)
+                .setUpdateCacheFrequency(0)
+                
.setNamespaceMapped(SchemaUtil.isNamespaceMappingEnabled(PTableType.SUBQUERY,
+                        
statement.getConnection().getQueryServices().getProps()))
+                .setAppendOnlySchema(false)
+                
.setImmutableStorageScheme(ImmutableStorageScheme.ONE_CELL_PER_COLUMN)
+                
.setQualifierEncodingScheme(QualifierEncodingScheme.NON_ENCODED_QUALIFIERS)
+                .setBaseColumnCount(BASE_TABLE_BASE_COLUMN_COUNT)
+                .setEncodedCQCounter(PTable.EncodedCQCounter.NULL_COUNTER)
+                .setUseStatsForParallelization(true)
+                .setExcludedColumns(ImmutableList.of())
+                .setTenantId(statement.getConnection().getTenantId())
+                .setSchemaName(UNION_SCHEMA_NAME)
+                .setTableName(UNION_TABLE_NAME)
+                .setRowKeyOrderOptimizable(false)
+                .setIndexes(Collections.emptyList())
+                .setPhysicalNames(ImmutableList.of())
+                .setColumns(projectedColumns)
+                .build();
+        return new TableRef(null, tempTable, 0, false);
     }
 
     private static void compareExperssions(int i, Expression expression,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11cc13b0/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java 
b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index 450a99f..4a88f4c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -32,6 +32,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.client.Scan;
@@ -701,9 +702,11 @@ public class UpsertCompiler {
                     }
                     // Build table from projectedColumns
                     // Hack to add default column family to be used on server 
in case no value column is projected.
-                    PTable projectedTable = PTableImpl.makePTable(table, 
projectedColumns,
-                            
PNameFactory.newName(SchemaUtil.getEmptyColumnFamily(table)));  
-                    
+                    PTable projectedTable = 
PTableImpl.builderWithColumns(table, projectedColumns)
+                            .setExcludedColumns(ImmutableList.of())
+                            
.setDefaultFamilyName(PNameFactory.newName(SchemaUtil.getEmptyColumnFamily(table)))
+                            .setColumns(projectedColumns)
+                            .build();
                     
                     SelectStatement select = 
SelectStatement.create(SelectStatement.COUNT_ONE, upsert.getHint());
                     StatementContext statementContext = queryPlan.getContext();
@@ -856,7 +859,8 @@ public class UpsertCompiler {
                     }
                     updateExpressions.add(updateExpression);
                 }
-                PTable onDupKeyTable = PTableImpl.makePTable(table, 
updateColumns);
+                PTable onDupKeyTable = PTableImpl.builderWithColumns(table, 
updateColumns)
+                        .build();
                 onDupKeyBytesToBe = 
PhoenixIndexBuilder.serializeOnDupKeyUpdate(onDupKeyTable, updateExpressions);
             }
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11cc13b0/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 d07f364..462e853 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
@@ -78,6 +78,7 @@ import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID_DATA
 import static 
org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
 import static org.apache.phoenix.schema.PTableType.INDEX;
 import static org.apache.phoenix.schema.PTableType.TABLE;
+import static org.apache.phoenix.schema.PTableImpl.getColumnsToClone;
 import static org.apache.phoenix.util.SchemaUtil.getVarCharLength;
 import static org.apache.phoenix.util.SchemaUtil.getVarChars;
 
@@ -101,6 +102,7 @@ import java.util.NavigableMap;
 import java.util.Properties;
 import java.util.Set;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellComparatorImpl;
@@ -216,6 +218,7 @@ import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ParentTableNotFoundException;
+import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.schema.SequenceAllocation;
 import org.apache.phoenix.schema.SequenceAlreadyExistsException;
@@ -690,7 +693,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements RegionCopr
                     }
                     indexes.add(latestIndex);
                 }
-                table = PTableImpl.makePTable(table, table.getTimeStamp(), 
indexes);
+                table = PTableImpl.builderWithColumns(table, 
getColumnsToClone(table))
+                        .setIndexes(indexes == null ? Collections.emptyList() 
: indexes)
+                        .build();
             }
         }
         
@@ -940,11 +945,31 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements RegionCopr
         int baseTableColumnCount =
                 isDiverged ? QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT
                         : columnsToAdd.size() - myColumns.size() + (isSalted ? 
1 : 0);
+
+        // When creating a PTable for views or view indexes, use the baseTable 
PTable for attributes
+        // inherited from the physical base table.
+        // if a TableProperty is not valid on a view we set it to the base 
table value
+        // if a TableProperty is valid on a view and is not mutable on a view 
we set it to the base table value
+        // if a TableProperty is valid on a view and is mutable on a view we 
use the value set on the view
         // TODO Implement PHOENIX-4763 to set the view properties correctly 
instead of just
         // setting them same as the base table
-        PTableImpl pTable =
-                PTableImpl.makePTable(table, baseTable, columnsToAdd, 
maxTableTimestamp,
-                    baseTableColumnCount, excludedColumns);
+        PTableImpl pTable = PTableImpl.builderWithColumns(table, columnsToAdd)
+                .setImmutableRows(baseTable.isImmutableRows())
+                .setDisableWAL(baseTable.isWALDisabled())
+                .setMultiTenant(baseTable.isMultiTenant())
+                .setStoreNulls(baseTable.getStoreNulls())
+                .setTransactionProvider(baseTable.getTransactionProvider())
+                .setAutoPartitionSeqName(baseTable.getAutoPartitionSeqName())
+                .setAppendOnlySchema(baseTable.isAppendOnlySchema())
+                
.setImmutableStorageScheme(baseTable.getImmutableStorageScheme() == null ?
+                        ImmutableStorageScheme.ONE_CELL_PER_COLUMN : 
baseTable.getImmutableStorageScheme())
+                .setQualifierEncodingScheme(baseTable.getEncodingScheme() == 
null ?
+                        QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : 
baseTable.getEncodingScheme())
+                .setBaseColumnCount(baseTableColumnCount)
+                .setTimeStamp(maxTableTimestamp)
+                .setExcludedColumns(excludedColumns == null ?
+                        ImmutableList.of() : 
ImmutableList.copyOf(excludedColumns))
+                .build();
         return WhereConstantParser.addViewInfoToPColumnsIfNeeded(pTable);
     }
 
@@ -1490,11 +1515,48 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements RegionCopr
         }
         // Avoid querying the stats table because we're holding the rowLock 
here. Issuing an RPC to a remote
         // server while holding this lock is a bad idea and likely to cause 
contention.
-        return PTableImpl.makePTable(tenantId, schemaName, tableName, 
tableType, indexState, timeStamp, tableSeqNum,
-                pkName, saltBucketNum, columns, parentSchemaName, 
parentTableName, indexes, isImmutableRows, physicalTables, defaultFamilyName,
-                viewStatement, disableWAL, multiTenant, storeNulls, viewType, 
viewIndexType, viewIndexId, indexType,
-                rowKeyOrderOptimizable, transactionProvider, 
updateCacheFrequency, baseColumnCount,
-                indexDisableTimestamp, isNamespaceMapped, autoPartitionSeq, 
isAppendOnlySchema, storageScheme, encodingScheme, cqCounter, 
useStatsForParallelization);
+        return new PTableImpl.Builder()
+                .setType(tableType)
+                .setState(indexState)
+                .setTimeStamp(timeStamp)
+                .setIndexDisableTimestamp(indexDisableTimestamp)
+                .setSequenceNumber(tableSeqNum)
+                .setImmutableRows(isImmutableRows)
+                .setViewStatement(viewStatement)
+                .setDisableWAL(disableWAL)
+                .setMultiTenant(multiTenant)
+                .setStoreNulls(storeNulls)
+                .setViewType(viewType)
+                .setViewIndexType(viewIndexType)
+                .setViewIndexId(viewIndexId)
+                .setIndexType(indexType)
+                .setTransactionProvider(transactionProvider)
+                .setUpdateCacheFrequency(updateCacheFrequency)
+                .setNamespaceMapped(isNamespaceMapped)
+                .setAutoPartitionSeqName(autoPartitionSeq)
+                .setAppendOnlySchema(isAppendOnlySchema)
+                .setImmutableStorageScheme(storageScheme == null ?
+                        ImmutableStorageScheme.ONE_CELL_PER_COLUMN : 
storageScheme)
+                .setQualifierEncodingScheme(encodingScheme == null ?
+                        QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : 
encodingScheme)
+                .setBaseColumnCount(baseColumnCount)
+                .setEncodedCQCounter(cqCounter)
+                .setUseStatsForParallelization(useStatsForParallelization)
+                .setExcludedColumns(ImmutableList.of())
+                .setTenantId(tenantId)
+                .setSchemaName(schemaName)
+                .setTableName(tableName)
+                .setPkName(pkName)
+                .setDefaultFamilyName(defaultFamilyName)
+                .setRowKeyOrderOptimizable(rowKeyOrderOptimizable)
+                .setBucketNum(saltBucketNum)
+                .setIndexes(indexes == null ? Collections.emptyList() : 
indexes)
+                .setParentSchemaName(parentSchemaName)
+                .setParentTableName(parentTableName)
+                .setPhysicalNames(physicalTables == null ?
+                        ImmutableList.of() : 
ImmutableList.copyOf(physicalTables))
+                .setColumns(columns)
+                .build();
     }
     private Long getViewIndexId(Cell[] tableKeyValues, PDataType 
viewIndexType) {
         Cell viewIndexIdKv = tableKeyValues[VIEW_INDEX_ID_INDEX];
@@ -1744,7 +1806,21 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements RegionCopr
     }
 
     private static PTable newDeletedTableMarker(long timestamp) {
-        return new PTableImpl(timestamp);
+        try {
+            return new PTableImpl.Builder()
+                    .setType(PTableType.TABLE)
+                    .setTimeStamp(timestamp)
+                    .setPkColumns(Collections.emptyList())
+                    .setAllColumns(Collections.emptyList())
+                    .setFamilyAttributes(Collections.emptyList())
+                    .setRowKeySchema(RowKeySchema.EMPTY_SCHEMA)
+                    .setIndexes(Collections.emptyList())
+                    .setPhysicalNames(Collections.emptyList())
+                    .build();
+        } catch (SQLException e) {
+            // Should never happen
+            return null;
+        }
     }
 
     private static PFunction newDeletedFunctionMarker(long timestamp) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11cc13b0/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index d94ce7f..2fc31c0 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -23,6 +23,7 @@ import static 
org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY;
 import static org.apache.phoenix.query.QueryConstants.UNGROUPED_AGG_ROW_KEY;
 import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_ATTRIB;
 import static 
org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB;
+import static org.apache.phoenix.schema.PTableImpl.getColumnsToClone;
 import static 
org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.COMPACTION_UPDATE_STATS_ROW_COUNT;
 import static 
org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.CONCURRENT_UPDATE_STATS_ROW_COUNT;
 
@@ -424,7 +425,10 @@ public class UngroupedAggregateRegionObserver extends 
BaseScannerRegionObserver
             logger.debug("Upgrading row key for " + 
region.getRegionInfo().getTable().getNameAsString());
             projectedTable = deserializeTable(descRowKeyTableBytes);
             try {
-                writeToTable = PTableImpl.makePTable(projectedTable, true);
+                writeToTable = PTableImpl.builderWithColumns(projectedTable,
+                        getColumnsToClone(projectedTable))
+                        .setRowKeyOrderOptimizable(true)
+                        .build();
             } catch (SQLException e) {
                 ServerUtil.throwIOException("Upgrade failed", e); // Impossible
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11cc13b0/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
index 4528f55..9fc030c 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
@@ -94,7 +94,8 @@ public class WhereConstantParser {
                 result.add(column);
             }
         }
-        return PTableImpl.makePTable(view, result);
+        return PTableImpl.builderWithColumns(view, result)
+                .build();
     }
 
     private static PhoenixConnection getConnectionlessConnection() throws 
SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11cc13b0/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 f78e3b3..ce107b0 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
@@ -18,6 +18,7 @@
 package org.apache.phoenix.query;
 
 import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE_BYTES;
+import static org.apache.phoenix.schema.PTableImpl.getColumnsToClone;
 
 import java.sql.SQLException;
 import java.util.ArrayList;
@@ -319,7 +320,8 @@ public class ConnectionlessQueryServicesImpl extends 
DelegateQueryServices imple
     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));
+        return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0,
+                PTableImpl.builderWithColumns(table, columns).build());
     }
 
     @Override
@@ -456,7 +458,10 @@ public class ConnectionlessQueryServicesImpl extends 
DelegateQueryServices imple
         String indexTableName = SchemaUtil.getTableName(schemaName, indexName);
         PName tenantId = tenantIdBytes.length == 0 ? null : 
PNameFactory.newName(tenantIdBytes);
         PTable index = metaData.getTableRef(new PTableKey(tenantId, 
indexTableName)).getTable();
-        index = PTableImpl.makePTable(index,newState == PIndexState.USABLE ? 
PIndexState.ACTIVE : newState == PIndexState.UNUSABLE ? PIndexState.INACTIVE : 
newState);
+        index = PTableImpl.builderWithColumns(index, getColumnsToClone(index))
+                .setState(newState == PIndexState.USABLE ? PIndexState.ACTIVE :
+                        newState == PIndexState.UNUSABLE ? 
PIndexState.INACTIVE : newState)
+                .build();
         return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 
0, index);
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11cc13b0/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 fe50a56..b86a1c8 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
@@ -103,6 +103,7 @@ import static 
org.apache.phoenix.schema.PTable.ImmutableStorageScheme.ONE_CELL_P
 import static 
org.apache.phoenix.schema.PTable.ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS;
 import static 
org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
 import static org.apache.phoenix.schema.PTable.ViewType.MAPPED;
+import static org.apache.phoenix.schema.PTableImpl.getColumnsToClone;
 import static org.apache.phoenix.schema.PTableType.TABLE;
 import static org.apache.phoenix.schema.PTableType.VIEW;
 import static org.apache.phoenix.schema.types.PDataType.FALSE_BYTES;
@@ -130,6 +131,7 @@ import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
 
+import com.google.common.base.Objects;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ClusterConnection;
@@ -247,6 +249,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -931,10 +934,21 @@ public class MetaDataClient {
                     + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + 
index.getName().getString());
                 // add the index table with a new name so that it does not 
conflict with the existing index table
                 // and set update cache frequency to that of the view
-                indexesToAdd.add(PTableImpl.makePTable(index, 
modifiedIndexName, viewStatement, view.getUpdateCacheFrequency(), 
view.getTenantId()));
+                if (Objects.equal(viewStatement, index.getViewStatement())) {
+                    indexesToAdd.add(index);
+                } else {
+                    indexesToAdd.add(PTableImpl.builderWithColumns(index, 
getColumnsToClone(index))
+                            .setTableName(modifiedIndexName)
+                            .setViewStatement(viewStatement)
+                            
.setUpdateCacheFrequency(view.getUpdateCacheFrequency())
+                            .setTenantId(view.getTenantId())
+                            .build());
+                }
             }
         }
-        PTable allIndexesTable = PTableImpl.makePTable(view, 
view.getTimeStamp(), indexesToAdd);
+        PTable allIndexesTable = PTableImpl.builderWithColumns(view, 
getColumnsToClone(view))
+                .setIndexes(indexesToAdd == null ? Collections.emptyList() : 
indexesToAdd)
+                .build();
         result.setTable(allIndexesTable);
         return true;
     }
@@ -2650,13 +2664,38 @@ public class MetaDataClient {
                 // TODO: what about stats for system catalog?
                 PName newSchemaName = PNameFactory.newName(schemaName);
                 // Column names and qualifiers and hardcoded for system tables.
-                PTable table = PTableImpl.makePTable(tenantId,newSchemaName, 
PNameFactory.newName(tableName), tableType,
-                        null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, 
PTable.INITIAL_SEQ_NUM,
-                        
PNameFactory.newName(QueryConstants.SYSTEM_TABLE_PK_NAME), null, 
columns.values(), null, null,
-                        Collections.<PTable>emptyList(), isImmutableRows,
-                        Collections.<PName>emptyList(), defaultFamilyName == 
null ? null :
-                                PNameFactory.newName(defaultFamilyName), null,
-                        Boolean.TRUE.equals(disableWAL), false, false, null, 
viewIndexType, null, indexType, true, null, 0, 0L, isNamespaceMapped, 
autoPartitionSeq, isAppendOnlySchema, ONE_CELL_PER_COLUMN, 
NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER, true);
+                PTable table = new PTableImpl.Builder()
+                        .setType(tableType)
+                        .setTimeStamp(MetaDataProtocol.MIN_TABLE_TIMESTAMP)
+                        .setIndexDisableTimestamp(0L)
+                        .setSequenceNumber(PTable.INITIAL_SEQ_NUM)
+                        .setImmutableRows(isImmutableRows)
+                        .setDisableWAL(Boolean.TRUE.equals(disableWAL))
+                        .setMultiTenant(false)
+                        .setStoreNulls(false)
+                        .setViewIndexType(viewIndexType)
+                        .setIndexType(indexType)
+                        .setUpdateCacheFrequency(0)
+                        .setNamespaceMapped(isNamespaceMapped)
+                        .setAutoPartitionSeqName(autoPartitionSeq)
+                        .setAppendOnlySchema(isAppendOnlySchema)
+                        .setImmutableStorageScheme(ONE_CELL_PER_COLUMN)
+                        .setQualifierEncodingScheme(NON_ENCODED_QUALIFIERS)
+                        
.setBaseColumnCount(QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT)
+                        
.setEncodedCQCounter(PTable.EncodedCQCounter.NULL_COUNTER)
+                        .setUseStatsForParallelization(true)
+                        .setExcludedColumns(ImmutableList.of())
+                        .setTenantId(tenantId)
+                        .setSchemaName(newSchemaName)
+                        .setTableName(PNameFactory.newName(tableName))
+                        
.setPkName(PNameFactory.newName(QueryConstants.SYSTEM_TABLE_PK_NAME))
+                        .setDefaultFamilyName(defaultFamilyName == null ? null 
:
+                                PNameFactory.newName(defaultFamilyName))
+                        .setRowKeyOrderOptimizable(true)
+                        .setIndexes(Collections.emptyList())
+                        .setPhysicalNames(ImmutableList.of())
+                        .setColumns(columns.values())
+                        .build();
                 connection.addTable(table, 
MetaDataProtocol.MIN_TABLE_TIMESTAMP);
             }
             
@@ -2930,12 +2969,49 @@ public class MetaDataClient {
                  * for extra safety.
                  */
                 EncodedCQCounter cqCounterToBe = tableType == PTableType.VIEW 
? NULL_COUNTER : cqCounter;
-                PTable table =  PTableImpl.makePTable(
-                        tenantId, newSchemaName, 
PNameFactory.newName(tableName), tableType, indexState, timestamp!=null ? 
timestamp : result.getMutationTime(),
-                        PTable.INITIAL_SEQ_NUM, pkName == null ? null : 
PNameFactory.newName(pkName), saltBucketNum, columns.values(),
-                        parent == null ? null : parent.getSchemaName(), parent 
== null ? null : parent.getTableName(), Collections.<PTable>emptyList(), 
isImmutableRows,
-                        physicalNames, defaultFamilyName == null ? null : 
PNameFactory.newName(defaultFamilyName), viewStatement, 
Boolean.TRUE.equals(disableWAL), multiTenant, storeNulls, viewType,
-                        viewIndexType, result.getViewIndexId(), indexType, 
rowKeyOrderOptimizable, transactionProvider, updateCacheFrequency, 0L, 
isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, 
immutableStorageScheme, encodingScheme, cqCounterToBe, 
useStatsForParallelizationProp);
+                PTable table = new PTableImpl.Builder()
+                        .setType(tableType)
+                        .setState(indexState)
+                        .setTimeStamp(timestamp != null ? timestamp : 
result.getMutationTime())
+                        .setIndexDisableTimestamp(0L)
+                        .setSequenceNumber(PTable.INITIAL_SEQ_NUM)
+                        .setImmutableRows(isImmutableRows)
+                        .setViewStatement(viewStatement)
+                        .setDisableWAL(Boolean.TRUE.equals(disableWAL))
+                        .setMultiTenant(multiTenant)
+                        .setStoreNulls(storeNulls)
+                        .setViewType(viewType)
+                        .setViewIndexType(viewIndexType)
+                        .setViewIndexId(result.getViewIndexId())
+                        .setIndexType(indexType)
+                        .setTransactionProvider(transactionProvider)
+                        .setUpdateCacheFrequency(updateCacheFrequency)
+                        .setNamespaceMapped(isNamespaceMapped)
+                        .setAutoPartitionSeqName(autoPartitionSeq)
+                        .setAppendOnlySchema(isAppendOnlySchema)
+                        .setImmutableStorageScheme(immutableStorageScheme == 
null ?
+                                ImmutableStorageScheme.ONE_CELL_PER_COLUMN : 
immutableStorageScheme)
+                        .setQualifierEncodingScheme(encodingScheme == null ?
+                                QualifierEncodingScheme.NON_ENCODED_QUALIFIERS 
: encodingScheme)
+                        
.setBaseColumnCount(QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT)
+                        .setEncodedCQCounter(cqCounterToBe)
+                        
.setUseStatsForParallelization(useStatsForParallelizationProp)
+                        .setExcludedColumns(ImmutableList.of())
+                        .setTenantId(tenantId)
+                        .setSchemaName(newSchemaName)
+                        .setTableName(PNameFactory.newName(tableName))
+                        .setPkName(pkName == null ? null : 
PNameFactory.newName(pkName))
+                        .setDefaultFamilyName(defaultFamilyName == null ?
+                                null : PNameFactory.newName(defaultFamilyName))
+                        .setRowKeyOrderOptimizable(rowKeyOrderOptimizable)
+                        .setBucketNum(saltBucketNum)
+                        .setIndexes(Collections.emptyList())
+                        .setParentSchemaName((parent == null) ? null : 
parent.getSchemaName())
+                        .setParentTableName((parent == null) ? null : 
parent.getTableName())
+                        .setPhysicalNames(physicalNames == null ?
+                                ImmutableList.of() : 
ImmutableList.copyOf(physicalNames))
+                        .setColumns(columns.values())
+                        .build();
                 result = new MetaDataMutationResult(code, 
result.getMutationTime(), table, true);
                 addTableToCache(result);
                 return table;
@@ -3131,10 +3207,29 @@ public class MetaDataClient {
                                 && (table.isMultiTenant() || 
hasViewIndexTable)) {
                             if (hasViewIndexTable) {
                                 byte[] viewIndexPhysicalName = 
MetaDataUtil.getViewIndexPhysicalName(table.getPhysicalName().getBytes());
-                                PTable viewIndexTable = new PTableImpl(null,
-                                        
SchemaUtil.getSchemaNameFromFullName(viewIndexPhysicalName),
-                                        
SchemaUtil.getTableNameFromFullName(viewIndexPhysicalName), ts,
-                                        
table.getColumnFamilies(),table.isNamespaceMapped(), 
table.getImmutableStorageScheme(), table.getEncodingScheme(), 
table.useStatsForParallelization());
+                                String viewIndexSchemaName = 
SchemaUtil.getSchemaNameFromFullName(viewIndexPhysicalName);
+                                String viewIndexTableName = 
SchemaUtil.getTableNameFromFullName(viewIndexPhysicalName);
+                                PName viewIndexName = 
PNameFactory.newName(SchemaUtil.getTableName(viewIndexSchemaName, 
viewIndexTableName));
+
+                                PTable viewIndexTable = new 
PTableImpl.Builder()
+                                        .setName(viewIndexName)
+                                        .setKey(new PTableKey(tenantId, 
viewIndexName.getString()))
+                                        
.setSchemaName(PNameFactory.newName(viewIndexSchemaName))
+                                        
.setTableName(PNameFactory.newName(viewIndexTableName))
+                                        .setType(PTableType.VIEW)
+                                        .setViewType(ViewType.MAPPED)
+                                        .setTimeStamp(ts)
+                                        .setPkColumns(Collections.emptyList())
+                                        .setAllColumns(Collections.emptyList())
+                                        
.setRowKeySchema(RowKeySchema.EMPTY_SCHEMA)
+                                        .setIndexes(Collections.emptyList())
+                                        
.setFamilyAttributes(table.getColumnFamilies())
+                                        
.setPhysicalNames(Collections.emptyList())
+                                        
.setNamespaceMapped(table.isNamespaceMapped())
+                                        
.setImmutableStorageScheme(table.getImmutableStorageScheme())
+                                        
.setQualifierEncodingScheme(table.getEncodingScheme())
+                                        
.setUseStatsForParallelization(table.useStatsForParallelization())
+                                        .build();
                                 tableRefs.add(new TableRef(null, 
viewIndexTable, ts, false));
                             }
                         }
@@ -3753,13 +3848,32 @@ public class MetaDataClient {
                             long ts = (scn == null ? result.getMutationTime() 
: scn);
                             byte[] viewIndexPhysicalName = MetaDataUtil
                                     
.getViewIndexPhysicalName(table.getPhysicalName().getBytes());
-                            PTable viewIndexTable = new PTableImpl(null,
-                                    
SchemaUtil.getSchemaNameFromFullName(viewIndexPhysicalName),
-                                    
SchemaUtil.getTableNameFromFullName(viewIndexPhysicalName), ts,
-                                    table.getColumnFamilies(), 
table.isNamespaceMapped(), table.getImmutableStorageScheme(), 
table.getEncodingScheme(), table.useStatsForParallelization());
+                            String viewIndexSchemaName = 
SchemaUtil.getSchemaNameFromFullName(viewIndexPhysicalName);
+                            String viewIndexTableName = 
SchemaUtil.getTableNameFromFullName(viewIndexPhysicalName);
+                            PName viewIndexName = 
PNameFactory.newName(SchemaUtil.getTableName(viewIndexSchemaName, 
viewIndexTableName));
+
+                            PTable viewIndexTable = new PTableImpl.Builder()
+                                    .setName(viewIndexName)
+                                    .setKey(new PTableKey(tenantId, 
viewIndexName.getString()))
+                                    
.setSchemaName(PNameFactory.newName(viewIndexSchemaName))
+                                    
.setTableName(PNameFactory.newName(viewIndexTableName))
+                                    .setType(PTableType.VIEW)
+                                    .setViewType(ViewType.MAPPED)
+                                    .setTimeStamp(ts)
+                                    .setPkColumns(Collections.emptyList())
+                                    .setAllColumns(Collections.emptyList())
+                                    .setRowKeySchema(RowKeySchema.EMPTY_SCHEMA)
+                                    .setIndexes(Collections.emptyList())
+                                    
.setFamilyAttributes(table.getColumnFamilies())
+                                    .setPhysicalNames(Collections.emptyList())
+                                    
.setNamespaceMapped(table.isNamespaceMapped())
+                                    
.setImmutableStorageScheme(table.getImmutableStorageScheme())
+                                    
.setQualifierEncodingScheme(table.getEncodingScheme())
+                                    
.setUseStatsForParallelization(table.useStatsForParallelization())
+                                    .build();
                             List<TableRef> tableRefs = 
Collections.singletonList(new TableRef(null, viewIndexTable, ts, false));
                             MutationPlan plan = new 
PostDDLCompiler(connection).compile(tableRefs, null, null,
-                                    Collections.<PColumn> emptyList(), ts);
+                                    Collections.emptyList(), ts);
                             connection.getQueryServices().updateData(plan);
                         }
                     }
@@ -4028,21 +4142,44 @@ public class MetaDataClient {
                         Map<String, List<TableRef>> tenantIdTableRefMap = 
Maps.newHashMap();
                         if (result.getSharedTablesToDelete() != null) {
                             for (SharedTableState sharedTableState : 
result.getSharedTablesToDelete()) {
-                                PTableImpl viewIndexTable =
-                                        new 
PTableImpl(sharedTableState.getTenantId(),
-                                                
sharedTableState.getSchemaName(),
-                                                
sharedTableState.getTableName(), ts,
-                                                table.getColumnFamilies(),
-                                                sharedTableState.getColumns(),
-                                                
sharedTableState.getPhysicalNames(),
-                                                
sharedTableState.getViewIndexType(),
-                                                
sharedTableState.getViewIndexId(),
-                                                table.isMultiTenant(), 
table.isNamespaceMapped(),
-                                                
table.getImmutableStorageScheme(),
-                                                table.getEncodingScheme(),
-                                                table.getEncodedCQCounter(),
-                                                
table.useStatsForParallelization(),
-                                                table.getBucketNum());
+                                ImmutableStorageScheme storageScheme = 
table.getImmutableStorageScheme();
+                                QualifierEncodingScheme 
qualifierEncodingScheme = table.getEncodingScheme();
+                                List<PColumn> columns = 
sharedTableState.getColumns();
+                                if (table.getBucketNum() != null) {
+                                    columns = columns.subList(1, 
columns.size());
+                                }
+
+                                PTableImpl viewIndexTable = new 
PTableImpl.Builder()
+                                        .setPkColumns(Collections.emptyList())
+                                        .setAllColumns(Collections.emptyList())
+                                        
.setRowKeySchema(RowKeySchema.EMPTY_SCHEMA)
+                                        .setIndexes(Collections.emptyList())
+                                        
.setFamilyAttributes(table.getColumnFamilies())
+                                        .setType(PTableType.INDEX)
+                                        .setTimeStamp(ts)
+                                        .setMultiTenant(table.isMultiTenant())
+                                        
.setViewIndexType(sharedTableState.getViewIndexType())
+                                        
.setViewIndexId(sharedTableState.getViewIndexId())
+                                        
.setNamespaceMapped(table.isNamespaceMapped())
+                                        .setAppendOnlySchema(false)
+                                        
.setImmutableStorageScheme(storageScheme == null ?
+                                                
ImmutableStorageScheme.ONE_CELL_PER_COLUMN : storageScheme)
+                                        
.setQualifierEncodingScheme(qualifierEncodingScheme == null ?
+                                                
QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : qualifierEncodingScheme)
+                                        
.setEncodedCQCounter(table.getEncodedCQCounter())
+                                        
.setUseStatsForParallelization(table.useStatsForParallelization())
+                                        .setExcludedColumns(ImmutableList.of())
+                                        
.setTenantId(sharedTableState.getTenantId())
+                                        
.setSchemaName(sharedTableState.getSchemaName())
+                                        
.setTableName(sharedTableState.getTableName())
+                                        .setRowKeyOrderOptimizable(false)
+                                        .setBucketNum(table.getBucketNum())
+                                        .setIndexes(Collections.emptyList())
+                                        
.setPhysicalNames(sharedTableState.getPhysicalNames() == null ?
+                                                ImmutableList.of() :
+                                                
ImmutableList.copyOf(sharedTableState.getPhysicalNames()))
+                                        .setColumns(columns)
+                                        .build();
                                 TableRef indexTableRef = new 
TableRef(viewIndexTable);
                                 PName indexTableTenantId = 
sharedTableState.getTenantId();
                                 if (indexTableTenantId==null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/11cc13b0/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 598cc79..404f5b4 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
@@ -17,7 +17,10 @@
  */
 package org.apache.phoenix.schema;
 
+import static org.apache.phoenix.schema.PTableImpl.getColumnsToClone;
+
 import java.sql.SQLException;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
@@ -27,7 +30,6 @@ import org.apache.phoenix.parse.PSchema;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.util.ReadOnlyProps;
-import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TimeKeeper;
 
 import com.google.common.collect.Lists;
@@ -121,7 +123,11 @@ public class PMetaDataImpl implements PMetaData {
                 }
                 newIndexes.add(table);
                 netGain -= oldParentRef.getEstimatedSize();
-                newParentTable = 
PTableImpl.makePTable(oldParentRef.getTable(), table.getTimeStamp(), 
newIndexes);
+                newParentTable = 
PTableImpl.builderWithColumns(oldParentRef.getTable(),
+                        getColumnsToClone(oldParentRef.getTable()))
+                        .setIndexes(newIndexes)
+                        .setTimeStamp(table.getTimeStamp())
+                        .build();
                 newParentTableRef = 
tableRefFactory.makePTableRef(newParentTable, this.timeKeeper.getCurrentTime(), 
parentResolvedTimestamp);
                 netGain += newParentTableRef.getEstimatedSize();
             }
@@ -173,10 +179,14 @@ public class PMetaDataImpl implements PMetaData {
                     PTable index = newIndexes.get(i);
                     if (index.getName().getString().equals(tableName)) {
                         newIndexes.remove(i);
-                        PTable parentTable = PTableImpl.makePTable(
-                                parentTableRef.getTable(),
-                                tableTimeStamp == HConstants.LATEST_TIMESTAMP 
? parentTableRef.getTable().getTimeStamp() : tableTimeStamp,
-                                newIndexes);
+                        PTableImpl.Builder parentTableBuilder =
+                                
PTableImpl.builderWithColumns(parentTableRef.getTable(),
+                                        
getColumnsToClone(parentTableRef.getTable()))
+                                .setIndexes(newIndexes == null ? 
Collections.emptyList() : newIndexes);
+                        if (tableTimeStamp != HConstants.LATEST_TIMESTAMP) {
+                            parentTableBuilder.setTimeStamp(tableTimeStamp);
+                        }
+                        PTable parentTable = parentTableBuilder.build();
                         metaData.put(parentTable.getKey(), 
tableRefFactory.makePTableRef(parentTable, this.timeKeeper.getCurrentTime(), 
parentTableRef.getResolvedTimeStamp()));
                         break;
                     }
@@ -218,8 +228,10 @@ public class PMetaDataImpl implements PMetaData {
                     oldColumn.getTimestamp());
                 columns.add(newColumn);
             }
-            
-            table = PTableImpl.makePTable(table, tableTimeStamp, tableSeqNum, 
columns);
+            table = PTableImpl.builderWithColumns(table, columns)
+                    .setTimeStamp(tableTimeStamp)
+                    .setSequenceNumber(tableSeqNum)
+                    .build();
         }
         tables.put(table.getKey(), tableRefFactory.makePTableRef(table, 
this.timeKeeper.getCurrentTime(), resolvedTime));
     }

Reply via email to