Merge remote-tracking branch 'upstream/master' into PHOENIX-3534

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

Branch: refs/heads/system-catalog
Commit: d615dc0cb707d33f58eaeb897269ee074b33c550
Parents: f30994f 78636a3
Author: Thomas D'Silva <tdsi...@apache.org>
Authored: Wed Apr 25 09:11:00 2018 -0700
Committer: Thomas D'Silva <tdsi...@apache.org>
Committed: Wed Apr 25 09:11:00 2018 -0700

----------------------------------------------------------------------
 bin/performance.py                              |    4 +-
 bin/phoenix_utils.py                            |    2 +-
 dev/make_rc.sh                                  |   15 +-
 phoenix-assembly/pom.xml                        |    2 +-
 phoenix-client/pom.xml                          |    2 +-
 phoenix-core/pom.xml                            |   16 +-
 ...ReplayWithIndexWritesAndCompressedWALIT.java |    2 +-
 .../org/apache/phoenix/end2end/AggregateIT.java |  130 +-
 .../apache/phoenix/end2end/AlterTableIT.java    |    2 +-
 .../phoenix/end2end/AlterTableWithViewsIT.java  |    6 +-
 .../phoenix/end2end/AppendOnlySchemaIT.java     |    2 +-
 .../phoenix/end2end/ArrayAppendFunctionIT.java  |   43 +-
 .../phoenix/end2end/ArrayRemoveFunctionIT.java  |  383 ++++
 .../phoenix/end2end/BasePermissionsIT.java      |    4 +
 .../phoenix/end2end/ChangePermissionsIT.java    |   34 +-
 .../phoenix/end2end/CollationKeyFunctionIT.java |   37 +-
 .../end2end/ColumnProjectionOptimizationIT.java |   14 +-
 .../ConnectionQueryServicesTestImpl.java        |   34 +-
 .../phoenix/end2end/CostBasedDecisionIT.java    |  420 +++--
 .../apache/phoenix/end2end/CreateTableIT.java   |    5 +-
 .../org/apache/phoenix/end2end/DeleteIT.java    |  101 ++
 .../org/apache/phoenix/end2end/DropTableIT.java |   43 +
 .../apache/phoenix/end2end/DynamicColumnIT.java |   14 +-
 .../apache/phoenix/end2end/DynamicFamilyIT.java |   26 +-
 .../end2end/ExplainPlanWithStatsEnabledIT.java  |  334 ++--
 .../apache/phoenix/end2end/IndexExtendedIT.java |    6 +-
 .../org/apache/phoenix/end2end/IndexToolIT.java |    6 +-
 .../phoenix/end2end/LocalIndexSplitMergeIT.java |   59 +
 .../org/apache/phoenix/end2end/MapReduceIT.java |   69 +-
 .../phoenix/end2end/MappingTableDataTypeIT.java |    6 +-
 .../MigrateSystemTablesToSystemNamespaceIT.java |   49 +-
 .../phoenix/end2end/MultiCfQueryExecIT.java     |   47 +
 .../end2end/NamespaceSchemaMappingIT.java       |   19 +-
 .../phoenix/end2end/NativeHBaseTypesIT.java     |   30 +-
 .../org/apache/phoenix/end2end/OrderByIT.java   |  129 +-
 .../phoenix/end2end/ParallelStatsEnabledIT.java |    1 +
 .../PartialScannerResultsDisabledIT.java        |    2 +-
 .../apache/phoenix/end2end/PhoenixDriverIT.java |   99 +-
 .../end2end/QueryDatabaseMetaDataIT.java        |   39 +-
 .../apache/phoenix/end2end/QueryLoggerIT.java   |  358 ++++
 .../apache/phoenix/end2end/ReverseScanIT.java   |   88 +-
 .../phoenix/end2end/SortMergeJoinMoreIT.java    |  116 +-
 .../phoenix/end2end/SpillableGroupByIT.java     |   74 +-
 .../SystemCatalogCreationOnConnectionIT.java    |  626 +++++++
 .../end2end/SystemTablePermissionsIT.java       |    2 +-
 .../end2end/TenantSpecificTablesDDLIT.java      |    2 +
 .../UngroupedAggregateRegionObserverIT.java     |  171 --
 .../org/apache/phoenix/end2end/UpgradeIT.java   |    2 +-
 .../end2end/UpsertSelectAutoCommitIT.java       |   28 +
 .../phoenix/end2end/UserDefinedFunctionsIT.java |  165 +-
 .../phoenix/end2end/index/BaseIndexIT.java      |   22 +
 .../index/ChildViewsUseParentViewIndexIT.java   |   10 +-
 .../phoenix/end2end/index/ImmutableIndexIT.java |    3 +
 .../end2end/index/MutableIndexFailureIT.java    |   27 +-
 .../phoenix/end2end/index/MutableIndexIT.java   |   95 +
 .../end2end/index/MutableIndexRebuilderIT.java  |  143 ++
 .../end2end/index/PartialIndexRebuilderIT.java  |  130 +-
 .../phoenix/end2end/index/ViewIndexIT.java      |   45 +-
 .../end2end/index/txn/TxWriteFailureIT.java     |    6 +-
 .../apache/phoenix/end2end/join/BaseJoinIT.java |    2 +
 .../phoenix/end2end/join/HashJoinMoreIT.java    |    4 +-
 .../apache/phoenix/end2end/join/SubqueryIT.java |   49 +-
 .../join/SubqueryUsingSortMergeJoinIT.java      |   12 +-
 .../phoenix/monitoring/PhoenixMetricsIT.java    |   24 +-
 .../phoenix/tx/FlappingTransactionIT.java       |   11 +-
 .../phoenix/tx/ParameterizedTransactionIT.java  |   18 +-
 .../org/apache/phoenix/tx/TransactionIT.java    |   12 +
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |    5 +-
 .../hadoop/hbase/ipc/PhoenixRpcScheduler.java   |   30 +
 .../controller/ClientRpcControllerFactory.java  |   18 +-
 .../ipc/controller/IndexRpcController.java      |    8 +-
 ...erRegionServerIndexRpcControllerFactory.java |   18 +-
 ...egionServerMetadataRpcControllerFactory.java |   17 +-
 .../ipc/controller/MetadataRpcController.java   |    8 +-
 .../IndexHalfStoreFileReaderGenerator.java      |   30 +-
 .../LocalIndexStoreFileScanner.java             |    4 +-
 .../hadoop/hbase/regionserver/ScanInfoUtil.java |   12 +
 .../org/apache/phoenix/cache/GlobalCache.java   |    6 +-
 .../org/apache/phoenix/cache/HashCache.java     |    1 +
 .../phoenix/cache/IndexMetaDataCache.java       |    7 +
 .../apache/phoenix/cache/ServerCacheClient.java |    7 +-
 .../org/apache/phoenix/cache/TenantCache.java   |    2 +-
 .../apache/phoenix/cache/TenantCacheImpl.java   |    4 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |  124 +-
 .../org/apache/phoenix/compile/ExplainPlan.java |   10 +
 .../apache/phoenix/compile/FromCompiler.java    |    2 +-
 .../apache/phoenix/compile/GroupByCompiler.java |   19 +-
 .../apache/phoenix/compile/JoinCompiler.java    |  324 ++--
 .../phoenix/compile/ListJarsQueryPlan.java      |    8 +-
 .../apache/phoenix/compile/OrderByCompiler.java |   11 +-
 .../phoenix/compile/OrderPreservingTracker.java |   13 +
 .../apache/phoenix/compile/PostDDLCompiler.java |    2 +-
 .../apache/phoenix/compile/QueryCompiler.java   |  498 +++---
 .../org/apache/phoenix/compile/QueryPlan.java   |    2 +
 .../org/apache/phoenix/compile/ScanRanges.java  |   17 +-
 .../apache/phoenix/compile/SequenceManager.java |    2 +-
 .../phoenix/compile/StatementContext.java       |   10 +
 .../phoenix/compile/SubselectRewriter.java      |    5 +
 .../apache/phoenix/compile/TraceQueryPlan.java  |    6 +
 .../compile/TupleProjectionCompiler.java        |   44 +-
 .../apache/phoenix/compile/UnionCompiler.java   |    6 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   47 +-
 .../apache/phoenix/compile/WhereOptimizer.java  |   23 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   15 +-
 .../DelegateRegionCoprocessorEnvironment.java   |   15 +-
 .../coprocessor/DelegateRegionObserver.java     |   32 +-
 .../GroupedAggregateRegionObserver.java         |    4 +
 .../coprocessor/MetaDataEndpointImpl.java       |  160 +-
 .../phoenix/coprocessor/MetaDataProtocol.java   |   16 +-
 .../coprocessor/MetaDataRegionObserver.java     |   21 +-
 .../coprocessor/PhoenixAccessController.java    |  102 +-
 .../PhoenixMetaDataCoprocessorHost.java         |    6 +
 .../PhoenixTransactionalProcessor.java          |   28 -
 .../phoenix/coprocessor/ScanRegionObserver.java |    4 +-
 .../coprocessor/ServerCachingEndpointImpl.java  |    4 +-
 .../coprocessor/ServerCachingProtocol.java      |    2 +-
 .../TephraTransactionalProcessor.java           |   29 +
 .../UngroupedAggregateRegionObserver.java       |  311 ++--
 .../coprocessor/generated/MetaDataProtos.java   |  183 +-
 .../coprocessor/generated/PTableProtos.java     |  110 +-
 .../generated/ServerCachingProtos.java          |  117 +-
 .../phoenix/exception/SQLExceptionCode.java     |   17 +-
 .../exception/UpgradeRequiredException.java     |   13 +-
 .../apache/phoenix/execute/AggregatePlan.java   |   53 +-
 .../apache/phoenix/execute/BaseQueryPlan.java   |   21 +-
 .../phoenix/execute/ClientAggregatePlan.java    |   46 +-
 .../phoenix/execute/ClientProcessingPlan.java   |    4 +
 .../apache/phoenix/execute/ClientScanPlan.java  |   22 +-
 .../apache/phoenix/execute/CorrelatePlan.java   |   26 +-
 .../apache/phoenix/execute/CursorFetchPlan.java |    6 +
 .../apache/phoenix/execute/DelegateHTable.java  |   20 +
 .../apache/phoenix/execute/HashJoinPlan.java    |  128 +-
 .../execute/LiteralResultIterationPlan.java     |    8 +-
 .../apache/phoenix/execute/MutationState.java   |  219 ++-
 .../PhoenixTxIndexMutationGenerator.java        |  443 +++++
 .../org/apache/phoenix/execute/ScanPlan.java    |   32 +-
 .../phoenix/execute/SortMergeJoinPlan.java      |   20 +-
 .../phoenix/execute/TupleProjectionPlan.java    |    6 +
 .../org/apache/phoenix/execute/UnionPlan.java   |   12 +-
 .../apache/phoenix/execute/UnnestArrayPlan.java |    6 +
 .../execute/visitor/AvgRowWidthVisitor.java     |  205 +++
 .../execute/visitor/ByteCountVisitor.java       |  125 ++
 .../execute/visitor/QueryPlanVisitor.java       |   46 +
 .../execute/visitor/RowCountVisitor.java        |  335 ++++
 .../expression/BaseSingleExpression.java        |    5 +
 .../phoenix/expression/ExpressionType.java      |  119 +-
 .../expression/aggregator/Aggregators.java      |    3 +-
 .../aggregator/ClientAggregators.java           |    3 +-
 .../DistinctValueWithCountServerAggregator.java |   20 +-
 .../aggregator/ServerAggregators.java           |    6 +-
 .../function/ArrayAppendFunction.java           |   11 +-
 .../function/ArrayConcatFunction.java           |    3 +-
 .../function/ArrayModifierFunction.java         |   55 +-
 .../function/ArrayPrependFunction.java          |    9 +-
 .../function/ArrayRemoveFunction.java           |   88 +
 .../function/CollationKeyFunction.java          |   24 +-
 .../expression/function/LowerFunction.java      |   28 +-
 .../expression/function/ScalarFunction.java     |   15 +-
 .../TransactionProviderNameFunction.java        |   81 +
 .../expression/function/UDFExpression.java      |   39 +-
 .../expression/function/UpperFunction.java      |   29 +-
 .../org/apache/phoenix/hbase/index/Indexer.java |   25 +-
 .../hbase/index/builder/BaseIndexBuilder.java   |    4 +-
 .../hbase/index/builder/BaseIndexCodec.java     |    7 -
 .../hbase/index/builder/IndexBuildManager.java  |    4 +
 .../phoenix/hbase/index/covered/IndexCodec.java |   14 +-
 .../hbase/index/covered/IndexMetaData.java      |   12 +-
 .../hbase/index/covered/LocalTableState.java    |   16 +-
 .../hbase/index/covered/NonTxIndexBuilder.java  |    2 +-
 .../phoenix/hbase/index/covered/TableState.java |   15 -
 .../index/exception/IndexWriteException.java    |   52 +-
 .../MultiIndexWriteFailureException.java        |   39 +-
 .../SingleIndexWriteFailureException.java       |   32 +-
 .../hbase/index/scanner/ScannerBuilder.java     |   16 +
 .../hbase/index/write/IndexCommitter.java       |    2 +-
 .../phoenix/hbase/index/write/IndexWriter.java  |   21 +-
 .../hbase/index/write/IndexWriterUtils.java     |   87 +-
 .../write/ParallelWriterIndexCommitter.java     |   19 +-
 .../hbase/index/write/RecoveryIndexWriter.java  |    4 +-
 .../TrackingParallelWriterIndexCommitter.java   |   20 +-
 .../apache/phoenix/index/IndexMaintainer.java   |   39 +-
 .../index/IndexMetaDataCacheFactory.java        |    9 +-
 .../phoenix/index/PhoenixIndexBuilder.java      |   21 +-
 .../apache/phoenix/index/PhoenixIndexCodec.java |   36 +-
 .../index/PhoenixIndexFailurePolicy.java        |  207 ++-
 .../phoenix/index/PhoenixIndexMetaData.java     |   74 +-
 .../index/PhoenixIndexMetaDataBuilder.java      |  107 ++
 .../index/PhoenixTransactionalIndexer.java      |  469 +----
 .../phoenix/iterate/BaseResultIterators.java    |  432 ++++-
 .../apache/phoenix/iterate/ExplainTable.java    |    3 +-
 .../NonAggregateRegionScannerFactory.java       |   61 +-
 .../phoenix/iterate/ParallelIterators.java      |    8 +-
 .../phoenix/iterate/RegionScannerFactory.java   |    4 +-
 .../phoenix/iterate/ScanningResultIterator.java |   26 +-
 .../apache/phoenix/iterate/SerialIterators.java |    4 +-
 .../apache/phoenix/iterate/SnapshotScanner.java |   12 +-
 .../phoenix/iterate/TableResultIterator.java    |    7 +-
 .../apache/phoenix/jdbc/PhoenixConnection.java  |   19 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   35 +-
 .../phoenix/jdbc/PhoenixPreparedStatement.java  |   11 +-
 .../apache/phoenix/jdbc/PhoenixResultSet.java   |   38 +
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   85 +-
 .../apache/phoenix/join/HashCacheFactory.java   |   34 +-
 .../java/org/apache/phoenix/log/LogLevel.java   |   22 +
 .../java/org/apache/phoenix/log/LogWriter.java  |   51 +
 .../log/QueryLogDetailsEventHandler.java        |   63 +
 .../org/apache/phoenix/log/QueryLogInfo.java    |   87 +
 .../org/apache/phoenix/log/QueryLogState.java   |   22 +
 .../org/apache/phoenix/log/QueryLogger.java     |  145 ++
 .../log/QueryLoggerDefaultExceptionHandler.java |   51 +
 .../phoenix/log/QueryLoggerDisruptor.java       |  117 ++
 .../org/apache/phoenix/log/QueryLoggerUtil.java |   62 +
 .../org/apache/phoenix/log/RingBufferEvent.java |   93 +
 .../phoenix/log/RingBufferEventTranslator.java  |   53 +
 .../org/apache/phoenix/log/TableLogWriter.java  |  125 ++
 .../phoenix/mapreduce/PhoenixInputFormat.java   |   51 +-
 .../index/PhoenixIndexPartialBuildMapper.java   |    5 +
 .../util/PhoenixConfigurationUtil.java          |   63 +-
 .../mapreduce/util/PhoenixMapReduceUtil.java    |    5 +-
 .../phoenix/memory/ChildMemoryManager.java      |    7 +-
 .../phoenix/memory/GlobalMemoryManager.java     |   19 +-
 .../phoenix/monitoring/GlobalClientMetrics.java |   27 +-
 .../phoenix/monitoring/ReadMetricQueue.java     |   44 +-
 .../phoenix/monitoring/ScanMetricsHolder.java   |   48 +-
 .../GenSubqueryParamValuesRewriter.java         |  153 ++
 .../apache/phoenix/optimize/QueryOptimizer.java |  212 ++-
 .../phoenix/parse/ArrayModifierParseNode.java   |   36 +
 .../apache/phoenix/parse/FunctionParseNode.java |   12 +-
 .../java/org/apache/phoenix/parse/HintNode.java |    4 +
 .../org/apache/phoenix/parse/ParseNode.java     |   26 +-
 .../phoenix/query/ConnectionQueryServices.java  |   12 +-
 .../query/ConnectionQueryServicesImpl.java      |  985 ++++++-----
 .../query/ConnectionlessQueryServicesImpl.java  |   77 +-
 .../query/DelegateConnectionQueryServices.java  |   23 +-
 .../apache/phoenix/query/QueryConstants.java    |  136 +-
 .../org/apache/phoenix/query/QueryServices.java |   16 +-
 .../phoenix/query/QueryServicesOptions.java     |   29 +-
 .../apache/phoenix/schema/DelegateTable.java    |    8 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  335 ++--
 .../org/apache/phoenix/schema/PColumnImpl.java  |    6 +-
 .../org/apache/phoenix/schema/PIndexState.java  |    7 +-
 .../apache/phoenix/schema/PMetaDataImpl.java    |    2 +-
 .../java/org/apache/phoenix/schema/PTable.java  |    2 +
 .../org/apache/phoenix/schema/PTableImpl.java   |   92 +-
 .../phoenix/schema/RowKeyValueAccessor.java     |    2 +-
 .../apache/phoenix/schema/TableProperty.java    |   18 +
 .../schema/TablesNotInSyncException.java        |   17 +
 .../phoenix/schema/stats/StatisticsWriter.java  |    9 +-
 .../transaction/OmidTransactionContext.java     |   57 +-
 .../transaction/OmidTransactionProvider.java    |   84 +
 .../transaction/OmidTransactionTable.java       |  364 ----
 .../transaction/PhoenixTransactionClient.java   |   23 +
 .../transaction/PhoenixTransactionContext.java  |  169 +-
 .../transaction/PhoenixTransactionProvider.java |   51 +
 .../transaction/PhoenixTransactionService.java  |   24 +
 .../transaction/PhoenixTransactionalTable.java  |  149 --
 .../transaction/TephraTransactionContext.java   |  205 +--
 .../transaction/TephraTransactionProvider.java  |  197 +++
 .../transaction/TephraTransactionTable.java     |  350 ----
 .../phoenix/transaction/TransactionFactory.java |  139 +-
 .../java/org/apache/phoenix/util/CostUtil.java  |   61 +-
 .../org/apache/phoenix/util/ExpressionUtil.java |    2 +-
 .../java/org/apache/phoenix/util/IndexUtil.java |    6 +-
 .../org/apache/phoenix/util/KeyValueUtil.java   |   12 +
 .../org/apache/phoenix/util/PhoenixRuntime.java |   27 +-
 .../java/org/apache/phoenix/util/QueryUtil.java |   14 +-
 .../java/org/apache/phoenix/util/ScanUtil.java  |   29 +-
 .../org/apache/phoenix/util/SchemaUtil.java     |    6 +
 .../org/apache/phoenix/util/ServerUtil.java     |   95 +-
 .../java/org/apache/phoenix/util/SizedUtil.java |    2 +-
 .../apache/phoenix/util/TransactionUtil.java    |   93 +-
 .../org/apache/phoenix/util/UpgradeUtil.java    |   40 +-
 .../wal/ReadWriteKeyValuesWithCodecTest.java    |    6 +-
 .../apache/phoenix/cache/TenantCacheTest.java   |   14 +-
 .../phoenix/compile/QueryCompilerTest.java      |  815 ++++++++-
 .../phoenix/compile/QueryOptimizerTest.java     |   41 +
 .../phoenix/compile/WhereOptimizerTest.java     |  235 +++
 .../phoenix/execute/CorrelatePlanTest.java      |    4 +-
 .../execute/LiteralResultIteratorPlanTest.java  |    4 +-
 .../expression/ArrayRemoveFunctionTest.java     |  285 +++
 .../function/CollationKeyFunctionTest.java      |   14 +-
 .../expression/function/LowerFunctionTest.java  |   93 +
 .../expression/function/UpperFunctionTest.java  |   92 +
 .../index/covered/CoveredColumnIndexCodec.java  |    8 +-
 .../covered/CoveredIndexCodecForTesting.java    |    5 +-
 .../index/covered/LocalTableStateTest.java      |   38 +-
 .../index/covered/NonTxIndexBuilderTest.java    |    3 +
 .../covered/TestCoveredColumnIndexCodec.java    |   12 +-
 .../hbase/index/write/TestIndexWriter.java      |   25 +-
 .../index/write/TestParalleIndexWriter.java     |   12 +-
 .../write/TestParalleWriterIndexCommitter.java  |   11 +-
 .../index/write/TestWALRecoveryCaching.java     |    7 +-
 .../recovery/TestPerRegionIndexWriteCache.java  |    4 +-
 .../phoenix/memory/MemoryManagerTest.java       |   15 +
 .../java/org/apache/phoenix/query/BaseTest.java |   26 +-
 .../query/ConnectionQueryServicesImplTest.java  |   11 +-
 .../apache/phoenix/query/KeyRangeClipTest.java  |  155 ++
 .../query/ParallelIteratorsSplitTest.java       |    8 +-
 .../phoenix/query/QueryServicesTestImpl.java    |   15 +-
 .../phoenix/schema/PMetaDataImplTest.java       |   21 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |   54 +-
 phoenix-flume/pom.xml                           |    8 +-
 phoenix-hive/pom.xml                            |    2 +-
 .../hive/mapreduce/PhoenixInputFormat.java      |    3 +-
 phoenix-kafka/pom.xml                           |   12 +-
 phoenix-load-balancer/pom.xml                   |    2 +-
 phoenix-pherf/pom.xml                           |    6 +-
 .../phoenix/pherf/configuration/Column.java     |   23 +-
 .../pherf/configuration/DataTypeMapping.java    |    6 +-
 .../phoenix/pherf/configuration/Query.java      |   25 +
 .../phoenix/pherf/configuration/Scenario.java   |    3 +-
 .../phoenix/pherf/result/QueryResult.java       |   10 +-
 .../phoenix/pherf/result/ResultManager.java     |   18 +-
 .../apache/phoenix/pherf/result/ResultUtil.java |    9 +-
 .../phoenix/pherf/rules/RulesApplier.java       |  142 +-
 .../phoenix/pherf/schema/SchemaReader.java      |    2 +-
 .../apache/phoenix/pherf/util/PhoenixUtil.java  |   44 +-
 .../pherf/workload/MultiThreadedRunner.java     |   35 +-
 .../phoenix/pherf/workload/QueryExecutor.java   |   36 +-
 .../phoenix/pherf/workload/WriteWorkload.java   |   54 +-
 .../scenario/prod_test_unsalted_scenario.xml    |   14 +-
 .../org/apache/phoenix/pherf/ColumnTest.java    |    3 +
 .../phoenix/pherf/ConfigurationParserTest.java  |    4 +-
 .../apache/phoenix/pherf/RuleGeneratorTest.java |  107 +-
 .../test/resources/datamodel/test_schema.sql    |    4 +
 .../test/resources/scenario/test_scenario.xml   |   48 +-
 phoenix-pig/pom.xml                             |    6 +-
 phoenix-protocol/src/main/MetaDataService.proto |    1 +
 phoenix-protocol/src/main/PTable.proto          |    1 +
 .../src/main/ServerCachingService.proto         |    1 +
 phoenix-queryserver-client/pom.xml              |    6 +-
 phoenix-queryserver/pom.xml                     |    6 +-
 .../HttpParamImpersonationQueryServerIT.java    |   16 +-
 .../phoenix/end2end/SecureQueryServerIT.java    |   15 +-
 .../phoenix/queryserver/server/QueryServer.java |   21 +-
 phoenix-server/pom.xml                          |    2 +-
 phoenix-spark/pom.xml                           |    2 +-
 .../phoenix/spark/DataFrameFunctions.scala      |    6 +-
 .../phoenix/spark/ProductRDDFunctions.scala     |    6 +-
 phoenix-tracing-webapp/pom.xml                  |    2 +-
 pom.xml                                         |   35 +-
 python/.gitignore                               |    8 +
 python/.gitlab-ci.yml                           |  149 ++
 python/NEWS.rst                                 |   44 +
 python/README.rst                               |  136 ++
 python/RELEASING.rst                            |   12 +
 python/ci/build-env/Dockerfile                  |    7 +
 python/ci/phoenix/Dockerfile                    |   33 +
 python/ci/phoenix/docker-entrypoint.sh          |   24 +
 python/ci/phoenix/hbase-site.xml                |   12 +
 python/doc/Makefile                             |  192 ++
 python/doc/api.rst                              |   30 +
 python/doc/conf.py                              |  287 +++
 python/doc/index.rst                            |   27 +
 python/doc/versions.rst                         |    3 +
 python/docker-compose.yml                       |   21 +
 python/examples/basic.py                        |   27 +
 python/examples/shell.py                        |   33 +
 python/gen-protobuf.sh                          |   38 +
 python/phoenixdb/__init__.py                    |   68 +
 python/phoenixdb/avatica/__init__.py            |   16 +
 python/phoenixdb/avatica/client.py              |  510 ++++++
 python/phoenixdb/avatica/proto/__init__.py      |    0
 python/phoenixdb/avatica/proto/common_pb2.py    | 1667 ++++++++++++++++++
 python/phoenixdb/avatica/proto/requests_pb2.py  | 1206 +++++++++++++
 python/phoenixdb/avatica/proto/responses_pb2.py |  917 ++++++++++
 python/phoenixdb/connection.py                  |  187 ++
 python/phoenixdb/cursor.py                      |  347 ++++
 python/phoenixdb/errors.py                      |   93 +
 python/phoenixdb/tests/__init__.py              |   44 +
 python/phoenixdb/tests/dbapi20.py               |  857 +++++++++
 python/phoenixdb/tests/test_avatica.py          |   25 +
 python/phoenixdb/tests/test_connection.py       |   42 +
 python/phoenixdb/tests/test_db.py               |   99 ++
 python/phoenixdb/tests/test_dbapi20.py          |  122 ++
 python/phoenixdb/tests/test_errors.py           |   60 +
 python/phoenixdb/tests/test_types.py            |  327 ++++
 python/phoenixdb/types.py                       |  202 +++
 python/requirements.txt                         |   20 +
 python/setup.cfg                                |   34 +
 python/setup.py                                 |   64 +
 python/tox.ini                                  |   24 +
 382 files changed, 22540 insertions(+), 5304 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
----------------------------------------------------------------------
diff --cc 
phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
index cbc1021,627e453..7960543
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
@@@ -59,10 -63,10 +63,10 @@@ public class MigrateSystemTablesToSyste
  
      private static final Set<String> PHOENIX_SYSTEM_TABLES = new 
HashSet<>(Arrays.asList(
              "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", 
"SYSTEM.FUNCTION",
-             "SYSTEM.MUTEX", "SYSTEM.CHILD_LINK"));
+             "SYSTEM.MUTEX","SYSTEM.LOG"));
      private static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = 
new HashSet<>(
              Arrays.asList("SYSTEM:CATALOG", "SYSTEM:SEQUENCE", 
"SYSTEM:STATS", "SYSTEM:FUNCTION",
-                     "SYSTEM:MUTEX", "SYSTEM:CHILD_LINK"));
 -                    "SYSTEM:MUTEX","SYSTEM:LOG"));
++                    "SYSTEM:MUTEX","SYSTEM:LOG", "SYSTEM:CHILD_LINK"));
      private static final String SCHEMA_NAME = "MIGRATETEST";
      private static final String TABLE_NAME =
              SCHEMA_NAME + "." + 
MigrateSystemTablesToSystemNamespaceIT.class.getSimpleName().toUpperCase();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
----------------------------------------------------------------------
diff --cc 
phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
index 6830d54,34a1312..41beaa1
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
@@@ -491,10 -491,10 +491,12 @@@ public class TenantSpecificTablesDDLIT 
              assertTrue(rs.next());
              assertTableMetaData(rs, 
PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, 
PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE, PTableType.SYSTEM);
              assertTrue(rs.next());
 +            assertTableMetaData(rs, 
PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, 
PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_TABLE, PTableType.SYSTEM);
 +            assertTrue(rs.next());
              assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, 
SYSTEM_FUNCTION_TABLE, SYSTEM);
              assertTrue(rs.next());
+             assertTableMetaData(rs, 
PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, 
PhoenixDatabaseMetaData.SYSTEM_LOG_TABLE, PTableType.SYSTEM);
+             assertTrue(rs.next());
              assertTableMetaData(rs, 
PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, 
PhoenixDatabaseMetaData.TYPE_SEQUENCE, PTableType.SYSTEM);
              assertTrue(rs.next());
              assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, 
PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE, PTableType.SYSTEM);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseIndexIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ChildViewsUseParentViewIndexIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --cc 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index 6277b78,4c72c2d..06c9aa9
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@@ -93,11 -95,8 +94,10 @@@ import java.util.Collections
  import java.util.Comparator;
  import java.util.HashSet;
  import java.util.Iterator;
- import java.util.LinkedList;
  import java.util.List;
 +import java.util.ListIterator;
  import java.util.Map;
 +import java.util.Map.Entry;
  import java.util.NavigableMap;
  import java.util.Set;
  
@@@ -213,7 -216,9 +212,6 @@@ import org.apache.phoenix.schema.Sequen
  import org.apache.phoenix.schema.SequenceNotFoundException;
  import org.apache.phoenix.schema.SortOrder;
  import org.apache.phoenix.schema.TableNotFoundException;
--import org.apache.phoenix.schema.TableProperty;
 -import org.apache.phoenix.schema.TableRef;
 -import org.apache.phoenix.schema.tuple.ResultTuple;
  import org.apache.phoenix.schema.types.PBinary;
  import org.apache.phoenix.schema.types.PBoolean;
  import org.apache.phoenix.schema.types.PDataType;
@@@ -550,30 -557,30 +552,35 @@@ public class MetaDataEndpointImpl exten
                      TableName.valueOf(table.getPhysicalName().getBytes()));
  
              
builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
-             long disableIndexTimestamp = table.getIndexDisableTimestamp();
-             long minNonZerodisableIndexTimestamp = disableIndexTimestamp > 0 
? disableIndexTimestamp : Long.MAX_VALUE;
-             for (PTable index : table.getIndexes()) {
-                 disableIndexTimestamp = index.getIndexDisableTimestamp();
-                 if (disableIndexTimestamp > 0 && (index.getIndexState() == 
PIndexState.ACTIVE || index.getIndexState() == PIndexState.PENDING_ACTIVE) && 
disableIndexTimestamp < minNonZerodisableIndexTimestamp) {
-                     minNonZerodisableIndexTimestamp = disableIndexTimestamp;
+             builder.setMutationTime(currentTime);
+             if (blockWriteRebuildIndex) {
+                 long disableIndexTimestamp = table.getIndexDisableTimestamp();
+                 long minNonZerodisableIndexTimestamp = disableIndexTimestamp 
> 0 ? disableIndexTimestamp : Long.MAX_VALUE;
+                 for (PTable index : table.getIndexes()) {
+                     disableIndexTimestamp = index.getIndexDisableTimestamp();
+                     if (disableIndexTimestamp > 0
+                             && (index.getIndexState() == PIndexState.ACTIVE
+                                     || index.getIndexState() == 
PIndexState.PENDING_ACTIVE
+                                     || index.getIndexState() == 
PIndexState.PENDING_DISABLE)
+                             && disableIndexTimestamp < 
minNonZerodisableIndexTimestamp) {
+                         minNonZerodisableIndexTimestamp = 
disableIndexTimestamp;
+                     }
+                 }
+                 // Freeze time for table at min non-zero value of 
INDEX_DISABLE_TIMESTAMP
+                 // This will keep the table consistent with index as the 
table has had one more
+                 // batch applied to it.
+                 if (minNonZerodisableIndexTimestamp != Long.MAX_VALUE) {
+                     // Subtract one because we add one due to timestamp 
granularity in Windows
+                     builder.setMutationTime(minNonZerodisableIndexTimestamp - 
1);
                  }
-             }
-             // Freeze time for table at min non-zero value of 
INDEX_DISABLE_TIMESTAMP
-             // This will keep the table consistent with index as the table 
has had one more
-             // batch applied to it.
-             if (minNonZerodisableIndexTimestamp == Long.MAX_VALUE) {
-                 builder.setMutationTime(currentTime);
-             } else {
-                 // Subtract one because we add one due to timestamp 
granularity in Windows
-                 builder.setMutationTime(minNonZerodisableIndexTimestamp - 1);
              }
 -
 -            if (table.getTimeStamp() != tableTimeStamp) {
 +            Pair<PTable, MetaDataProtos.MutationCode> pair = 
combineColumns(table, tenantId, schemaName, tableName, 
request.getClientTimestamp(), request.getClientVersion());
 +            table = pair.getFirst();
 +            if (table == null) {
 +                builder.setReturnCode(pair.getSecond());
 +                builder.setMutationTime(currentTime);
 +            }
 +            else if (table.getTimeStamp() != tableTimeStamp) {
                  builder.setTable(PTableImpl.toProto(table));
              }
              done.run(builder.build());
@@@ -1250,10 -1066,10 +1280,10 @@@
          return PTableImpl.makePTable(tenantId, schemaName, tableName, 
tableType, indexState, timeStamp, tableSeqNum,
                  pkName, saltBucketNum, columns, parentSchemaName, 
parentTableName, indexes, isImmutableRows, physicalTables, defaultFamilyName,
                  viewStatement, disableWAL, multiTenant, storeNulls, viewType, 
viewIndexId, indexType,
-                 rowKeyOrderOptimizable, transactional, updateCacheFrequency, 
baseColumnCount,
+                 rowKeyOrderOptimizable, transactionProvider, 
updateCacheFrequency, baseColumnCount,
                  indexDisableTimestamp, isNamespaceMapped, autoPartitionSeq, 
isAppendOnlySchema, storageScheme, encodingScheme, cqCounter, 
useStatsForParallelization);
      }
 -    
 +
      private boolean isQualifierCounterKV(Cell kv) {
          int cmp =
                  Bytes.compareTo(kv.getQualifierArray(), 
kv.getQualifierOffset(),
@@@ -2146,12 -1998,16 +2183,16 @@@
              byte[] key =
                      parentTableName == null ? lockKey : 
SchemaUtil.getTableKey(tenantIdBytes,
                          schemaName, tableName);
- 
-             
+             Region region = env.getRegion();
+             MetaDataMutationResult result = checkTableKeyInRegion(key, 
region);
+             if (result != null) {
+                 done.run(MetaDataMutationResult.toProto(result));
+                 return;
+             }
              PTableType ptableType=PTableType.fromSerializedValue(tableType);
 -            long clientTimeStamp = 
MetaDataUtil.getClientTimeStamp(tableMetadata);
 +            long clientTimeStamp = 
MetaDataUtil.getClientTimeStamp(catalogMutations);
              byte[] cKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, 
tableName);
-             PTable loadedTable = loadTable(env, cKey, new 
ImmutableBytesPtr(cKey), clientTimeStamp, clientTimeStamp,
+             PTable loadedTable = getTable(env, cKey, new 
ImmutableBytesPtr(cKey), clientTimeStamp, clientTimeStamp,
                      request.getClientVersion());
              if (loadedTable == null) {
                  
builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
@@@ -2163,14 -2019,8 +2204,7 @@@
                      SchemaUtil.getTableName(schemaName, tableName),
                      
TableName.valueOf(loadedTable.getPhysicalName().getBytes()),
                      getParentPhysicalTableName(loadedTable), 
ptableType,loadedTable.getIndexes());
- 
-             Region region = env.getRegion();
-             MetaDataMutationResult result = checkTableKeyInRegion(key, 
region);
-             if (result != null) {
-                 done.run(MetaDataMutationResult.toProto(result));
-                 return;
-             }
              List<RowLock> locks = Lists.newArrayList();
 -
              try {
                  acquireLock(region, lockKey, locks);
                  if (key != lockKey) {
@@@ -2746,82 -2490,75 +2781,58 @@@
          }
          return false;
      }
 -    
 -    private MetaDataMutationResult 
addColumnsAndTablePropertiesToChildViews(PTable basePhysicalTable, 
List<Mutation> tableMetadata, List<Mutation> mutationsForAddingColumnsToViews, 
byte[] schemaName, byte[] tableName,
 -            List<ImmutableBytesPtr> invalidateList, long clientTimeStamp, 
TableViewFinder childViewsResult,
 -            Region region, List<RowLock> locks, int clientVersion) throws 
IOException, SQLException {
 -        List<PutWithOrdinalPosition> columnPutsForBaseTable = 
Lists.newArrayListWithExpectedSize(tableMetadata.size());
 -        Map<TableProperty, Cell> tablePropertyCellMap = 
Maps.newHashMapWithExpectedSize(tableMetadata.size());
 -        // Isolate the puts relevant to adding columns. Also figure out what 
kind of columns are being added.
 +
 +
 +    private MetaDataMutationResult validateColumnForAddToBaseTable(PTable 
basePhysicalTable,
 +            List<Mutation> tableMetadata, byte[][] rowKeyMetaData,
 +            TableViewFinderResult childViewsResult, long clientTimeStamp, int 
clientVersion)
 +            throws IOException, SQLException {
 +        byte[] schemaName = rowKeyMetaData[SCHEMA_NAME_INDEX];
 +        byte[] tableName = rowKeyMetaData[TABLE_NAME_INDEX];
 +        List<PutWithOrdinalPosition> columnPutsForBaseTable =
 +                Lists.newArrayListWithExpectedSize(tableMetadata.size());
-         Map<TableProperty, Cell> tablePropertyCellMap =
-                 Maps.newHashMapWithExpectedSize(tableMetadata.size());
 +        // Isolate the puts relevant to adding columns. Also figure out what 
kind of columns are
 +        // being added.
          for (Mutation m : tableMetadata) {
              if (m instanceof Put) {
                  byte[][] rkmd = new byte[5][];
                  int pkCount = getVarChars(m.getRow(), rkmd);
                  // check if this put is for adding a column
 -                if (pkCount > COLUMN_NAME_INDEX
 -                        && rkmd[COLUMN_NAME_INDEX] != null && 
rkmd[COLUMN_NAME_INDEX].length > 0
 -                        && Bytes.compareTo(schemaName, 
rkmd[SCHEMA_NAME_INDEX]) == 0
 -                        && Bytes.compareTo(tableName, rkmd[TABLE_NAME_INDEX]) 
== 0) {
 -                    columnPutsForBaseTable.add(new 
PutWithOrdinalPosition((Put)m, getInteger((Put)m, TABLE_FAMILY_BYTES, 
ORDINAL_POSITION_BYTES)));
 -                }
 -                // check if the put is for a table property
 -                else if (pkCount <= COLUMN_NAME_INDEX
 +                if (pkCount > COLUMN_NAME_INDEX && rkmd[COLUMN_NAME_INDEX] != 
null
 +                        && rkmd[COLUMN_NAME_INDEX].length > 0
                          && Bytes.compareTo(schemaName, 
rkmd[SCHEMA_NAME_INDEX]) == 0
                          && Bytes.compareTo(tableName, rkmd[TABLE_NAME_INDEX]) 
== 0) {
 -                    for (Cell cell : 
m.getFamilyCellMap().get(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES)) {
 -                        for (TableProperty tableProp : 
TableProperty.values()) {
 -                            byte[] propNameBytes = 
Bytes.toBytes(tableProp.getPropertyName());
 -                            if (Bytes.compareTo(propNameBytes, 0, 
propNameBytes.length, cell.getQualifierArray(), cell.getQualifierOffset(), 
cell.getQualifierLength())==0
 -                                    && tableProp.isValidOnView() && 
tableProp.isMutable()) {
 -                                Cell tablePropCell = 
CellUtil.createCell(cell.getRow(), CellUtil.cloneFamily(cell),
 -                                    CellUtil.cloneQualifier(cell), 
cell.getTimestamp(), cell.getTypeByte(),
 -                                    CellUtil.cloneValue(cell));
 -                                tablePropertyCellMap.put(tableProp, 
tablePropCell);
 -                            }
 -                        }
 -                    }
 +                    columnPutsForBaseTable.add(new 
PutWithOrdinalPosition((Put) m,
 +                            getInteger((Put) m, TABLE_FAMILY_BYTES, 
ORDINAL_POSITION_BYTES)));
                  }
-                 // check if the put is for a table property
-                 else if (pkCount <= COLUMN_NAME_INDEX
-                         && Bytes.compareTo(schemaName, 
rkmd[SCHEMA_NAME_INDEX]) == 0
-                         && Bytes.compareTo(tableName, rkmd[TABLE_NAME_INDEX]) 
== 0) {
-                     for (Cell cell : m.getFamilyCellMap()
-                             .get(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES)) 
{
-                         for (TableProperty tableProp : 
TableProperty.values()) {
-                             byte[] propNameBytes = 
Bytes.toBytes(tableProp.getPropertyName());
-                             if (Bytes.compareTo(propNameBytes, 0, 
propNameBytes.length,
-                                 cell.getQualifierArray(), 
cell.getQualifierOffset(),
-                                 cell.getQualifierLength()) == 0 && 
tableProp.isValidOnView()
-                                     && tableProp.isMutable()) {
-                                 Cell tablePropCell =
-                                         CellUtil.createCell(cell.getRow(),
-                                             CellUtil.cloneFamily(cell),
-                                             CellUtil.cloneQualifier(cell), 
cell.getTimestamp(),
-                                             cell.getTypeByte(), 
CellUtil.cloneValue(cell));
-                                 tablePropertyCellMap.put(tableProp, 
tablePropCell);
-                             }
-                         }
-                     }
-                 }
              }
          }
 -        // Sort the puts by ordinal position 
 +        // Sort the puts by ordinal position
          Collections.sort(columnPutsForBaseTable);
 -        for (ViewInfo viewInfo : childViewsResult.getViewInfoList()) {
 -            short deltaNumPkColsSoFar = 0;
 -            short columnsAddedToView = 0;
 -            short columnsAddedToBaseTable = 0;
 -            byte[] tenantId = viewInfo.getTenantId();
 -            byte[] schema = viewInfo.getSchemaName();
 -            byte[] table = viewInfo.getViewName();
 -            byte[] viewKey = SchemaUtil.getTableKey(tenantId, schema, table);
 -            
 -            // lock the rows corresponding to views so that no other thread 
can modify the view meta-data
 -            RowLock viewRowLock = acquireLock(region, viewKey, locks);
 -            PTable view = doGetTable(viewKey, clientTimeStamp, viewRowLock, 
clientVersion);
 -            
 -            ColumnOrdinalPositionUpdateList ordinalPositionList = new 
ColumnOrdinalPositionUpdateList();
 +        for (TableInfo viewInfo : childViewsResult.getResults()) {
 +            byte[] viewKey = SchemaUtil.getTableKey(viewInfo.getTenantId(), 
viewInfo.getSchemaName(), viewInfo.getTableName());
 +            PTable view = doGetTable(viewKey, clientTimeStamp, clientVersion);
 +
 +            // add the new columns to the child view
              List<PColumn> viewPkCols = new ArrayList<>(view.getPKColumns());
              boolean addingExistingPkCol = false;
 -            int numCols = view.getColumns().size();
 -            // add the new columns to the child view
              for (PutWithOrdinalPosition p : columnPutsForBaseTable) {
 -                Put baseTableColumnPut = p.put;
 +                Put columnToBeAdded = p.put;
                  PColumn existingViewColumn = null;
                  byte[][] rkmd = new byte[5][];
 -                getVarChars(baseTableColumnPut.getRow(), rkmd);
 +                getVarChars(columnToBeAdded.getRow(), rkmd);
                  String columnName = Bytes.toString(rkmd[COLUMN_NAME_INDEX]);
 -                String columnFamily = rkmd[FAMILY_NAME_INDEX] == null ? null 
: Bytes.toString(rkmd[FAMILY_NAME_INDEX]);
 +                String columnFamily =
 +                        rkmd[FAMILY_NAME_INDEX] == null ? null
 +                                : Bytes.toString(rkmd[FAMILY_NAME_INDEX]);
                  try {
 -                    existingViewColumn = columnFamily == null ? 
view.getColumnForColumnName(columnName) : view.getColumnFamily(
 -                            columnFamily).getPColumnForColumnName(columnName);
 +                    existingViewColumn =
 +                            columnFamily == null ? 
view.getColumnForColumnName(columnName)
 +                                    : view.getColumnFamily(columnFamily)
 +                                            
.getPColumnForColumnName(columnName);
                  } catch (ColumnFamilyNotFoundException e) {
 -                    // ignore since it means that the column family is not 
present for the column to be added.
 +                    // ignore since it means that the column family is not 
present for the column to
 +                    // be added.
                  } catch (ColumnNotFoundException e) {
                      // ignore since it means the column is not present in the 
view
                  }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
----------------------------------------------------------------------
diff --cc 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
index b46894e,8119c6e..e088be2
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
@@@ -17112,51 -17193,52 +17202,52 @@@ public final class MetaDataProtos 
        "cheRequest\022\025\n\rclientVersion\030\001 \001(\005\"*\n\022Cle" +
        "arCacheResponse\022\024\n\014unfreedBytes\030\001 \001(\003\"*\n" +
        "\021GetVersionRequest\022\025\n\rclientVersion\030\001 \001(" +
-       "\005\"%\n\022GetVersionResponse\022\017\n\007version\030\001 
\002(\003" +
-       "\"\205\001\n\032ClearTableFromCacheRequest\022\020\n\010tenan",
-       "tId\030\001 \002(\014\022\022\n\nschemaName\030\002 
\002(\014\022\021\n\ttableNa" +
-       "me\030\003 \002(\014\022\027\n\017clientTimestamp\030\004 
\002(\003\022\025\n\rcli" +
-       "entVersion\030\005 \001(\005\"\035\n\033ClearTableFromCacheR" +
-       "esponse*\207\005\n\014MutationCode\022\030\n\024TABLE_ALREAD" +
-       
"Y_EXISTS\020\000\022\023\n\017TABLE_NOT_FOUND\020\001\022\024\n\020COLUM" +
-       "N_NOT_FOUND\020\002\022\031\n\025COLUMN_ALREADY_EXISTS\020\003" +
-       "\022\035\n\031CONCURRENT_TABLE_MUTATION\020\004\022\027\n\023TABLE" +
-       "_NOT_IN_REGION\020\005\022\025\n\021NEWER_TABLE_FOUND\020\006\022" +
-       "\034\n\030UNALLOWED_TABLE_MUTATION\020\007\022\021\n\rNO_PK_C" +
-       
"OLUMNS\020\010\022\032\n\026PARENT_TABLE_NOT_FOUND\020\t\022\033\n\027",
-       "FUNCTION_ALREADY_EXISTS\020\n\022\026\n\022FUNCTION_NO" +
-       
"T_FOUND\020\013\022\030\n\024NEWER_FUNCTION_FOUND\020\014\022\032\n\026F" +
-       "UNCTION_NOT_IN_REGION\020\r\022\031\n\025SCHEMA_ALREAD" +
-       
"Y_EXISTS\020\016\022\026\n\022NEWER_SCHEMA_FOUND\020\017\022\024\n\020SC" +
-       "HEMA_NOT_FOUND\020\020\022\030\n\024SCHEMA_NOT_IN_REGION" +
-       
"\020\021\022\032\n\026TABLES_EXIST_ON_SCHEMA\020\022\022\035\n\031UNALLO" +
-       "WED_SCHEMA_MUTATION\020\023\022%\n!AUTO_PARTITION_" +
-       "SEQUENCE_NOT_FOUND\020\024\022#\n\037CANNOT_COERCE_AU" +
-       "TO_PARTITION_ID\020\025\022\024\n\020TOO_MANY_INDEXES\020\026\022" +
-       "\020\n\014INVALID_VIEW\020\0272\345\006\n\017MetaDataService\022/\n",
-       "\010getTable\022\020.GetTableRequest\032\021.MetaDataRe" +
-       "sponse\0227\n\014getFunctions\022\024.GetFunctionsReq" +
-       "uest\032\021.MetaDataResponse\0221\n\tgetSchema\022\021.G" +
-       "etSchemaRequest\032\021.MetaDataResponse\0225\n\013cr" +
-       "eateTable\022\023.CreateTableRequest\032\021.MetaDat" +
-       "aResponse\022;\n\016createFunction\022\026.CreateFunc" +
-       "tionRequest\032\021.MetaDataResponse\0227\n\014create" +
-       "Schema\022\024.CreateSchemaRequest\032\021.MetaDataR" +
-       "esponse\0221\n\tdropTable\022\021.DropTableRequest\032" +
-       "\021.MetaDataResponse\0223\n\ndropSchema\022\022.DropS",
-       "chemaRequest\032\021.MetaDataResponse\0227\n\014dropF" +
-       "unction\022\024.DropFunctionRequest\032\021.MetaData" +
-       "Response\0221\n\taddColumn\022\021.AddColumnRequest" +
-       "\032\021.MetaDataResponse\0223\n\ndropColumn\022\022.Drop" +
-       "ColumnRequest\032\021.MetaDataResponse\022?\n\020upda" +
-       "teIndexState\022\030.UpdateIndexStateRequest\032\021" +
-       ".MetaDataResponse\0225\n\nclearCache\022\022.ClearC" +
-       "acheRequest\032\023.ClearCacheResponse\0225\n\ngetV" +
-       "ersion\022\022.GetVersionRequest\032\023.GetVersionR" +
-       "esponse\022P\n\023clearTableFromCache\022\033.ClearTa",
-       "bleFromCacheRequest\032\034.ClearTableFromCach" +
-       "eResponseBB\n(org.apache.phoenix.coproces" +
-       "sor.generatedB\016MetaDataProtosH\001\210\001\001\240\001\001"
+       "\005\"E\n\022GetVersionResponse\022\017\n\007version\030\001 
\002(\003" +
+       "\022\036\n\026systemCatalogTimestamp\030\002 
\001(\003\"\205\001\n\032Cle",
+       "arTableFromCacheRequest\022\020\n\010tenantId\030\001 \002(" +
+       "\014\022\022\n\nschemaName\030\002 
\002(\014\022\021\n\ttableName\030\003 \002(\014" +
+       "\022\027\n\017clientTimestamp\030\004 
\002(\003\022\025\n\rclientVersi" +
+       "on\030\005 \001(\005\"\035\n\033ClearTableFromCacheResponse*" +
 -      "\365\004\n\014MutationCode\022\030\n\024TABLE_ALREADY_EXISTS" +
++      "\207\005\n\014MutationCode\022\030\n\024TABLE_ALREADY_EXISTS" +
+       
"\020\000\022\023\n\017TABLE_NOT_FOUND\020\001\022\024\n\020COLUMN_NOT_FO" +
+       
"UND\020\002\022\031\n\025COLUMN_ALREADY_EXISTS\020\003\022\035\n\031CONC" +
+       "URRENT_TABLE_MUTATION\020\004\022\027\n\023TABLE_NOT_IN_" +
+       
"REGION\020\005\022\025\n\021NEWER_TABLE_FOUND\020\006\022\034\n\030UNALL" +
+       "OWED_TABLE_MUTATION\020\007\022\021\n\rNO_PK_COLUMNS\020\010",
+       "\022\032\n\026PARENT_TABLE_NOT_FOUND\020\t\022\033\n\027FUNCTION" +
+       "_ALREADY_EXISTS\020\n\022\026\n\022FUNCTION_NOT_FOUND\020" +
+       "\013\022\030\n\024NEWER_FUNCTION_FOUND\020\014\022\032\n\026FUNCTION_" 
+
+       "NOT_IN_REGION\020\r\022\031\n\025SCHEMA_ALREADY_EXISTS" +
+       
"\020\016\022\026\n\022NEWER_SCHEMA_FOUND\020\017\022\024\n\020SCHEMA_NOT" +
+       
"_FOUND\020\020\022\030\n\024SCHEMA_NOT_IN_REGION\020\021\022\032\n\026TA" +
+       "BLES_EXIST_ON_SCHEMA\020\022\022\035\n\031UNALLOWED_SCHE" +
+       "MA_MUTATION\020\023\022%\n!AUTO_PARTITION_SEQUENCE" +
+       "_NOT_FOUND\020\024\022#\n\037CANNOT_COERCE_AUTO_PARTI" +
 -      
"TION_ID\020\025\022\024\n\020TOO_MANY_INDEXES\020\0262\345\006\n\017Meta",
 -      "DataService\022/\n\010getTable\022\020.GetTableReques" +
 -      "t\032\021.MetaDataResponse\0227\n\014getFunctions\022\024.G" +
 -      "etFunctionsRequest\032\021.MetaDataResponse\0221\n" +
 -      "\tgetSchema\022\021.GetSchemaRequest\032\021.MetaData" +
 -      "Response\0225\n\013createTable\022\023.CreateTableReq" +
 -      "uest\032\021.MetaDataResponse\022;\n\016createFunctio" +
 -      "n\022\026.CreateFunctionRequest\032\021.MetaDataResp" +
 -      "onse\0227\n\014createSchema\022\024.CreateSchemaReque" +
 -      "st\032\021.MetaDataResponse\0221\n\tdropTable\022\021.Dro" +
 -      "pTableRequest\032\021.MetaDataResponse\0223\n\ndrop",
 -      "Schema\022\022.DropSchemaRequest\032\021.MetaDataRes" +
 -      "ponse\0227\n\014dropFunction\022\024.DropFunctionRequ" +
 -      "est\032\021.MetaDataResponse\0221\n\taddColumn\022\021.Ad" +
 -      "dColumnRequest\032\021.MetaDataResponse\0223\n\ndro" +
 -      "pColumn\022\022.DropColumnRequest\032\021.MetaDataRe" +
 -      "sponse\022?\n\020updateIndexState\022\030.UpdateIndex" +
 -      "StateRequest\032\021.MetaDataResponse\0225\n\nclear" +
 -      "Cache\022\022.ClearCacheRequest\032\023.ClearCacheRe" +
 -      "sponse\0225\n\ngetVersion\022\022.GetVersionRequest" +
 -      "\032\023.GetVersionResponse\022P\n\023clearTableFromC",
 -      "ache\022\033.ClearTableFromCacheRequest\032\034.Clea" +
 -      "rTableFromCacheResponseBB\n(org.apache.ph" +
 -      "oenix.coprocessor.generatedB\016MetaDataPro" +
 -      "tosH\001\210\001\001\240\001\001"
++      
"TION_ID\020\025\022\024\n\020TOO_MANY_INDEXES\020\026\022\020\n\014INVAL",
++      "ID_VIEW\020\0272\345\006\n\017MetaDataService\022/\n\010getTabl" +
++      "e\022\020.GetTableRequest\032\021.MetaDataResponse\0227" +
++      "\n\014getFunctions\022\024.GetFunctionsRequest\032\021.M" +
++      "etaDataResponse\0221\n\tgetSchema\022\021.GetSchema" +
++      "Request\032\021.MetaDataResponse\0225\n\013createTabl" +
++      "e\022\023.CreateTableRequest\032\021.MetaDataRespons" +
++      "e\022;\n\016createFunction\022\026.CreateFunctionRequ" +
++      "est\032\021.MetaDataResponse\0227\n\014createSchema\022\024" +
++      ".CreateSchemaRequest\032\021.MetaDataResponse\022" +
++      "1\n\tdropTable\022\021.DropTableRequest\032\021.MetaDa",
++      "taResponse\0223\n\ndropSchema\022\022.DropSchemaReq" +
++      "uest\032\021.MetaDataResponse\0227\n\014dropFunction\022" +
++      "\024.DropFunctionRequest\032\021.MetaDataResponse" +
++      "\0221\n\taddColumn\022\021.AddColumnRequest\032\021.MetaD" +
++      "ataResponse\0223\n\ndropColumn\022\022.DropColumnRe" +
++      "quest\032\021.MetaDataResponse\022?\n\020updateIndexS" +
++      "tate\022\030.UpdateIndexStateRequest\032\021.MetaDat" +
++      "aResponse\0225\n\nclearCache\022\022.ClearCacheRequ" +
++      "est\032\023.ClearCacheResponse\0225\n\ngetVersion\022\022" +
++      ".GetVersionRequest\032\023.GetVersionResponse\022",
++      "P\n\023clearTableFromCache\022\033.ClearTableFromC" +
++      "acheRequest\032\034.ClearTableFromCacheRespons" +
++      "eBB\n(org.apache.phoenix.coprocessor.gene" +
++      "ratedB\016MetaDataProtosH\001\210\001\001\240\001\001"
      };
      com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner 
assigner =
        new 
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
----------------------------------------------------------------------
diff --cc 
phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
index 12bfcf9,8d500e8..9b8f45e
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
@@@ -8764,39 -8677,39 +8857,40 @@@ public final class PTableProtos 
        "\001(\014\022\026\n\016viewReferenced\030\013 
\001(\010\022\022\n\nexpressio" +
        "n\030\014 \001(\t\022\026\n\016isRowTimestamp\030\r 
\001(\010\022\021\n\tisDyn" +
        "amic\030\016 \001(\010\022\034\n\024columnQualifierBytes\030\017 
\001(\014" +
 -      "\"\232\001\n\013PTableStats\022\013\n\003key\030\001 
\002(\014\022\016\n\006values\030",
 -      "\002 \003(\014\022\033\n\023guidePostsByteCount\030\003 
\001(\003\022\025\n\rke" +
 -      "yBytesCount\030\004 \001(\003\022\027\n\017guidePostsCount\030\005 
\001" +
 -      "(\005\022!\n\013pGuidePosts\030\006 
\001(\0132\014.PGuidePosts\"\255\007" +
 -      "\n\006PTable\022\027\n\017schemaNameBytes\030\001 
\002(\014\022\026\n\016tab" +
 -      "leNameBytes\030\002 \002(\014\022\036\n\ttableType\030\003 
\002(\0162\013.P" +
 -      "TableType\022\022\n\nindexState\030\004 \001(\t\022\026\n\016sequenc" +
 -      "eNumber\030\005 \002(\003\022\021\n\ttimeStamp\030\006 
\002(\003\022\023\n\013pkNa" +
 -      "meBytes\030\007 \001(\014\022\021\n\tbucketNum\030\010 
\002(\005\022\031\n\007colu" +
 -      "mns\030\t \003(\0132\010.PColumn\022\030\n\007indexes\030\n 
\003(\0132\007.P" +
 -      "Table\022\027\n\017isImmutableRows\030\013 
\002(\010\022\032\n\022dataTa",
 -      "bleNameBytes\030\r \001(\014\022\031\n\021defaultFamilyName\030" +
 -      "\016 \001(\014\022\022\n\ndisableWAL\030\017 
\002(\010\022\023\n\013multiTenant" +
 -      "\030\020 \002(\010\022\020\n\010viewType\030\021 
\001(\014\022\025\n\rviewStatemen" +
 -      "t\030\022 \001(\014\022\025\n\rphysicalNames\030\023 
\003(\014\022\020\n\010tenant" +
 -      "Id\030\024 \001(\014\022\023\n\013viewIndexId\030\025 
\001(\005\022\021\n\tindexTy" +
 -      "pe\030\026 \001(\014\022\026\n\016statsTimeStamp\030\027 
\001(\003\022\022\n\nstor" +
 -      "eNulls\030\030 \001(\010\022\027\n\017baseColumnCount\030\031 
\001(\005\022\036\n" +
 -      "\026rowKeyOrderOptimizable\030\032 \001(\010\022\025\n\rtransac" +
 -      "tional\030\033 \001(\010\022\034\n\024updateCacheFrequency\030\034 
\001" +
 -      "(\003\022\035\n\025indexDisableTimestamp\030\035 
\001(\003\022\031\n\021isN",
 -      "amespaceMapped\030\036 \001(\010\022\034\n\024autoParititonSeq" +
 -      "Name\030\037 \001(\t\022\032\n\022isAppendOnlySchema\030  
\001(\010\022\027" +
 -      "\n\017parentNameBytes\030! \001(\014\022\025\n\rstorageScheme" +
 -      "\030\" \001(\014\022\026\n\016encodingScheme\030# 
\001(\014\022,\n\021encode" +
 -      "dCQCounters\030$ \003(\0132\021.EncodedCQCounter\022\"\n\032" +
 -      "useStatsForParallelization\030% \001(\010\022\033\n\023tran" +
 -      "sactionProvider\030& \001(\005\"6\n\020EncodedCQCounte" +
 -      "r\022\021\n\tcolFamily\030\001 \002(\t\022\017\n\007counter\030\002 
\002(\005*A\n" +
 -      
"\nPTableType\022\n\n\006SYSTEM\020\000\022\010\n\004USER\020\001\022\010\n\004VIE"
 +
 -      
"W\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020\004B@\n(org.apache.p",
 -      "hoenix.coprocessor.generatedB\014PTableProt" +
 -      "osH\001\210\001\001\240\001\001"
 +      "\022\021\n\ttimestamp\030\020 \001(\003\022\026\n\007derived\030\021 
\001(\010:\005fa",
 +      "lse\"\232\001\n\013PTableStats\022\013\n\003key\030\001 
\002(\014\022\016\n\006valu" +
 +      "es\030\002 \003(\014\022\033\n\023guidePostsByteCount\030\003 
\001(\003\022\025\n" +
 +      "\rkeyBytesCount\030\004 \001(\003\022\027\n\017guidePostsCount\030" +
 +      "\005 \001(\005\022!\n\013pGuidePosts\030\006 
\001(\0132\014.PGuidePosts" +
-       "\"\220\007\n\006PTable\022\027\n\017schemaNameBytes\030\001 
\002(\014\022\026\n\016" +
++      "\"\255\007\n\006PTable\022\027\n\017schemaNameBytes\030\001 
\002(\014\022\026\n\016" +
 +      "tableNameBytes\030\002 \002(\014\022\036\n\ttableType\030\003 
\002(\0162" +
 +      "\013.PTableType\022\022\n\nindexState\030\004 
\001(\t\022\026\n\016sequ" +
 +      "enceNumber\030\005 \002(\003\022\021\n\ttimeStamp\030\006 
\002(\003\022\023\n\013p" +
 +      "kNameBytes\030\007 \001(\014\022\021\n\tbucketNum\030\010 
\002(\005\022\031\n\007c" +
 +      "olumns\030\t \003(\0132\010.PColumn\022\030\n\007indexes\030\n 
\003(\0132",
 +      "\007.PTable\022\027\n\017isImmutableRows\030\013 
\002(\010\022\032\n\022dat" +
 +      "aTableNameBytes\030\r \001(\014\022\031\n\021defaultFamilyNa" +
 +      "me\030\016 \001(\014\022\022\n\ndisableWAL\030\017 
\002(\010\022\023\n\013multiTen" +
 +      "ant\030\020 \002(\010\022\020\n\010viewType\030\021 
\001(\014\022\025\n\rviewState" +
 +      "ment\030\022 \001(\014\022\025\n\rphysicalNames\030\023 
\003(\014\022\020\n\010ten" +
 +      "antId\030\024 \001(\014\022\023\n\013viewIndexId\030\025 
\001(\005\022\021\n\tinde" +
 +      "xType\030\026 \001(\014\022\026\n\016statsTimeStamp\030\027 
\001(\003\022\022\n\ns" +
 +      "toreNulls\030\030 \001(\010\022\027\n\017baseColumnCount\030\031 
\001(\005" +
 +      "\022\036\n\026rowKeyOrderOptimizable\030\032 
\001(\010\022\025\n\rtran" +
 +      "sactional\030\033 \001(\010\022\034\n\024updateCacheFrequency\030",
 +      "\034 \001(\003\022\035\n\025indexDisableTimestamp\030\035 
\001(\003\022\031\n\021" +
 +      "isNamespaceMapped\030\036 \001(\010\022\034\n\024autoParititon" +
 +      "SeqName\030\037 \001(\t\022\032\n\022isAppendOnlySchema\030  \001(" +
 +      "\010\022\027\n\017parentNameBytes\030! 
\001(\014\022\025\n\rstorageSch" +
 +      "eme\030\" \001(\014\022\026\n\016encodingScheme\030# 
\001(\014\022,\n\021enc" +
 +      "odedCQCounters\030$ \003(\0132\021.EncodedCQCounter\022" +
-       "\"\n\032useStatsForParallelization\030% \001(\010\"6\n\020E" +
-       "ncodedCQCounter\022\021\n\tcolFamily\030\001 \002(\t\022\017\n\007co" +
-       "unter\030\002 
\002(\005*A\n\nPTableType\022\n\n\006SYSTEM\020\000\022\010\n" +
-       
"\004USER\020\001\022\010\n\004VIEW\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020\004B@",
-       "\n(org.apache.phoenix.coprocessor.generat" +
-       "edB\014PTableProtosH\001\210\001\001\240\001\001"
++      "\"\n\032useStatsForParallelization\030% \001(\010\022\033\n\023t" +
++      "ransactionProvider\030& \001(\005\"6\n\020EncodedCQCou" +
++      "nter\022\021\n\tcolFamily\030\001 \002(\t\022\017\n\007counter\030\002 
\002(\005" +
++      
"*A\n\nPTableType\022\n\n\006SYSTEM\020\000\022\010\n\004USER\020\001\022\010\n\004",
++      
"VIEW\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020\004B@\n(org.apach" 
+
++      "e.phoenix.coprocessor.generatedB\014PTableP" +
++      "rotosH\001\210\001\001\240\001\001"
      };
      com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner 
assigner =
        new 
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --cc 
phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index 3a42584,e9e209b..d29ef66
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@@ -294,9 -293,11 +293,13 @@@ public enum SQLExceptionCode 
      
      SEQUENCE_NOT_CASTABLE_TO_AUTO_PARTITION_ID_COLUMN(1086, "44A17", 
"Sequence Value not castable to auto-partition id column"),
      CANNOT_COERCE_AUTO_PARTITION_ID(1087, "44A18", "Auto-partition id cannot 
be coerced"),
 +    
      CANNOT_CREATE_INDEX_ON_MUTABLE_TABLE_WITH_ROWTIMESTAMP(1088, "44A19", 
"Cannot create an index on a mutable table that has a ROW_TIMESTAMP column."),
-     INVALID_VIEW(1089, "44A20", "View is invalid as a required column was 
dropped"),
+     UNKNOWN_TRANSACTION_PROVIDER(1089,"44A20", "Unknown TRANSACTION_PROVIDER: 
"),
+     CANNOT_START_TXN_IF_TXN_DISABLED(1091, "44A22", "Cannot start transaction 
if transactions are disabled."),
+     CANNOT_MIX_TXN_PROVIDERS(1092, "44A23", "Cannot mix transaction 
providers: "),
+     CANNOT_ALTER_TABLE_FROM_NON_TXN_TO_TXNL(1093, "44A24", "Cannot alter 
table from non transactional to transactional for "),
++    INVALID_VIEW(1094, "44A25", "View is invalid as a required column was 
dropped"),
  
      /** Sequence related */
      SEQUENCE_ALREADY_EXIST(1200, "42Z00", "Sequence already exists.", new 
Factory() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------
diff --cc 
phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index fbb5a42,d56628a..7fbaef1
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@@ -368,12 -347,25 +375,30 @@@ public class PhoenixDatabaseMetaData im
      public static final byte[] COLUMN_QUALIFIER_COUNTER_BYTES = 
Bytes.toBytes(COLUMN_QUALIFIER_COUNTER);
      public static final String USE_STATS_FOR_PARALLELIZATION = 
"USE_STATS_FOR_PARALLELIZATION";
      public static final byte[] USE_STATS_FOR_PARALLELIZATION_BYTES = 
Bytes.toBytes(USE_STATS_FOR_PARALLELIZATION);
 +    
 +    public static final String SYSTEM_CHILD_LINK_TABLE = "CHILD_LINK";
 +    public static final String SYSTEM_CHILD_LINK_NAME = 
SchemaUtil.getTableName(SYSTEM_CATALOG_SCHEMA, SYSTEM_CHILD_LINK_TABLE);
 +    public static final byte[] SYSTEM_CHILD_LINK_NAME_BYTES = 
Bytes.toBytes(SYSTEM_CHILD_LINK_NAME);
 +    public static final TableName SYSTEM_LINK_HBASE_TABLE_NAME = 
TableName.valueOf(SYSTEM_CHILD_LINK_NAME);
  
+     
+     //SYSTEM:LOG
+     public static final String SYSTEM_LOG_TABLE = "LOG";
+     public static final String QUERY_ID = "QUERY_ID";
+     public static final String USER = "USER";
+     public static final String CLIENT_IP = "CLIENT_IP";
+     public static final String QUERY = "QUERY";
+     public static final String EXPLAIN_PLAN = "EXPLAIN_PLAN";
+     public static final String TOTAL_EXECUTION_TIME = "TOTAL_EXECUTION_TIME";
+     public static final String NO_OF_RESULTS_ITERATED = 
"NO_OF_RESULTS_ITERATED";
+     public static final String QUERY_STATUS = "QUERY_STATUS";
+     public static final String EXCEPTION_TRACE = "EXCEPTION_TRACE";
+     public static final String GLOBAL_SCAN_DETAILS = "GLOBAL_SCAN_DETAILS";
+     public static final String SCAN_METRICS_JSON = "SCAN_METRICS_JSON";
+     public static final String START_TIME = "START_TIME";
+     public static final String BIND_PARAMETERS = "BIND_PARAMETERS";
+             
+     
      PhoenixDatabaseMetaData(PhoenixConnection connection) throws SQLException 
{
          this.emptyResultSet = new 
PhoenixResultSet(ResultIterator.EMPTY_ITERATOR, RowProjector.EMPTY_PROJECTOR, 
new StatementContext(new PhoenixStatement(connection), false));
          this.connection = connection;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --cc 
phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index cb47a73,25b9fb0..3e9a597
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@@ -515,9 -531,9 +531,9 @@@ public class PhoenixStatement implement
      public static final String EXPLAIN_PLAN_BYTES_ESTIMATE_COLUMN_ALIAS = 
"EST_BYTES_READ";
      private static final PColumnImpl EXPLAIN_PLAN_BYTES_ESTIMATE_COLUMN =
              new PColumnImpl(PNameFactory.newName(EXPLAIN_PLAN_BYTES_ESTIMATE),
-                     PNameFactory.newName(EXPLAIN_PLAN_FAMILY), 
PLong.INSTANCE, null, null, false, 1,
+                     PNameFactory.newName(EXPLAIN_PLAN_FAMILY), 
PLong.INSTANCE, null, null, true, 1,
                      SortOrder.getDefault(), 0, null, false, null, false, 
false,
 -                    EXPLAIN_PLAN_BYTES_ESTIMATE);
 +                    EXPLAIN_PLAN_BYTES_ESTIMATE, 0, false);
  
      private static final String EXPLAIN_PLAN_ROWS_ESTIMATE_COLUMN_NAME = 
"RowsEstimate";
      private static final byte[] EXPLAIN_PLAN_ROWS_ESTIMATE =
@@@ -525,9 -541,9 +541,9 @@@
      public static final String EXPLAIN_PLAN_ROWS_COLUMN_ALIAS = 
"EST_ROWS_READ";
      private static final PColumnImpl EXPLAIN_PLAN_ROWS_ESTIMATE_COLUMN =
              new PColumnImpl(PNameFactory.newName(EXPLAIN_PLAN_ROWS_ESTIMATE),
-                     PNameFactory.newName(EXPLAIN_PLAN_FAMILY), 
PLong.INSTANCE, null, null, false, 2,
+                     PNameFactory.newName(EXPLAIN_PLAN_FAMILY), 
PLong.INSTANCE, null, null, true, 2,
                      SortOrder.getDefault(), 0, null, false, null, false, 
false,
 -                    EXPLAIN_PLAN_ROWS_ESTIMATE);
 +                    EXPLAIN_PLAN_ROWS_ESTIMATE, 0, false);
  
      private static final String EXPLAIN_PLAN_ESTIMATE_INFO_TS_COLUMN_NAME = 
"EstimateInfoTS";
      private static final byte[] EXPLAIN_PLAN_ESTIMATE_INFO_TS =
@@@ -535,9 -551,9 +551,9 @@@
      public static final String EXPLAIN_PLAN_ESTIMATE_INFO_TS_COLUMN_ALIAS = 
"EST_INFO_TS";
      private static final PColumnImpl EXPLAIN_PLAN_ESTIMATE_INFO_TS_COLUMN =
              new 
PColumnImpl(PNameFactory.newName(EXPLAIN_PLAN_ESTIMATE_INFO_TS),
-                 PNameFactory.newName(EXPLAIN_PLAN_FAMILY), PLong.INSTANCE, 
null, null, false, 3,
+                 PNameFactory.newName(EXPLAIN_PLAN_FAMILY), PLong.INSTANCE, 
null, null, true, 3,
                  SortOrder.getDefault(), 0, null, false, null, false, false,
 -                EXPLAIN_PLAN_ESTIMATE_INFO_TS);
 +                EXPLAIN_PLAN_ESTIMATE_INFO_TS, 0, false);
  
      private static final RowProjector 
EXPLAIN_PLAN_ROW_PROJECTOR_WITH_BYTE_ROW_ESTIMATES =
              new RowProjector(Arrays

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d615dc0c/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --cc 
phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 8bd06bc,8866ced..c453e13
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@@ -169,7 -170,8 +172,8 @@@ import org.apache.phoenix.exception.Ret
  import org.apache.phoenix.exception.SQLExceptionCode;
  import org.apache.phoenix.exception.SQLExceptionInfo;
  import org.apache.phoenix.exception.UpgradeInProgressException;
 -import org.apache.phoenix.exception.UpgradeRequiredException;
  import org.apache.phoenix.exception.UpgradeNotRequiredException;
++import org.apache.phoenix.exception.UpgradeRequiredException;
  import org.apache.phoenix.execute.MutationState;
  import org.apache.phoenix.hbase.index.IndexRegionSplitPolicy;
  import org.apache.phoenix.hbase.index.Indexer;
@@@ -2594,15 -2685,14 +2687,17 @@@ public class ConnectionQueryServicesImp
              
metaConnection.createStatement().execute(QueryConstants.CREATE_STATS_TABLE_METADATA);
          } catch (TableAlreadyExistsException ignore) {}
          try {
-             
metaConnection.createStatement().execute(QueryConstants.CREATE_FUNCTION_METADATA);
+             metaConnection.createStatement().execute(getFunctionTableDDL());
+         } catch (TableAlreadyExistsException ignore) {}
+         try {
+             metaConnection.createStatement().execute(getLogTableDDL());
          } catch (TableAlreadyExistsException ignore) {}
 +        try {
 +            
metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_CHILD_LINK_METADATA);
 +        } catch (TableAlreadyExistsException e) {}
- 
          // Catch the IOException to log the error message and then bubble it 
up for the client to retry.
          try {
-             createSysMutexTable(hbaseAdmin, 
ConnectionQueryServicesImpl.this.getProps());
+             createSysMutexTableIfNotExists(hbaseAdmin);
          } catch (IOException exception) {
              logger.error("Failed to created SYSMUTEX table. Upgrade or 
migration is not possible without it. Please retry.");
              throw exception;
@@@ -2610,6 -2700,265 +2705,270 @@@
      }
  
      /**
+      * Create an entry for the SYSTEM namespace in the SYSCAT table in case 
namespace mapping is enabled and system table
+      * to system namespace mapping is also enabled. If not enabled, this 
method returns immediately without doing anything
+      * @param metaConnection
+      * @throws SQLException
+      */
+     private void 
createSchemaIfNotExistsSystemNSMappingEnabled(PhoenixConnection metaConnection) 
throws SQLException {
+         // HBase Namespace SYSTEM is assumed to be already created inside 
{@link ensureTableCreated(byte[], PTableType,
+         // Map<String, Object>, List<Pair<byte[], Map<String, Object>>>, 
byte[][], boolean, boolean, boolean)}.
+         // This statement will create an entry for the SYSTEM namespace in 
the SYSCAT table, so that GRANT/REVOKE
+         // commands can work with SYSTEM Namespace. (See PHOENIX-4227 
https://issues.apache.org/jira/browse/PHOENIX-4227)
+         if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
+           ConnectionQueryServicesImpl.this.getProps())) {
+             try {
+                 metaConnection.createStatement().execute("CREATE SCHEMA IF 
NOT EXISTS "
+                   + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA);
+             } catch (NewerSchemaAlreadyExistsException e) {
+                 // Older clients with appropriate perms may try getting a new 
connection
+                 // This results in NewerSchemaAlreadyExistsException, so we 
can safely ignore it here
+             } catch (PhoenixIOException e) {
+                 if 
(!Iterables.isEmpty(Iterables.filter(Throwables.getCausalChain(e), 
AccessDeniedException.class))) {
+                     // Ignore ADE
+                 } else {
+                     throw e;
+                 }
+             }
+         }
+     }
+ 
+     /**
+      * Upgrade the SYSCAT schema if required
+      * @param metaConnection
+      * @param currentServerSideTableTimeStamp
+      * @return Phoenix connection object
+      * @throws SQLException
+      * @throws IOException
+      * @throws TimeoutException
+      * @throws InterruptedException
+      */
+     // Available for testing
+     protected PhoenixConnection 
upgradeSystemCatalogIfRequired(PhoenixConnection metaConnection,
+       long currentServerSideTableTimeStamp) throws SQLException, IOException, 
TimeoutException, InterruptedException {
+         String columnsToAdd = "";
+         // This will occur if we have an older SYSTEM.CATALOG and we need to 
update it to
+         // include any new columns we've added.
+         if (currentServerSideTableTimeStamp < 
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0) {
+             // We know that we always need to add the STORE_NULLS column for 
4.3 release
+             columnsToAdd = addColumn(columnsToAdd, 
PhoenixDatabaseMetaData.STORE_NULLS
+               + " " + PBoolean.INSTANCE.getSqlTypeName());
+             try (HBaseAdmin admin = getAdmin()) {
+                 HTableDescriptor[] localIndexTables = admin
+                   .listTables(MetaDataUtil.LOCAL_INDEX_TABLE_PREFIX + ".*");
+                 for (HTableDescriptor table : localIndexTables) {
+                     if (table.getValue(MetaDataUtil.PARENT_TABLE_KEY) == null
+                       && 
table.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_NAME) != null) {
+                         table.setValue(MetaDataUtil.PARENT_TABLE_KEY,
+                           
MetaDataUtil.getLocalIndexUserTableName(table.getNameAsString()));
+                         // Explicitly disable, modify and enable the table to 
ensure
+                         // co-location of data and index regions. If we just 
modify the
+                         // table descriptor when online schema change enabled 
may reopen
+                         // the region in same region server instead of 
following data region.
+                         admin.disableTable(table.getTableName());
+                         admin.modifyTable(table.getTableName(), table);
+                         admin.enableTable(table.getTableName());
+                     }
+                 }
+             }
+         }
+ 
+         // If the server side schema is before 
MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0 then
+         // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too.
+         // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is 
fixed,
+         // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with 
all
+         // the column names that have been added to SYSTEM.CATALOG since 4.0.
+         if (currentServerSideTableTimeStamp < 
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
+             columnsToAdd = addColumn(columnsToAdd, 
PhoenixDatabaseMetaData.INDEX_TYPE + " "
+               + PUnsignedTinyint.INSTANCE.getSqlTypeName() + ", "
+               + PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " "
+               + PLong.INSTANCE.getSqlTypeName());
+         }
+ 
+         // If we have some new columns from 4.1-4.3 to add, add them now.
+         if (!columnsToAdd.isEmpty()) {
+             // Ugh..need to assign to another local variable to keep eclipse 
happy.
+             PhoenixConnection newMetaConnection = 
addColumnsIfNotExists(metaConnection,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0, 
columnsToAdd);
+             metaConnection = newMetaConnection;
+         }
+ 
+         if (currentServerSideTableTimeStamp < 
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0) {
+             columnsToAdd = PhoenixDatabaseMetaData.BASE_COLUMN_COUNT + " "
+               + PInteger.INSTANCE.getSqlTypeName();
+             try {
+                 metaConnection = addColumn(metaConnection,
+                   PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                   MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0, 
columnsToAdd,
+                   false);
+                 upgradeTo4_5_0(metaConnection);
+             } catch (ColumnAlreadyExistsException ignored) {
+                 /*
+                  * Upgrade to 4.5 is a slightly special case. We use the fact 
that the
+                  * column BASE_COLUMN_COUNT is already part of the meta-data 
schema as the
+                  * signal that the server side upgrade has finished or is in 
progress.
+                  */
+                 logger.debug("No need to run 4.5 upgrade");
+             }
+             Properties p = 
PropertiesUtil.deepCopy(metaConnection.getClientInfo());
+             p.remove(PhoenixRuntime.CURRENT_SCN_ATTRIB);
+             p.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
+             PhoenixConnection conn = new PhoenixConnection(
+               ConnectionQueryServicesImpl.this, metaConnection.getURL(), p,
+               metaConnection.getMetaDataCache());
+             try {
+                 List<String> tablesNeedingUpgrade = UpgradeUtil
+                   .getPhysicalTablesWithDescRowKey(conn);
+                 if (!tablesNeedingUpgrade.isEmpty()) {
+                     logger.warn("The following tables require upgrade due to 
a bug causing the row key to be incorrect for descending columns and ascending 
BINARY columns (PHOENIX-2067 and PHOENIX-2120):\n"
+                       + Joiner.on(' ').join(tablesNeedingUpgrade)
+                       + "\nTo upgrade issue the \"bin/psql.py -u\" command.");
+                 }
+                 List<String> unsupportedTables = UpgradeUtil
+                   .getPhysicalTablesWithDescVarbinaryRowKey(conn);
+                 if (!unsupportedTables.isEmpty()) {
+                     logger.warn("The following tables use an unsupported 
VARBINARY DESC construct and need to be changed:\n"
+                       + Joiner.on(' ').join(unsupportedTables));
+                 }
+             } catch (Exception ex) {
+                 logger.error(
+                   "Unable to determine tables requiring upgrade due to 
PHOENIX-2067",
+                   ex);
+             } finally {
+                 conn.close();
+             }
+         }
+         // Add these columns one at a time, each with different timestamps so 
that if folks
+         // have
+         // run the upgrade code already for a snapshot, we'll still enter 
this block (and do
+         // the
+         // parts we haven't yet done).
+         if (currentServerSideTableTimeStamp < 
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_6_0) {
+             columnsToAdd = PhoenixDatabaseMetaData.IS_ROW_TIMESTAMP + " "
+               + PBoolean.INSTANCE.getSqlTypeName();
+             metaConnection = addColumnsIfNotExists(metaConnection,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_6_0, 
columnsToAdd);
+         }
+         if (currentServerSideTableTimeStamp < 
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
+             // Drop old stats table so that new stats table is created
+             metaConnection = dropStatsTable(metaConnection,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 4);
+             metaConnection = addColumnsIfNotExists(
+               metaConnection,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 3,
+               PhoenixDatabaseMetaData.TRANSACTIONAL + " "
+                 + PBoolean.INSTANCE.getSqlTypeName());
+             metaConnection = addColumnsIfNotExists(
+               metaConnection,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 2,
+               PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " "
+                 + PLong.INSTANCE.getSqlTypeName());
+             metaConnection = setImmutableTableIndexesImmutable(metaConnection,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1);
+             metaConnection = updateSystemCatalogTimestamp(metaConnection,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
+             ConnectionQueryServicesImpl.this.removeTable(null,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
+             clearCache();
+         }
+ 
+         if (currentServerSideTableTimeStamp < 
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0) {
+             metaConnection = addColumnsIfNotExists(
+               metaConnection,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0 - 2,
+               PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED + " "
+                 + PBoolean.INSTANCE.getSqlTypeName());
+             metaConnection = addColumnsIfNotExists(
+               metaConnection,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0 - 1,
+               PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ + " "
+                 + PVarchar.INSTANCE.getSqlTypeName());
+             metaConnection = addColumnsIfNotExists(
+               metaConnection,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0,
+               PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA + " "
+                 + PBoolean.INSTANCE.getSqlTypeName());
+             metaConnection = UpgradeUtil.disableViewIndexes(metaConnection);
+             if 
(getProps().getBoolean(QueryServices.LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB,
+               QueryServicesOptions.DEFAULT_LOCAL_INDEX_CLIENT_UPGRADE)) {
+                 metaConnection = 
UpgradeUtil.upgradeLocalIndexes(metaConnection);
+             }
+             ConnectionQueryServicesImpl.this.removeTable(null,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0);
+             clearCache();
+         }
+         if (currentServerSideTableTimeStamp < 
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0) {
+             metaConnection = addColumnsIfNotExists(
+               metaConnection,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0,
+               PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH + " "
+                 + PLong.INSTANCE.getSqlTypeName());
+             ConnectionQueryServicesImpl.this.removeTable(null,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0);
+             clearCache();
+         }
+         if (currentServerSideTableTimeStamp < 
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0) {
+             metaConnection = addColumnQualifierColumn(metaConnection, 
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 - 3);
+             metaConnection = addColumnsIfNotExists(
+               metaConnection,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 - 2,
+               PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME + " "
+                 + PTinyint.INSTANCE.getSqlTypeName());
+             metaConnection = addColumnsIfNotExists(
+               metaConnection,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 - 1,
+               PhoenixDatabaseMetaData.ENCODING_SCHEME + " "
+                 + PTinyint.INSTANCE.getSqlTypeName());
+             metaConnection = addColumnsIfNotExists(
+               metaConnection,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0,
+               PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER + " "
+                 + PInteger.INSTANCE.getSqlTypeName());
+             ConnectionQueryServicesImpl.this.removeTable(null,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0);
+             clearCache();
+         }
+         if (currentServerSideTableTimeStamp < 
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0) {
+             metaConnection = addColumnsIfNotExists(
+               metaConnection,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0,
+               PhoenixDatabaseMetaData.USE_STATS_FOR_PARALLELIZATION + " "
+                 + PBoolean.INSTANCE.getSqlTypeName());
+             addParentToChildLinks(metaConnection);
+         }
+         if (currentServerSideTableTimeStamp < 
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0) {
+             metaConnection = addColumnsIfNotExists(
+               metaConnection,
+               PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0,
+               PhoenixDatabaseMetaData.TRANSACTION_PROVIDER + " "
+                 + PTinyint.INSTANCE.getSqlTypeName());
+         }
++        // TODO set the version for which the following upgrade code runs 
correct
++        if (currentServerSideTableTimeStamp < 
MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0) {
++            addViewIndexToParentLinks(metaConnection);
++            moveChildLinks(metaConnection);
++        }
+         return metaConnection;
+     }
+ 
+     /**
       * There is no other locking needed here since only one connection (on 
the same or different JVM) will be able to
       * acquire the upgrade mutex via {@link #acquireUpgradeMutex(long, 
byte[])}.
       */
@@@ -2957,11 -3113,17 +3123,20 @@@
                  }
              }
              try {
-                 
metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_FUNCTION_METADATA);
+                 
metaConnection.createStatement().executeUpdate(getFunctionTableDDL());
+             } catch (NewerTableAlreadyExistsException e) {} catch 
(TableAlreadyExistsException e) {}
+             try {
+                 
metaConnection.createStatement().executeUpdate(getLogTableDDL());
              } catch (NewerTableAlreadyExistsException e) {} catch 
(TableAlreadyExistsException e) {}
 +            try {
 +                
metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_CHILD_LINK_METADATA);
 +            } catch (NewerTableAlreadyExistsException e) {} catch 
(TableAlreadyExistsException e) {}
+ 
+             // In case namespace mapping is enabled and system table to 
system namespace mapping is also enabled,
+             // create an entry for the SYSTEM namespace in the SYSCAT table, 
so that GRANT/REVOKE commands can work
+             // with SYSTEM Namespace
+             createSchemaIfNotExistsSystemNSMappingEnabled(metaConnection);
+ 
              ConnectionQueryServicesImpl.this.upgradeRequired.set(false);
              success = true;
          } catch (UpgradeInProgressException | UpgradeNotRequiredException e) {

Reply via email to