[1/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.3 73aca1e5e -> 7f37966c1


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7f37966c/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
deleted file mode 100644
index c191d8d..000
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
+++ /dev/null
@@ -1,350 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.phoenix.transaction;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Row;
-import org.apache.hadoop.hbase.client.RowMutations;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.Call;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
-import org.apache.tephra.TxConstants;
-import org.apache.tephra.hbase.TransactionAwareHTable;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableType;
-
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
-public class TephraTransactionTable implements PhoenixTransactionalTable {
-
-private TransactionAwareHTable transactionAwareHTable;
-
-private TephraTransactionContext tephraTransactionContext;
-
-public TephraTransactionTable(PhoenixTransactionContext ctx, 
HTableInterface hTable) {
-this(ctx, hTable, null);
-}
-
-public TephraTransactionTable(PhoenixTransactionContext ctx, 
HTableInterface hTable, PTable pTable) {
-
-assert(ctx instanceof TephraTransactionContext);
-
-tephraTransactionContext = (TephraTransactionContext) ctx;
-
-transactionAwareHTable = new TransactionAwareHTable(hTable, (pTable != 
null && pTable.isImmutableRows()) ? TxConstants.ConflictDetection.NONE : 
TxConstants.ConflictDetection.ROW);
-
-tephraTransactionContext.addTransactionAware(transactionAwareHTable);
-
-if (pTable != null && pTable.getType() != PTableType.INDEX) {
-tephraTransactionContext.markDMLFence(pTable);
-}
-}
-
-@Override
-public Result get(Get get) throws IOException {
-return transactionAwareHTable.get(get);
-}
-
-@Override
-public void put(Put put) throws IOException {
-transactionAwareHTable.put(put);
-}
-
-@Override
-public void delete(Delete delete) throws IOException {
-transactionAwareHTable.delete(delete);
-}
-
-@Override
-public ResultScanner getScanner(Scan scan) throws IOException {
-return transactionAwareHTable.getScanner(scan);
-}
-
-@Override
-public byte[] getTableName() {
-return transactionAwareHTable.getTableName();
-}
-
-@Override
-public Configuration getConfiguration() {
-return transactionAwareHTable.getConfiguration();
-}
-
-@Override
-public HTableDescriptor getTableDescriptor() throws IOException {
-return transactionAwareHTable.getTableDescriptor();
-}
-
-@Override
-public boolean exists(Get get) throws IOException {
-return transactionAwareHTable.exist

[3/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
http://git-wip-us.apache.org/repos/asf/phoenix/blob/7f37966c/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index c5065e0..59c10ad 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -19,120 +19,7 @@ package org.apache.phoenix.expression;
 
 import java.util.Map;
 
-import org.apache.phoenix.expression.function.AbsFunction;
-import org.apache.phoenix.expression.function.ArrayAllComparisonExpression;
-import org.apache.phoenix.expression.function.ArrayAnyComparisonExpression;
-import org.apache.phoenix.expression.function.ArrayAppendFunction;
-import org.apache.phoenix.expression.function.ArrayConcatFunction;
-import org.apache.phoenix.expression.function.ArrayElemRefExpression;
-import org.apache.phoenix.expression.function.ArrayFillFunction;
-import org.apache.phoenix.expression.function.ArrayIndexFunction;
-import org.apache.phoenix.expression.function.ArrayLengthFunction;
-import org.apache.phoenix.expression.function.ArrayPrependFunction;
-import org.apache.phoenix.expression.function.ArrayRemoveFunction;
-import org.apache.phoenix.expression.function.ArrayToStringFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpReplaceFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpSplitFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpSubstrFunction;
-import org.apache.phoenix.expression.function.CbrtFunction;
-import org.apache.phoenix.expression.function.CeilDateExpression;
-import org.apache.phoenix.expression.function.CeilDecimalExpression;
-import org.apache.phoenix.expression.function.CeilFunction;
-import org.apache.phoenix.expression.function.CeilMonthExpression;
-import org.apache.phoenix.expression.function.CeilTimestampExpression;
-import org.apache.phoenix.expression.function.CeilWeekExpression;
-import org.apache.phoenix.expression.function.CeilYearExpression;
-import org.apache.phoenix.expression.function.CoalesceFunction;
-import org.apache.phoenix.expression.function.CollationKeyFunction;
-import org.apache.phoenix.expression.function.ConvertTimezoneFunction;
-import org.apache.phoenix.expression.function.CountAggregateFunction;
-import org.apache.phoenix.expression.function.DayOfMonthFunction;
-import org.apache.phoenix.expression.function.DayOfWeekFunction;
-import org.apache.phoenix.expression.function.DayOfYearFunction;
-import org.apache.phoenix.expression.function.DecodeFunction;
-import org.apache.phoenix.expression.function.DefaultValueExpression;
-import org.apache.phoenix.expression.function.DistinctCountAggregateFunction;
-import 
org.apache.phoenix.expression.function.DistinctCountHyperLogLogAggregateFunction;
-import org.apache.phoenix.expression.function.DistinctValueAggregateFunction;
-import org.apache.phoenix.expression.function.EncodeFunction;
-import org.apache.phoenix.expression.function.ExpFunction;
-import org.apache.phoenix.expression.function.ExternalSqlTypeIdFunction;
-import org.apache.phoenix.expression.function.FirstValueFunction;
-import org.apache.phoenix.expression.function.FirstValuesFunction;
-import org.apache.phoenix.expression.function.FloorDateExpression;
-import org.apache.phoenix.expression.function.FloorDecimalExpression;
-import org.apache.phoenix.expression.function.FloorFunction;
-import org.apache.phoenix.expression.function.FloorMonthExpression;
-import org.apache.phoenix.expression.function.FloorWeekExpression;
-import org.apache.phoenix.expression.function.FloorYearExpression;
-import org.apache.phoenix.expression.function.GetBitFunction;
-import org.apache.phoenix.expression.function.GetByteFunction;
-import org.apache.phoenix.expression.function.HourFunction;
-import org.apache.phoenix.expression.function.IndexStateNameFunction;
-import org.apache.phoenix.expression.function.InstrFunction;
-import org.apache.phoenix.expression.function.InvertFunction;
-import org.apache.phoenix.expression.function.LTrimFunction;
-import org.apache.phoenix.expression.function.LastValueFunction;
-import org.apache.phoenix.expression.function.LastValuesFunction;
-import org.apache.phoenix.expression.function.LengthFunction;
-import org.apache.phoenix.expression.function.LnFunction;
-import org.apache.phoenix.expression.function.LogFunction;
-import org.apache.phoenix.expression.function.LowerFunction;
-import org.apache.phoenix.expression.function.LpadFunction;
-import org.apache.phoenix.expression.function.MD5Function;
-import org.apache.phoenix.expression.function.MaxAggregateFunction;
-import org.apache.phoenix.expression.function.MinAggregateFunction;
-import org.apache.phoenix.expression.function.MinuteFunctio

[4/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
PHOENIX-4605 Support running multiple transaction providers


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

Branch: refs/heads/4.x-HBase-1.3
Commit: 7f37966c19c0011f60c2d9b4e6889d5da27a901a
Parents: 73aca1e
Author: James Taylor 
Authored: Wed Apr 11 20:06:35 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 12 12:40:05 2018 -0700

--
 .../phoenix/end2end/AlterTableWithViewsIT.java  |   6 +-
 .../ConnectionQueryServicesTestImpl.java|  34 +-
 .../phoenix/tx/FlappingTransactionIT.java   |  11 +-
 .../phoenix/tx/ParameterizedTransactionIT.java  |  14 +-
 .../org/apache/phoenix/tx/TransactionIT.java|  12 +
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |   5 +-
 .../phoenix/cache/IndexMetaDataCache.java   |   5 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |   3 +-
 .../apache/phoenix/compile/FromCompiler.java|   2 +-
 .../apache/phoenix/compile/JoinCompiler.java|   2 +-
 .../compile/TupleProjectionCompiler.java|   4 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   6 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   2 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   4 +-
 .../coprocessor/MetaDataEndpointImpl.java   |  26 +-
 .../phoenix/coprocessor/MetaDataProtocol.java   |   2 +-
 .../coprocessor/MetaDataRegionObserver.java |   4 +-
 .../PhoenixTransactionalProcessor.java  |  28 --
 .../coprocessor/ServerCachingEndpointImpl.java  |   4 +-
 .../TephraTransactionalProcessor.java   |  29 ++
 .../UngroupedAggregateRegionObserver.java   |  10 +-
 .../coprocessor/generated/PTableProtos.java | 110 +-
 .../phoenix/exception/SQLExceptionCode.java |   4 +
 .../apache/phoenix/execute/BaseQueryPlan.java   |   3 +
 .../apache/phoenix/execute/MutationState.java   |  74 ++--
 .../PhoenixTxIndexMutationGenerator.java|  10 +-
 .../phoenix/expression/ExpressionType.java  | 119 +-
 .../TransactionProviderNameFunction.java|  81 +
 .../apache/phoenix/index/IndexMaintainer.java   |   4 +-
 .../index/IndexMetaDataCacheFactory.java|   2 +-
 .../apache/phoenix/index/PhoenixIndexCodec.java |   1 -
 .../index/PhoenixIndexMetaDataBuilder.java  |   7 +-
 .../NonAggregateRegionScannerFactory.java   |   5 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  12 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   2 +-
 .../index/PhoenixIndexPartialBuildMapper.java   |   4 +-
 .../phoenix/query/ConnectionQueryServices.java  |   4 +
 .../query/ConnectionQueryServicesImpl.java  |  69 ++--
 .../query/ConnectionlessQueryServicesImpl.java  |   9 +-
 .../query/DelegateConnectionQueryServices.java  |   7 +-
 .../apache/phoenix/query/QueryConstants.java|   2 +
 .../org/apache/phoenix/query/QueryServices.java |   1 +
 .../phoenix/query/QueryServicesOptions.java |  10 +-
 .../apache/phoenix/schema/DelegateTable.java|   8 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 129 +--
 .../java/org/apache/phoenix/schema/PTable.java  |   2 +
 .../org/apache/phoenix/schema/PTableImpl.java   |  68 ++--
 .../apache/phoenix/schema/TableProperty.java|  18 +
 .../transaction/OmidTransactionContext.java |  57 +--
 .../transaction/OmidTransactionProvider.java|  54 +--
 .../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  | 161 +++-
 .../transaction/TephraTransactionTable.java | 350 --
 .../phoenix/transaction/TransactionFactory.java |  57 ++-
 .../transaction/TransactionProvider.java|  36 --
 .../org/apache/phoenix/util/PhoenixRuntime.java |   3 +-
 .../java/org/apache/phoenix/util/ScanUtil.java  |  15 +
 .../apache/phoenix/util/TransactionUtil.java|  49 ++-
 .../phoenix/execute/CorrelatePlanTest.java  |   5 +-
 .../execute/LiteralResultIteratorPlanTest.java  |   5 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  26 +-
 .../phoenix/query/QueryServicesTestImpl.java|  15 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |   2 +-
 phoenix-protocol/src/main/PTable.proto  |   1 +
 70 files changed, 1176 insertions(+), 1623 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7f37966c/phoenix-core/src/it/java/org/apache/phoen

[1/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/master ca58fae20 -> de83b8d5d


http://git-wip-us.apache.org/repos/asf/phoenix/blob/de83b8d5/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
deleted file mode 100644
index 55785be..000
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
+++ /dev/null
@@ -1,366 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.phoenix.transaction;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Row;
-import org.apache.hadoop.hbase.client.RowMutations;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.Call;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
-import org.apache.tephra.TxConstants;
-import org.apache.tephra.hbase.TransactionAwareHTable;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableType;
-
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
-public class TephraTransactionTable implements PhoenixTransactionalTable {
-
-private TransactionAwareHTable transactionAwareHTable;
-
-private TephraTransactionContext tephraTransactionContext;
-
-public TephraTransactionTable(PhoenixTransactionContext ctx, 
HTableInterface hTable) {
-this(ctx, hTable, null);
-}
-
-public TephraTransactionTable(PhoenixTransactionContext ctx, 
HTableInterface hTable, PTable pTable) {
-
-assert(ctx instanceof TephraTransactionContext);
-
-tephraTransactionContext = (TephraTransactionContext) ctx;
-
-transactionAwareHTable = new TransactionAwareHTable(hTable, (pTable != 
null && pTable.isImmutableRows()) ? TxConstants.ConflictDetection.NONE : 
TxConstants.ConflictDetection.ROW);
-
-tephraTransactionContext.addTransactionAware(transactionAwareHTable);
-
-if (pTable != null && pTable.getType() != PTableType.INDEX) {
-tephraTransactionContext.markDMLFence(pTable);
-}
-}
-
-@Override
-public Result get(Get get) throws IOException {
-return transactionAwareHTable.get(get);
-}
-
-@Override
-public void put(Put put) throws IOException {
-transactionAwareHTable.put(put);
-}
-
-@Override
-public void delete(Delete delete) throws IOException {
-transactionAwareHTable.delete(delete);
-}
-
-@Override
-public ResultScanner getScanner(Scan scan) throws IOException {
-return transactionAwareHTable.getScanner(scan);
-}
-
-@Override
-public byte[] getTableName() {
-return transactionAwareHTable.getTableName();
-}
-
-@Override
-public Configuration getConfiguration() {
-return transactionAwareHTable.getConfiguration();
-}
-
-@Override
-public HTableDescriptor getTableDescriptor() throws IOException {
-return transactionAwareHTable.getTableDescriptor();
-}
-
-@Override
-public boolean exists(Get get) throws IOException {
-return transactionAwareHTable.exists(get);

[2/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
http://git-wip-us.apache.org/repos/asf/phoenix/blob/de83b8d5/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index a7b31e8..1a11427 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -69,6 +69,7 @@ import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.types.PFloat;
 import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -134,7 +135,7 @@ public class PTableImpl implements PTable {
 private boolean disableWAL;
 private boolean multiTenant;
 private boolean storeNulls;
-private boolean isTransactional;
+private TransactionFactory.Provider transactionProvider;
 private ViewType viewType;
 private Short viewIndexId;
 private int estimatedSize;
@@ -227,7 +228,7 @@ public class PTableImpl implements PTable {
 init(tenantId, this.schemaName, this.tableName, PTableType.INDEX, 
state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
 this.schemaName, parentTableName, indexes, isImmutableRows, 
physicalNames, defaultFamilyName,
 null, disableWAL, multiTenant, storeNulls, viewType, viewIndexId, 
indexType, baseColumnCount, rowKeyOrderOptimizable,
-isTransactional, updateCacheFrequency, indexDisableTimestamp, 
isNamespaceMpped, null, false, storageScheme, qualifierEncodingScheme, 
encodedCQCounter, useStatsForParallelization);
+transactionProvider, updateCacheFrequency, indexDisableTimestamp, 
isNamespaceMpped, null, false, storageScheme, qualifierEncodingScheme, 
encodedCQCounter, useStatsForParallelization);
 }
 
 public PTableImpl(long timeStamp) { // For delete marker
@@ -270,7 +271,7 @@ public class PTableImpl implements PTable {
 table.getSequenceNumber(), table.getPKName(), 
table.getBucketNum(), getColumnsToClone(table), table.getParentSchemaName(), 
table.getParentTableName(),
 indexes, table.isImmutableRows(), physicalNames, 
table.getDefaultFamilyName(), viewStatement,
 table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), 
table.getIndexType(),
-table.getBaseColumnCount(), 
table.rowKeyOrderOptimizable(), table.isTransactional(), updateCacheFrequency,
+table.getBaseColumnCount(), 
table.rowKeyOrderOptimizable(), table.getTransactionProvider(), 
updateCacheFrequency,
 table.getIndexDisableTimestamp(), 
table.isNamespaceMapped(), table.getAutoPartitionSeqName(), 
table.isAppendOnlySchema(), table.getImmutableStorageScheme(), 
table.getEncodingScheme(), table.getEncodedCQCounter(), 
table.useStatsForParallelization());
 }
 
@@ -280,7 +281,7 @@ public class PTableImpl implements PTable {
 table.getSequenceNumber(), table.getPKName(), 
table.getBucketNum(), getColumnsToClone(table), parentSchemaName, 
table.getParentTableName(),
 indexes, table.isImmutableRows(), table.getPhysicalNames(), 
table.getDefaultFamilyName(), viewStatement,
 table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), 
table.getIndexType(),
-table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), 
table.isTransactional(), table.getUpdateCacheFrequency(),
+table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), 
table.getTransactionProvider(), table.getUpdateCacheFrequency(),
 table.getIndexDisableTimestamp(), table.isNamespaceMapped(), 
table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), 
table.getImmutableStorageScheme(), table.getEncodingScheme(), 
table.getEncodedCQCounter(), table.useStatsForParallelization());
 }
 
@@ -290,7 +291,7 @@ public class PTableImpl implements PTable {
 table.getSequenceNumber(), table.getPKName(), 
table.getBucketNum(), columns, table.getParentSchemaName(), 
table.getParentTableName(),
 table.getIndexes(), table.isImmutableRows(), 
table.getPhysicalNames(), table.getDefaultFamilyName(), 
table.getViewStatement(),
 table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), 
table.getIndexType(),
-table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), 
table.isTransactional(), table.getUpdateCacheFrequency(),

[4/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
PHOENIX-4605 Support running multiple transaction providers


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

Branch: refs/heads/master
Commit: de83b8d5d042098faa294e742525ab84175bb271
Parents: ca58fae
Author: James Taylor 
Authored: Wed Apr 11 20:06:35 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 12 17:19:42 2018 -0700

--
 .../phoenix/end2end/AlterTableWithViewsIT.java  |   6 +-
 .../ConnectionQueryServicesTestImpl.java|  34 +-
 .../phoenix/tx/FlappingTransactionIT.java   |  11 +-
 .../phoenix/tx/ParameterizedTransactionIT.java  |  14 +-
 .../org/apache/phoenix/tx/TransactionIT.java|  12 +
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |   5 +-
 .../phoenix/cache/IndexMetaDataCache.java   |   5 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |   3 +-
 .../apache/phoenix/compile/FromCompiler.java|   2 +-
 .../apache/phoenix/compile/JoinCompiler.java|   2 +-
 .../compile/TupleProjectionCompiler.java|   4 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   6 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   2 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   4 +-
 .../coprocessor/MetaDataEndpointImpl.java   |  26 +-
 .../phoenix/coprocessor/MetaDataProtocol.java   |   2 +-
 .../coprocessor/MetaDataRegionObserver.java |   4 +-
 .../PhoenixTransactionalProcessor.java  |  28 --
 .../coprocessor/ServerCachingEndpointImpl.java  |   4 +-
 .../TephraTransactionalProcessor.java   |  29 ++
 .../UngroupedAggregateRegionObserver.java   |  10 +-
 .../coprocessor/generated/PTableProtos.java | 110 +-
 .../phoenix/exception/SQLExceptionCode.java |   4 +
 .../apache/phoenix/execute/BaseQueryPlan.java   |   3 +
 .../apache/phoenix/execute/MutationState.java   |  74 ++--
 .../PhoenixTxIndexMutationGenerator.java|  10 +-
 .../phoenix/expression/ExpressionType.java  | 119 +-
 .../TransactionProviderNameFunction.java|  81 
 .../apache/phoenix/index/IndexMaintainer.java   |   4 +-
 .../index/IndexMetaDataCacheFactory.java|   2 +-
 .../apache/phoenix/index/PhoenixIndexCodec.java |   1 -
 .../index/PhoenixIndexMetaDataBuilder.java  |   7 +-
 .../NonAggregateRegionScannerFactory.java   |   5 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  12 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   2 +-
 .../index/PhoenixIndexPartialBuildMapper.java   |   4 +-
 .../phoenix/query/ConnectionQueryServices.java  |   4 +
 .../query/ConnectionQueryServicesImpl.java  |  69 ++--
 .../query/ConnectionlessQueryServicesImpl.java  |   9 +-
 .../query/DelegateConnectionQueryServices.java  |   7 +-
 .../apache/phoenix/query/QueryConstants.java|   2 +
 .../org/apache/phoenix/query/QueryServices.java |   1 +
 .../phoenix/query/QueryServicesOptions.java |  10 +-
 .../apache/phoenix/schema/DelegateTable.java|   8 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 129 --
 .../java/org/apache/phoenix/schema/PTable.java  |   2 +
 .../org/apache/phoenix/schema/PTableImpl.java   |  68 ++--
 .../apache/phoenix/schema/TableProperty.java|  18 +
 .../transaction/OmidTransactionContext.java |  57 +--
 .../transaction/OmidTransactionProvider.java|  54 +--
 .../transaction/OmidTransactionTable.java   | 388 ---
 .../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  | 161 +++-
 .../transaction/TephraTransactionTable.java | 366 -
 .../phoenix/transaction/TransactionFactory.java |  57 ++-
 .../transaction/TransactionProvider.java|  36 --
 .../org/apache/phoenix/util/PhoenixRuntime.java |   3 +-
 .../java/org/apache/phoenix/util/ScanUtil.java  |  15 +
 .../apache/phoenix/util/TransactionUtil.java|  49 ++-
 .../phoenix/execute/CorrelatePlanTest.java  |   5 +-
 .../execute/LiteralResultIteratorPlanTest.java  |   5 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  26 +-
 .../phoenix/query/QueryServicesTestImpl.java|  15 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |   2 +-
 phoenix-protocol/src/main/PTable.proto  |   1 +
 70 files changed, 1176 insertions(+), 1663 deletions(-)
--


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

[3/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
http://git-wip-us.apache.org/repos/asf/phoenix/blob/de83b8d5/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index c5065e0..59c10ad 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -19,120 +19,7 @@ package org.apache.phoenix.expression;
 
 import java.util.Map;
 
-import org.apache.phoenix.expression.function.AbsFunction;
-import org.apache.phoenix.expression.function.ArrayAllComparisonExpression;
-import org.apache.phoenix.expression.function.ArrayAnyComparisonExpression;
-import org.apache.phoenix.expression.function.ArrayAppendFunction;
-import org.apache.phoenix.expression.function.ArrayConcatFunction;
-import org.apache.phoenix.expression.function.ArrayElemRefExpression;
-import org.apache.phoenix.expression.function.ArrayFillFunction;
-import org.apache.phoenix.expression.function.ArrayIndexFunction;
-import org.apache.phoenix.expression.function.ArrayLengthFunction;
-import org.apache.phoenix.expression.function.ArrayPrependFunction;
-import org.apache.phoenix.expression.function.ArrayRemoveFunction;
-import org.apache.phoenix.expression.function.ArrayToStringFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpReplaceFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpSplitFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpSubstrFunction;
-import org.apache.phoenix.expression.function.CbrtFunction;
-import org.apache.phoenix.expression.function.CeilDateExpression;
-import org.apache.phoenix.expression.function.CeilDecimalExpression;
-import org.apache.phoenix.expression.function.CeilFunction;
-import org.apache.phoenix.expression.function.CeilMonthExpression;
-import org.apache.phoenix.expression.function.CeilTimestampExpression;
-import org.apache.phoenix.expression.function.CeilWeekExpression;
-import org.apache.phoenix.expression.function.CeilYearExpression;
-import org.apache.phoenix.expression.function.CoalesceFunction;
-import org.apache.phoenix.expression.function.CollationKeyFunction;
-import org.apache.phoenix.expression.function.ConvertTimezoneFunction;
-import org.apache.phoenix.expression.function.CountAggregateFunction;
-import org.apache.phoenix.expression.function.DayOfMonthFunction;
-import org.apache.phoenix.expression.function.DayOfWeekFunction;
-import org.apache.phoenix.expression.function.DayOfYearFunction;
-import org.apache.phoenix.expression.function.DecodeFunction;
-import org.apache.phoenix.expression.function.DefaultValueExpression;
-import org.apache.phoenix.expression.function.DistinctCountAggregateFunction;
-import 
org.apache.phoenix.expression.function.DistinctCountHyperLogLogAggregateFunction;
-import org.apache.phoenix.expression.function.DistinctValueAggregateFunction;
-import org.apache.phoenix.expression.function.EncodeFunction;
-import org.apache.phoenix.expression.function.ExpFunction;
-import org.apache.phoenix.expression.function.ExternalSqlTypeIdFunction;
-import org.apache.phoenix.expression.function.FirstValueFunction;
-import org.apache.phoenix.expression.function.FirstValuesFunction;
-import org.apache.phoenix.expression.function.FloorDateExpression;
-import org.apache.phoenix.expression.function.FloorDecimalExpression;
-import org.apache.phoenix.expression.function.FloorFunction;
-import org.apache.phoenix.expression.function.FloorMonthExpression;
-import org.apache.phoenix.expression.function.FloorWeekExpression;
-import org.apache.phoenix.expression.function.FloorYearExpression;
-import org.apache.phoenix.expression.function.GetBitFunction;
-import org.apache.phoenix.expression.function.GetByteFunction;
-import org.apache.phoenix.expression.function.HourFunction;
-import org.apache.phoenix.expression.function.IndexStateNameFunction;
-import org.apache.phoenix.expression.function.InstrFunction;
-import org.apache.phoenix.expression.function.InvertFunction;
-import org.apache.phoenix.expression.function.LTrimFunction;
-import org.apache.phoenix.expression.function.LastValueFunction;
-import org.apache.phoenix.expression.function.LastValuesFunction;
-import org.apache.phoenix.expression.function.LengthFunction;
-import org.apache.phoenix.expression.function.LnFunction;
-import org.apache.phoenix.expression.function.LogFunction;
-import org.apache.phoenix.expression.function.LowerFunction;
-import org.apache.phoenix.expression.function.LpadFunction;
-import org.apache.phoenix.expression.function.MD5Function;
-import org.apache.phoenix.expression.function.MaxAggregateFunction;
-import org.apache.phoenix.expression.function.MinAggregateFunction;
-import org.apache.phoenix.expression.function.MinuteFunctio

[1/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.2 5d6b39cde -> 6d9140ed6


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6d9140ed/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
deleted file mode 100644
index cf48521..000
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
+++ /dev/null
@@ -1,330 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.phoenix.transaction;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Row;
-import org.apache.hadoop.hbase.client.RowMutations;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.Call;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
-import org.apache.tephra.TxConstants;
-import org.apache.tephra.hbase.TransactionAwareHTable;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableType;
-
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
-public class TephraTransactionTable implements PhoenixTransactionalTable {
-
-private TransactionAwareHTable transactionAwareHTable;
-
-private TephraTransactionContext tephraTransactionContext;
-
-public TephraTransactionTable(PhoenixTransactionContext ctx, 
HTableInterface hTable) {
-this(ctx, hTable, null);
-}
-
-public TephraTransactionTable(PhoenixTransactionContext ctx, 
HTableInterface hTable, PTable pTable) {
-
-assert(ctx instanceof TephraTransactionContext);
-
-tephraTransactionContext = (TephraTransactionContext) ctx;
-
-transactionAwareHTable = new TransactionAwareHTable(hTable, (pTable != 
null && pTable.isImmutableRows()) ? TxConstants.ConflictDetection.NONE : 
TxConstants.ConflictDetection.ROW);
-
-tephraTransactionContext.addTransactionAware(transactionAwareHTable);
-
-if (pTable != null && pTable.getType() != PTableType.INDEX) {
-tephraTransactionContext.markDMLFence(pTable);
-}
-}
-
-@Override
-public Result get(Get get) throws IOException {
-return transactionAwareHTable.get(get);
-}
-
-@Override
-public void put(Put put) throws IOException {
-transactionAwareHTable.put(put);
-}
-
-@Override
-public void delete(Delete delete) throws IOException {
-transactionAwareHTable.delete(delete);
-}
-
-@Override
-public ResultScanner getScanner(Scan scan) throws IOException {
-return transactionAwareHTable.getScanner(scan);
-}
-
-@Override
-public byte[] getTableName() {
-return transactionAwareHTable.getTableName();
-}
-
-@Override
-public Configuration getConfiguration() {
-return transactionAwareHTable.getConfiguration();
-}
-
-@Override
-public HTableDescriptor getTableDescriptor() throws IOException {
-return transactionAwareHTable.getTableDescriptor();
-}
-
-@Override
-public boolean exists(Get get) throws IOException {
-return transactionAwareHTable.exist

[3/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
http://git-wip-us.apache.org/repos/asf/phoenix/blob/6d9140ed/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index c5065e0..59c10ad 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -19,120 +19,7 @@ package org.apache.phoenix.expression;
 
 import java.util.Map;
 
-import org.apache.phoenix.expression.function.AbsFunction;
-import org.apache.phoenix.expression.function.ArrayAllComparisonExpression;
-import org.apache.phoenix.expression.function.ArrayAnyComparisonExpression;
-import org.apache.phoenix.expression.function.ArrayAppendFunction;
-import org.apache.phoenix.expression.function.ArrayConcatFunction;
-import org.apache.phoenix.expression.function.ArrayElemRefExpression;
-import org.apache.phoenix.expression.function.ArrayFillFunction;
-import org.apache.phoenix.expression.function.ArrayIndexFunction;
-import org.apache.phoenix.expression.function.ArrayLengthFunction;
-import org.apache.phoenix.expression.function.ArrayPrependFunction;
-import org.apache.phoenix.expression.function.ArrayRemoveFunction;
-import org.apache.phoenix.expression.function.ArrayToStringFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpReplaceFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpSplitFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpSubstrFunction;
-import org.apache.phoenix.expression.function.CbrtFunction;
-import org.apache.phoenix.expression.function.CeilDateExpression;
-import org.apache.phoenix.expression.function.CeilDecimalExpression;
-import org.apache.phoenix.expression.function.CeilFunction;
-import org.apache.phoenix.expression.function.CeilMonthExpression;
-import org.apache.phoenix.expression.function.CeilTimestampExpression;
-import org.apache.phoenix.expression.function.CeilWeekExpression;
-import org.apache.phoenix.expression.function.CeilYearExpression;
-import org.apache.phoenix.expression.function.CoalesceFunction;
-import org.apache.phoenix.expression.function.CollationKeyFunction;
-import org.apache.phoenix.expression.function.ConvertTimezoneFunction;
-import org.apache.phoenix.expression.function.CountAggregateFunction;
-import org.apache.phoenix.expression.function.DayOfMonthFunction;
-import org.apache.phoenix.expression.function.DayOfWeekFunction;
-import org.apache.phoenix.expression.function.DayOfYearFunction;
-import org.apache.phoenix.expression.function.DecodeFunction;
-import org.apache.phoenix.expression.function.DefaultValueExpression;
-import org.apache.phoenix.expression.function.DistinctCountAggregateFunction;
-import 
org.apache.phoenix.expression.function.DistinctCountHyperLogLogAggregateFunction;
-import org.apache.phoenix.expression.function.DistinctValueAggregateFunction;
-import org.apache.phoenix.expression.function.EncodeFunction;
-import org.apache.phoenix.expression.function.ExpFunction;
-import org.apache.phoenix.expression.function.ExternalSqlTypeIdFunction;
-import org.apache.phoenix.expression.function.FirstValueFunction;
-import org.apache.phoenix.expression.function.FirstValuesFunction;
-import org.apache.phoenix.expression.function.FloorDateExpression;
-import org.apache.phoenix.expression.function.FloorDecimalExpression;
-import org.apache.phoenix.expression.function.FloorFunction;
-import org.apache.phoenix.expression.function.FloorMonthExpression;
-import org.apache.phoenix.expression.function.FloorWeekExpression;
-import org.apache.phoenix.expression.function.FloorYearExpression;
-import org.apache.phoenix.expression.function.GetBitFunction;
-import org.apache.phoenix.expression.function.GetByteFunction;
-import org.apache.phoenix.expression.function.HourFunction;
-import org.apache.phoenix.expression.function.IndexStateNameFunction;
-import org.apache.phoenix.expression.function.InstrFunction;
-import org.apache.phoenix.expression.function.InvertFunction;
-import org.apache.phoenix.expression.function.LTrimFunction;
-import org.apache.phoenix.expression.function.LastValueFunction;
-import org.apache.phoenix.expression.function.LastValuesFunction;
-import org.apache.phoenix.expression.function.LengthFunction;
-import org.apache.phoenix.expression.function.LnFunction;
-import org.apache.phoenix.expression.function.LogFunction;
-import org.apache.phoenix.expression.function.LowerFunction;
-import org.apache.phoenix.expression.function.LpadFunction;
-import org.apache.phoenix.expression.function.MD5Function;
-import org.apache.phoenix.expression.function.MaxAggregateFunction;
-import org.apache.phoenix.expression.function.MinAggregateFunction;
-import org.apache.phoenix.expression.function.MinuteFunctio

[4/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
PHOENIX-4605 Support running multiple transaction providers


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

Branch: refs/heads/4.x-HBase-1.2
Commit: 6d9140ed68d08863c418171ea5bd2ccadab05d8b
Parents: 5d6b39c
Author: James Taylor 
Authored: Wed Apr 11 20:06:35 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 12 17:29:00 2018 -0700

--
 .../phoenix/end2end/AlterTableWithViewsIT.java  |   6 +-
 .../ConnectionQueryServicesTestImpl.java|  34 +-
 .../phoenix/tx/FlappingTransactionIT.java   |  11 +-
 .../phoenix/tx/ParameterizedTransactionIT.java  |  14 +-
 .../org/apache/phoenix/tx/TransactionIT.java|  12 +
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |   5 +-
 .../phoenix/cache/IndexMetaDataCache.java   |   5 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |   3 +-
 .../apache/phoenix/compile/FromCompiler.java|   2 +-
 .../apache/phoenix/compile/JoinCompiler.java|   2 +-
 .../compile/TupleProjectionCompiler.java|   4 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   6 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   2 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   4 +-
 .../coprocessor/MetaDataEndpointImpl.java   |  26 +-
 .../phoenix/coprocessor/MetaDataProtocol.java   |   2 +-
 .../coprocessor/MetaDataRegionObserver.java |   4 +-
 .../PhoenixTransactionalProcessor.java  |  28 --
 .../coprocessor/ServerCachingEndpointImpl.java  |   4 +-
 .../TephraTransactionalProcessor.java   |  29 ++
 .../UngroupedAggregateRegionObserver.java   |  10 +-
 .../coprocessor/generated/PTableProtos.java | 110 +-
 .../phoenix/exception/SQLExceptionCode.java |   4 +
 .../apache/phoenix/execute/BaseQueryPlan.java   |   3 +
 .../apache/phoenix/execute/MutationState.java   |  74 ++--
 .../PhoenixTxIndexMutationGenerator.java|  10 +-
 .../phoenix/expression/ExpressionType.java  | 119 +--
 .../TransactionProviderNameFunction.java|  81 +
 .../apache/phoenix/index/IndexMaintainer.java   |   4 +-
 .../index/IndexMetaDataCacheFactory.java|   2 +-
 .../apache/phoenix/index/PhoenixIndexCodec.java |   1 -
 .../index/PhoenixIndexMetaDataBuilder.java  |   7 +-
 .../NonAggregateRegionScannerFactory.java   |   5 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  12 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   2 +-
 .../index/PhoenixIndexPartialBuildMapper.java   |   4 +-
 .../phoenix/query/ConnectionQueryServices.java  |   4 +
 .../query/ConnectionQueryServicesImpl.java  |  69 ++--
 .../query/ConnectionlessQueryServicesImpl.java  |   9 +-
 .../query/DelegateConnectionQueryServices.java  |   7 +-
 .../apache/phoenix/query/QueryConstants.java|   2 +
 .../org/apache/phoenix/query/QueryServices.java |   1 +
 .../phoenix/query/QueryServicesOptions.java |  10 +-
 .../apache/phoenix/schema/DelegateTable.java|   8 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 129 +--
 .../java/org/apache/phoenix/schema/PTable.java  |   2 +
 .../org/apache/phoenix/schema/PTableImpl.java   |  68 ++--
 .../apache/phoenix/schema/TableProperty.java|  18 +
 .../transaction/OmidTransactionContext.java |  57 +---
 .../transaction/OmidTransactionProvider.java|  54 +--
 .../transaction/OmidTransactionTable.java   | 339 ---
 .../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  | 161 +++--
 .../transaction/TephraTransactionTable.java | 330 --
 .../phoenix/transaction/TransactionFactory.java |  57 +++-
 .../transaction/TransactionProvider.java|  36 --
 .../org/apache/phoenix/util/PhoenixRuntime.java |   3 +-
 .../java/org/apache/phoenix/util/ScanUtil.java  |  15 +
 .../apache/phoenix/util/TransactionUtil.java|  93 -
 .../phoenix/execute/CorrelatePlanTest.java  |   5 +-
 .../execute/LiteralResultIteratorPlanTest.java  |   5 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  26 +-
 .../phoenix/query/QueryServicesTestImpl.java|  15 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |   2 +-
 phoenix-protocol/src/main/PTable.proto  |   1 +
 70 files changed, 1222 insertions(+), 1576 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6d9140ed/phoenix-core/src/it/java/org/apache

[2/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
http://git-wip-us.apache.org/repos/asf/phoenix/blob/6d9140ed/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index a7b31e8..1a11427 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -69,6 +69,7 @@ import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.types.PFloat;
 import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -134,7 +135,7 @@ public class PTableImpl implements PTable {
 private boolean disableWAL;
 private boolean multiTenant;
 private boolean storeNulls;
-private boolean isTransactional;
+private TransactionFactory.Provider transactionProvider;
 private ViewType viewType;
 private Short viewIndexId;
 private int estimatedSize;
@@ -227,7 +228,7 @@ public class PTableImpl implements PTable {
 init(tenantId, this.schemaName, this.tableName, PTableType.INDEX, 
state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
 this.schemaName, parentTableName, indexes, isImmutableRows, 
physicalNames, defaultFamilyName,
 null, disableWAL, multiTenant, storeNulls, viewType, viewIndexId, 
indexType, baseColumnCount, rowKeyOrderOptimizable,
-isTransactional, updateCacheFrequency, indexDisableTimestamp, 
isNamespaceMpped, null, false, storageScheme, qualifierEncodingScheme, 
encodedCQCounter, useStatsForParallelization);
+transactionProvider, updateCacheFrequency, indexDisableTimestamp, 
isNamespaceMpped, null, false, storageScheme, qualifierEncodingScheme, 
encodedCQCounter, useStatsForParallelization);
 }
 
 public PTableImpl(long timeStamp) { // For delete marker
@@ -270,7 +271,7 @@ public class PTableImpl implements PTable {
 table.getSequenceNumber(), table.getPKName(), 
table.getBucketNum(), getColumnsToClone(table), table.getParentSchemaName(), 
table.getParentTableName(),
 indexes, table.isImmutableRows(), physicalNames, 
table.getDefaultFamilyName(), viewStatement,
 table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), 
table.getIndexType(),
-table.getBaseColumnCount(), 
table.rowKeyOrderOptimizable(), table.isTransactional(), updateCacheFrequency,
+table.getBaseColumnCount(), 
table.rowKeyOrderOptimizable(), table.getTransactionProvider(), 
updateCacheFrequency,
 table.getIndexDisableTimestamp(), 
table.isNamespaceMapped(), table.getAutoPartitionSeqName(), 
table.isAppendOnlySchema(), table.getImmutableStorageScheme(), 
table.getEncodingScheme(), table.getEncodedCQCounter(), 
table.useStatsForParallelization());
 }
 
@@ -280,7 +281,7 @@ public class PTableImpl implements PTable {
 table.getSequenceNumber(), table.getPKName(), 
table.getBucketNum(), getColumnsToClone(table), parentSchemaName, 
table.getParentTableName(),
 indexes, table.isImmutableRows(), table.getPhysicalNames(), 
table.getDefaultFamilyName(), viewStatement,
 table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), 
table.getIndexType(),
-table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), 
table.isTransactional(), table.getUpdateCacheFrequency(),
+table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), 
table.getTransactionProvider(), table.getUpdateCacheFrequency(),
 table.getIndexDisableTimestamp(), table.isNamespaceMapped(), 
table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), 
table.getImmutableStorageScheme(), table.getEncodingScheme(), 
table.getEncodedCQCounter(), table.useStatsForParallelization());
 }
 
@@ -290,7 +291,7 @@ public class PTableImpl implements PTable {
 table.getSequenceNumber(), table.getPKName(), 
table.getBucketNum(), columns, table.getParentSchemaName(), 
table.getParentTableName(),
 table.getIndexes(), table.isImmutableRows(), 
table.getPhysicalNames(), table.getDefaultFamilyName(), 
table.getViewStatement(),
 table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), 
table.getIndexType(),
-table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), 
table.isTransactional(), table.getUpdateCacheFrequency(),

[1/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 22ceb4167 -> b8c42054e


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b8c42054/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
deleted file mode 100644
index cf48521..000
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
+++ /dev/null
@@ -1,330 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.phoenix.transaction;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Row;
-import org.apache.hadoop.hbase.client.RowMutations;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.Call;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
-import org.apache.tephra.TxConstants;
-import org.apache.tephra.hbase.TransactionAwareHTable;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableType;
-
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
-public class TephraTransactionTable implements PhoenixTransactionalTable {
-
-private TransactionAwareHTable transactionAwareHTable;
-
-private TephraTransactionContext tephraTransactionContext;
-
-public TephraTransactionTable(PhoenixTransactionContext ctx, 
HTableInterface hTable) {
-this(ctx, hTable, null);
-}
-
-public TephraTransactionTable(PhoenixTransactionContext ctx, 
HTableInterface hTable, PTable pTable) {
-
-assert(ctx instanceof TephraTransactionContext);
-
-tephraTransactionContext = (TephraTransactionContext) ctx;
-
-transactionAwareHTable = new TransactionAwareHTable(hTable, (pTable != 
null && pTable.isImmutableRows()) ? TxConstants.ConflictDetection.NONE : 
TxConstants.ConflictDetection.ROW);
-
-tephraTransactionContext.addTransactionAware(transactionAwareHTable);
-
-if (pTable != null && pTable.getType() != PTableType.INDEX) {
-tephraTransactionContext.markDMLFence(pTable);
-}
-}
-
-@Override
-public Result get(Get get) throws IOException {
-return transactionAwareHTable.get(get);
-}
-
-@Override
-public void put(Put put) throws IOException {
-transactionAwareHTable.put(put);
-}
-
-@Override
-public void delete(Delete delete) throws IOException {
-transactionAwareHTable.delete(delete);
-}
-
-@Override
-public ResultScanner getScanner(Scan scan) throws IOException {
-return transactionAwareHTable.getScanner(scan);
-}
-
-@Override
-public byte[] getTableName() {
-return transactionAwareHTable.getTableName();
-}
-
-@Override
-public Configuration getConfiguration() {
-return transactionAwareHTable.getConfiguration();
-}
-
-@Override
-public HTableDescriptor getTableDescriptor() throws IOException {
-return transactionAwareHTable.getTableDescriptor();
-}
-
-@Override
-public boolean exists(Get get) throws IOException {
-return transactionAwareHTable.exist

[3/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
http://git-wip-us.apache.org/repos/asf/phoenix/blob/b8c42054/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index c5065e0..59c10ad 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -19,120 +19,7 @@ package org.apache.phoenix.expression;
 
 import java.util.Map;
 
-import org.apache.phoenix.expression.function.AbsFunction;
-import org.apache.phoenix.expression.function.ArrayAllComparisonExpression;
-import org.apache.phoenix.expression.function.ArrayAnyComparisonExpression;
-import org.apache.phoenix.expression.function.ArrayAppendFunction;
-import org.apache.phoenix.expression.function.ArrayConcatFunction;
-import org.apache.phoenix.expression.function.ArrayElemRefExpression;
-import org.apache.phoenix.expression.function.ArrayFillFunction;
-import org.apache.phoenix.expression.function.ArrayIndexFunction;
-import org.apache.phoenix.expression.function.ArrayLengthFunction;
-import org.apache.phoenix.expression.function.ArrayPrependFunction;
-import org.apache.phoenix.expression.function.ArrayRemoveFunction;
-import org.apache.phoenix.expression.function.ArrayToStringFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpReplaceFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpSplitFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpSubstrFunction;
-import org.apache.phoenix.expression.function.CbrtFunction;
-import org.apache.phoenix.expression.function.CeilDateExpression;
-import org.apache.phoenix.expression.function.CeilDecimalExpression;
-import org.apache.phoenix.expression.function.CeilFunction;
-import org.apache.phoenix.expression.function.CeilMonthExpression;
-import org.apache.phoenix.expression.function.CeilTimestampExpression;
-import org.apache.phoenix.expression.function.CeilWeekExpression;
-import org.apache.phoenix.expression.function.CeilYearExpression;
-import org.apache.phoenix.expression.function.CoalesceFunction;
-import org.apache.phoenix.expression.function.CollationKeyFunction;
-import org.apache.phoenix.expression.function.ConvertTimezoneFunction;
-import org.apache.phoenix.expression.function.CountAggregateFunction;
-import org.apache.phoenix.expression.function.DayOfMonthFunction;
-import org.apache.phoenix.expression.function.DayOfWeekFunction;
-import org.apache.phoenix.expression.function.DayOfYearFunction;
-import org.apache.phoenix.expression.function.DecodeFunction;
-import org.apache.phoenix.expression.function.DefaultValueExpression;
-import org.apache.phoenix.expression.function.DistinctCountAggregateFunction;
-import 
org.apache.phoenix.expression.function.DistinctCountHyperLogLogAggregateFunction;
-import org.apache.phoenix.expression.function.DistinctValueAggregateFunction;
-import org.apache.phoenix.expression.function.EncodeFunction;
-import org.apache.phoenix.expression.function.ExpFunction;
-import org.apache.phoenix.expression.function.ExternalSqlTypeIdFunction;
-import org.apache.phoenix.expression.function.FirstValueFunction;
-import org.apache.phoenix.expression.function.FirstValuesFunction;
-import org.apache.phoenix.expression.function.FloorDateExpression;
-import org.apache.phoenix.expression.function.FloorDecimalExpression;
-import org.apache.phoenix.expression.function.FloorFunction;
-import org.apache.phoenix.expression.function.FloorMonthExpression;
-import org.apache.phoenix.expression.function.FloorWeekExpression;
-import org.apache.phoenix.expression.function.FloorYearExpression;
-import org.apache.phoenix.expression.function.GetBitFunction;
-import org.apache.phoenix.expression.function.GetByteFunction;
-import org.apache.phoenix.expression.function.HourFunction;
-import org.apache.phoenix.expression.function.IndexStateNameFunction;
-import org.apache.phoenix.expression.function.InstrFunction;
-import org.apache.phoenix.expression.function.InvertFunction;
-import org.apache.phoenix.expression.function.LTrimFunction;
-import org.apache.phoenix.expression.function.LastValueFunction;
-import org.apache.phoenix.expression.function.LastValuesFunction;
-import org.apache.phoenix.expression.function.LengthFunction;
-import org.apache.phoenix.expression.function.LnFunction;
-import org.apache.phoenix.expression.function.LogFunction;
-import org.apache.phoenix.expression.function.LowerFunction;
-import org.apache.phoenix.expression.function.LpadFunction;
-import org.apache.phoenix.expression.function.MD5Function;
-import org.apache.phoenix.expression.function.MaxAggregateFunction;
-import org.apache.phoenix.expression.function.MinAggregateFunction;
-import org.apache.phoenix.expression.function.MinuteFunctio

[4/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
PHOENIX-4605 Support running multiple transaction providers


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

Branch: refs/heads/4.x-HBase-1.1
Commit: b8c42054e549d99cb90a7dabf7d55d3998db23fe
Parents: 22ceb41
Author: James Taylor 
Authored: Wed Apr 11 20:06:35 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 12 17:54:13 2018 -0700

--
 .../phoenix/end2end/AlterTableWithViewsIT.java  |   6 +-
 .../ConnectionQueryServicesTestImpl.java|  34 +-
 .../phoenix/tx/FlappingTransactionIT.java   |  11 +-
 .../phoenix/tx/ParameterizedTransactionIT.java  |  14 +-
 .../org/apache/phoenix/tx/TransactionIT.java|  12 +
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |   5 +-
 .../phoenix/cache/IndexMetaDataCache.java   |   5 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |   3 +-
 .../apache/phoenix/compile/FromCompiler.java|   2 +-
 .../apache/phoenix/compile/JoinCompiler.java|   2 +-
 .../compile/TupleProjectionCompiler.java|   4 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   6 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   2 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   4 +-
 .../coprocessor/MetaDataEndpointImpl.java   |  26 +-
 .../phoenix/coprocessor/MetaDataProtocol.java   |   2 +-
 .../coprocessor/MetaDataRegionObserver.java |   4 +-
 .../PhoenixTransactionalProcessor.java  |  28 --
 .../coprocessor/ServerCachingEndpointImpl.java  |   4 +-
 .../TephraTransactionalProcessor.java   |  29 ++
 .../UngroupedAggregateRegionObserver.java   |  10 +-
 .../coprocessor/generated/PTableProtos.java | 110 +++-
 .../phoenix/exception/SQLExceptionCode.java |   4 +
 .../apache/phoenix/execute/BaseQueryPlan.java   |   3 +
 .../apache/phoenix/execute/MutationState.java   |  74 ++-
 .../PhoenixTxIndexMutationGenerator.java|  10 +-
 .../phoenix/expression/ExpressionType.java  | 119 +
 .../TransactionProviderNameFunction.java|  81 +++
 .../apache/phoenix/index/IndexMaintainer.java   |   4 +-
 .../index/IndexMetaDataCacheFactory.java|   2 +-
 .../apache/phoenix/index/PhoenixIndexCodec.java |   1 -
 .../index/PhoenixIndexMetaDataBuilder.java  |   7 +-
 .../NonAggregateRegionScannerFactory.java   |   5 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  12 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   2 +-
 .../index/PhoenixIndexPartialBuildMapper.java   |   4 +-
 .../phoenix/query/ConnectionQueryServices.java  |   4 +
 .../query/ConnectionQueryServicesImpl.java  |  69 ++-
 .../query/ConnectionlessQueryServicesImpl.java  |   9 +-
 .../query/DelegateConnectionQueryServices.java  |   7 +-
 .../apache/phoenix/query/QueryConstants.java|   2 +
 .../org/apache/phoenix/query/QueryServices.java |   1 +
 .../phoenix/query/QueryServicesOptions.java |  10 +-
 .../apache/phoenix/schema/DelegateTable.java|   8 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 497 +--
 .../java/org/apache/phoenix/schema/PTable.java  |   2 +
 .../org/apache/phoenix/schema/PTableImpl.java   |  68 ++-
 .../apache/phoenix/schema/TableProperty.java|  18 +
 .../transaction/OmidTransactionContext.java |  57 +--
 .../transaction/OmidTransactionProvider.java|  54 +-
 .../transaction/OmidTransactionTable.java   | 339 -
 .../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  | 161 +-
 .../transaction/TephraTransactionTable.java | 330 
 .../phoenix/transaction/TransactionFactory.java |  57 ++-
 .../transaction/TransactionProvider.java|  36 --
 .../org/apache/phoenix/util/PhoenixRuntime.java |   3 +-
 .../java/org/apache/phoenix/util/ScanUtil.java  |  15 +
 .../apache/phoenix/util/TransactionUtil.java|  93 +++-
 .../phoenix/execute/CorrelatePlanTest.java  |   5 +-
 .../execute/LiteralResultIteratorPlanTest.java  |   5 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  27 +-
 .../phoenix/query/QueryServicesTestImpl.java|  15 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |   2 +-
 phoenix-protocol/src/main/PTable.proto  |   1 +
 70 files changed, 1591 insertions(+), 1576 deletions(-)
--


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

[2/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
http://git-wip-us.apache.org/repos/asf/phoenix/blob/b8c42054/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index a7b31e8..1a11427 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -69,6 +69,7 @@ import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.types.PFloat;
 import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -134,7 +135,7 @@ public class PTableImpl implements PTable {
 private boolean disableWAL;
 private boolean multiTenant;
 private boolean storeNulls;
-private boolean isTransactional;
+private TransactionFactory.Provider transactionProvider;
 private ViewType viewType;
 private Short viewIndexId;
 private int estimatedSize;
@@ -227,7 +228,7 @@ public class PTableImpl implements PTable {
 init(tenantId, this.schemaName, this.tableName, PTableType.INDEX, 
state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
 this.schemaName, parentTableName, indexes, isImmutableRows, 
physicalNames, defaultFamilyName,
 null, disableWAL, multiTenant, storeNulls, viewType, viewIndexId, 
indexType, baseColumnCount, rowKeyOrderOptimizable,
-isTransactional, updateCacheFrequency, indexDisableTimestamp, 
isNamespaceMpped, null, false, storageScheme, qualifierEncodingScheme, 
encodedCQCounter, useStatsForParallelization);
+transactionProvider, updateCacheFrequency, indexDisableTimestamp, 
isNamespaceMpped, null, false, storageScheme, qualifierEncodingScheme, 
encodedCQCounter, useStatsForParallelization);
 }
 
 public PTableImpl(long timeStamp) { // For delete marker
@@ -270,7 +271,7 @@ public class PTableImpl implements PTable {
 table.getSequenceNumber(), table.getPKName(), 
table.getBucketNum(), getColumnsToClone(table), table.getParentSchemaName(), 
table.getParentTableName(),
 indexes, table.isImmutableRows(), physicalNames, 
table.getDefaultFamilyName(), viewStatement,
 table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), 
table.getIndexType(),
-table.getBaseColumnCount(), 
table.rowKeyOrderOptimizable(), table.isTransactional(), updateCacheFrequency,
+table.getBaseColumnCount(), 
table.rowKeyOrderOptimizable(), table.getTransactionProvider(), 
updateCacheFrequency,
 table.getIndexDisableTimestamp(), 
table.isNamespaceMapped(), table.getAutoPartitionSeqName(), 
table.isAppendOnlySchema(), table.getImmutableStorageScheme(), 
table.getEncodingScheme(), table.getEncodedCQCounter(), 
table.useStatsForParallelization());
 }
 
@@ -280,7 +281,7 @@ public class PTableImpl implements PTable {
 table.getSequenceNumber(), table.getPKName(), 
table.getBucketNum(), getColumnsToClone(table), parentSchemaName, 
table.getParentTableName(),
 indexes, table.isImmutableRows(), table.getPhysicalNames(), 
table.getDefaultFamilyName(), viewStatement,
 table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), 
table.getIndexType(),
-table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), 
table.isTransactional(), table.getUpdateCacheFrequency(),
+table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), 
table.getTransactionProvider(), table.getUpdateCacheFrequency(),
 table.getIndexDisableTimestamp(), table.isNamespaceMapped(), 
table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), 
table.getImmutableStorageScheme(), table.getEncodingScheme(), 
table.getEncodedCQCounter(), table.useStatsForParallelization());
 }
 
@@ -290,7 +291,7 @@ public class PTableImpl implements PTable {
 table.getSequenceNumber(), table.getPKName(), 
table.getBucketNum(), columns, table.getParentSchemaName(), 
table.getParentTableName(),
 table.getIndexes(), table.isImmutableRows(), 
table.getPhysicalNames(), table.getDefaultFamilyName(), 
table.getViewStatement(),
 table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), 
table.getIndexType(),
-table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), 
table.isTransactional(), table.getUpdateCacheFrequency(),

[4/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
PHOENIX-4605 Support running multiple transaction providers


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 06e9c1f03f3c19eeb41ac40eb8365473bbdd29a4
Parents: ade0c52
Author: James Taylor 
Authored: Wed Apr 11 20:06:35 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 12 18:01:49 2018 -0700

--
 .../phoenix/end2end/AlterTableWithViewsIT.java  |   6 +-
 .../ConnectionQueryServicesTestImpl.java|  34 +-
 .../phoenix/tx/FlappingTransactionIT.java   |  11 +-
 .../phoenix/tx/ParameterizedTransactionIT.java  |  14 +-
 .../org/apache/phoenix/tx/TransactionIT.java|  12 +
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |   5 +-
 .../phoenix/cache/IndexMetaDataCache.java   |   5 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |   3 +-
 .../apache/phoenix/compile/FromCompiler.java|   2 +-
 .../apache/phoenix/compile/JoinCompiler.java|   2 +-
 .../compile/TupleProjectionCompiler.java|   4 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   6 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   2 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   4 +-
 .../coprocessor/MetaDataEndpointImpl.java   |  26 +-
 .../phoenix/coprocessor/MetaDataProtocol.java   |   2 +-
 .../coprocessor/MetaDataRegionObserver.java |   4 +-
 .../PhoenixTransactionalProcessor.java  |  28 --
 .../coprocessor/ServerCachingEndpointImpl.java  |   4 +-
 .../TephraTransactionalProcessor.java   |  29 ++
 .../UngroupedAggregateRegionObserver.java   |  10 +-
 .../coprocessor/generated/PTableProtos.java | 110 ++-
 .../phoenix/exception/SQLExceptionCode.java |   4 +
 .../apache/phoenix/execute/BaseQueryPlan.java   |   3 +
 .../apache/phoenix/execute/MutationState.java   |  74 +++--
 .../PhoenixTxIndexMutationGenerator.java|   9 +-
 .../phoenix/expression/ExpressionType.java  | 119 +--
 .../TransactionProviderNameFunction.java|  81 +
 .../apache/phoenix/index/IndexMaintainer.java   |   4 +-
 .../index/IndexMetaDataCacheFactory.java|   2 +-
 .../apache/phoenix/index/PhoenixIndexCodec.java |   1 -
 .../index/PhoenixIndexMetaDataBuilder.java  |   7 +-
 .../NonAggregateRegionScannerFactory.java   |   4 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  12 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   2 +-
 .../index/PhoenixIndexPartialBuildMapper.java   |   4 +-
 .../phoenix/query/ConnectionQueryServices.java  |   6 +-
 .../query/ConnectionQueryServicesImpl.java  |  69 ++--
 .../query/ConnectionlessQueryServicesImpl.java  |   9 +-
 .../query/DelegateConnectionQueryServices.java  |   9 +-
 .../apache/phoenix/query/QueryConstants.java|   2 +
 .../org/apache/phoenix/query/QueryServices.java |   1 +
 .../phoenix/query/QueryServicesOptions.java |  10 +-
 .../apache/phoenix/schema/DelegateTable.java|   8 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 129 ++--
 .../java/org/apache/phoenix/schema/PTable.java  |   2 +
 .../org/apache/phoenix/schema/PTableImpl.java   |  68 ++--
 .../apache/phoenix/schema/TableProperty.java|  18 ++
 .../transaction/OmidTransactionContext.java |  57 +---
 .../transaction/OmidTransactionProvider.java|  54 ++--
 .../transaction/OmidTransactionTable.java   | 318 ---
 .../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  | 161 --
 .../transaction/TephraTransactionTable.java | 312 --
 .../phoenix/transaction/TransactionFactory.java |  57 +++-
 .../transaction/TransactionProvider.java|  36 ---
 .../org/apache/phoenix/util/PhoenixRuntime.java |   3 +-
 .../java/org/apache/phoenix/util/ScanUtil.java  |  15 +
 .../apache/phoenix/util/TransactionUtil.java|  93 +-
 .../phoenix/execute/CorrelatePlanTest.java  |   5 +-
 .../execute/LiteralResultIteratorPlanTest.java  |   5 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  26 +-
 .../phoenix/query/QueryServicesTestImpl.java|  15 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |   2 +-
 phoenix-protocol/src/main/PTable.proto  |   1 +
 70 files changed, 1223 insertions(+), 1538 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/06e9c1f0/phoenix-core/src/it/ja

[2/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
http://git-wip-us.apache.org/repos/asf/phoenix/blob/06e9c1f0/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index a7b31e8..1a11427 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -69,6 +69,7 @@ import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.types.PFloat;
 import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -134,7 +135,7 @@ public class PTableImpl implements PTable {
 private boolean disableWAL;
 private boolean multiTenant;
 private boolean storeNulls;
-private boolean isTransactional;
+private TransactionFactory.Provider transactionProvider;
 private ViewType viewType;
 private Short viewIndexId;
 private int estimatedSize;
@@ -227,7 +228,7 @@ public class PTableImpl implements PTable {
 init(tenantId, this.schemaName, this.tableName, PTableType.INDEX, 
state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
 this.schemaName, parentTableName, indexes, isImmutableRows, 
physicalNames, defaultFamilyName,
 null, disableWAL, multiTenant, storeNulls, viewType, viewIndexId, 
indexType, baseColumnCount, rowKeyOrderOptimizable,
-isTransactional, updateCacheFrequency, indexDisableTimestamp, 
isNamespaceMpped, null, false, storageScheme, qualifierEncodingScheme, 
encodedCQCounter, useStatsForParallelization);
+transactionProvider, updateCacheFrequency, indexDisableTimestamp, 
isNamespaceMpped, null, false, storageScheme, qualifierEncodingScheme, 
encodedCQCounter, useStatsForParallelization);
 }
 
 public PTableImpl(long timeStamp) { // For delete marker
@@ -270,7 +271,7 @@ public class PTableImpl implements PTable {
 table.getSequenceNumber(), table.getPKName(), 
table.getBucketNum(), getColumnsToClone(table), table.getParentSchemaName(), 
table.getParentTableName(),
 indexes, table.isImmutableRows(), physicalNames, 
table.getDefaultFamilyName(), viewStatement,
 table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), 
table.getIndexType(),
-table.getBaseColumnCount(), 
table.rowKeyOrderOptimizable(), table.isTransactional(), updateCacheFrequency,
+table.getBaseColumnCount(), 
table.rowKeyOrderOptimizable(), table.getTransactionProvider(), 
updateCacheFrequency,
 table.getIndexDisableTimestamp(), 
table.isNamespaceMapped(), table.getAutoPartitionSeqName(), 
table.isAppendOnlySchema(), table.getImmutableStorageScheme(), 
table.getEncodingScheme(), table.getEncodedCQCounter(), 
table.useStatsForParallelization());
 }
 
@@ -280,7 +281,7 @@ public class PTableImpl implements PTable {
 table.getSequenceNumber(), table.getPKName(), 
table.getBucketNum(), getColumnsToClone(table), parentSchemaName, 
table.getParentTableName(),
 indexes, table.isImmutableRows(), table.getPhysicalNames(), 
table.getDefaultFamilyName(), viewStatement,
 table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), 
table.getIndexType(),
-table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), 
table.isTransactional(), table.getUpdateCacheFrequency(),
+table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), 
table.getTransactionProvider(), table.getUpdateCacheFrequency(),
 table.getIndexDisableTimestamp(), table.isNamespaceMapped(), 
table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), 
table.getImmutableStorageScheme(), table.getEncodingScheme(), 
table.getEncodedCQCounter(), table.useStatsForParallelization());
 }
 
@@ -290,7 +291,7 @@ public class PTableImpl implements PTable {
 table.getSequenceNumber(), table.getPKName(), 
table.getBucketNum(), columns, table.getParentSchemaName(), 
table.getParentTableName(),
 table.getIndexes(), table.isImmutableRows(), 
table.getPhysicalNames(), table.getDefaultFamilyName(), 
table.getViewStatement(),
 table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), 
table.getIndexType(),
-table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), 
table.isTransactional(), table.getUpdateCacheFrequency(),

[1/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 ade0c52e7 -> 06e9c1f03


http://git-wip-us.apache.org/repos/asf/phoenix/blob/06e9c1f0/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
deleted file mode 100644
index 220dc30..000
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
+++ /dev/null
@@ -1,312 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.phoenix.transaction;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Row;
-import org.apache.hadoop.hbase.client.RowMutations;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.Call;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
-import org.apache.tephra.TxConstants;
-import org.apache.tephra.hbase.TransactionAwareHTable;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableType;
-
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
-public class TephraTransactionTable implements PhoenixTransactionalTable {
-
-private TransactionAwareHTable transactionAwareHTable;
-
-private TephraTransactionContext tephraTransactionContext;
-
-public TephraTransactionTable(PhoenixTransactionContext ctx, 
HTableInterface hTable) {
-this(ctx, hTable, null);
-}
-
-public TephraTransactionTable(PhoenixTransactionContext ctx, 
HTableInterface hTable, PTable pTable) {
-
-assert(ctx instanceof TephraTransactionContext);
-
-tephraTransactionContext = (TephraTransactionContext) ctx;
-
-transactionAwareHTable = new TransactionAwareHTable(hTable, (pTable != 
null && pTable.isImmutableRows()) ? TxConstants.ConflictDetection.NONE : 
TxConstants.ConflictDetection.ROW);
-
-tephraTransactionContext.addTransactionAware(transactionAwareHTable);
-
-if (pTable != null && pTable.getType() != PTableType.INDEX) {
-tephraTransactionContext.markDMLFence(pTable);
-}
-}
-
-@Override
-public Result get(Get get) throws IOException {
-return transactionAwareHTable.get(get);
-}
-
-@Override
-public void put(Put put) throws IOException {
-transactionAwareHTable.put(put);
-}
-
-@Override
-public void delete(Delete delete) throws IOException {
-transactionAwareHTable.delete(delete);
-}
-
-@Override
-public ResultScanner getScanner(Scan scan) throws IOException {
-return transactionAwareHTable.getScanner(scan);
-}
-
-@Override
-public byte[] getTableName() {
-return transactionAwareHTable.getTableName();
-}
-
-@Override
-public Configuration getConfiguration() {
-return transactionAwareHTable.getConfiguration();
-}
-
-@Override
-public HTableDescriptor getTableDescriptor() throws IOException {
-return transactionAwareHTable.getTableDescriptor();
-}
-
-@Override
-public boolean exists(Get get) throws IOException {
-return transactionAwareHTable.exis

[3/4] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
http://git-wip-us.apache.org/repos/asf/phoenix/blob/06e9c1f0/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index 7863ebd..832b650 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -19,120 +19,7 @@ package org.apache.phoenix.expression;
 
 import java.util.Map;
 
-import org.apache.phoenix.expression.function.AbsFunction;
-import org.apache.phoenix.expression.function.ArrayAllComparisonExpression;
-import org.apache.phoenix.expression.function.ArrayAnyComparisonExpression;
-import org.apache.phoenix.expression.function.ArrayAppendFunction;
-import org.apache.phoenix.expression.function.ArrayConcatFunction;
-import org.apache.phoenix.expression.function.ArrayElemRefExpression;
-import org.apache.phoenix.expression.function.ArrayFillFunction;
-import org.apache.phoenix.expression.function.ArrayIndexFunction;
-import org.apache.phoenix.expression.function.ArrayLengthFunction;
-import org.apache.phoenix.expression.function.ArrayPrependFunction;
-import org.apache.phoenix.expression.function.ArrayRemoveFunction;
-import org.apache.phoenix.expression.function.ArrayToStringFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpReplaceFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpSplitFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpSubstrFunction;
-import org.apache.phoenix.expression.function.CbrtFunction;
-import org.apache.phoenix.expression.function.CeilDateExpression;
-import org.apache.phoenix.expression.function.CeilDecimalExpression;
-import org.apache.phoenix.expression.function.CeilFunction;
-import org.apache.phoenix.expression.function.CeilMonthExpression;
-import org.apache.phoenix.expression.function.CeilTimestampExpression;
-import org.apache.phoenix.expression.function.CeilWeekExpression;
-import org.apache.phoenix.expression.function.CeilYearExpression;
-import org.apache.phoenix.expression.function.CoalesceFunction;
-import org.apache.phoenix.expression.function.CollationKeyFunction;
-import org.apache.phoenix.expression.function.ConvertTimezoneFunction;
-import org.apache.phoenix.expression.function.CountAggregateFunction;
-import org.apache.phoenix.expression.function.DayOfMonthFunction;
-import org.apache.phoenix.expression.function.DayOfWeekFunction;
-import org.apache.phoenix.expression.function.DayOfYearFunction;
-import org.apache.phoenix.expression.function.DecodeFunction;
-import org.apache.phoenix.expression.function.DefaultValueExpression;
-import org.apache.phoenix.expression.function.DistinctCountAggregateFunction;
-import 
org.apache.phoenix.expression.function.DistinctCountHyperLogLogAggregateFunction;
-import org.apache.phoenix.expression.function.DistinctValueAggregateFunction;
-import org.apache.phoenix.expression.function.EncodeFunction;
-import org.apache.phoenix.expression.function.ExpFunction;
-import org.apache.phoenix.expression.function.ExternalSqlTypeIdFunction;
-import org.apache.phoenix.expression.function.FirstValueFunction;
-import org.apache.phoenix.expression.function.FirstValuesFunction;
-import org.apache.phoenix.expression.function.FloorDateExpression;
-import org.apache.phoenix.expression.function.FloorDecimalExpression;
-import org.apache.phoenix.expression.function.FloorFunction;
-import org.apache.phoenix.expression.function.FloorMonthExpression;
-import org.apache.phoenix.expression.function.FloorWeekExpression;
-import org.apache.phoenix.expression.function.FloorYearExpression;
-import org.apache.phoenix.expression.function.GetBitFunction;
-import org.apache.phoenix.expression.function.GetByteFunction;
-import org.apache.phoenix.expression.function.HourFunction;
-import org.apache.phoenix.expression.function.IndexStateNameFunction;
-import org.apache.phoenix.expression.function.InstrFunction;
-import org.apache.phoenix.expression.function.InvertFunction;
-import org.apache.phoenix.expression.function.LTrimFunction;
-import org.apache.phoenix.expression.function.LastValueFunction;
-import org.apache.phoenix.expression.function.LastValuesFunction;
-import org.apache.phoenix.expression.function.LengthFunction;
-import org.apache.phoenix.expression.function.LnFunction;
-import org.apache.phoenix.expression.function.LogFunction;
-import org.apache.phoenix.expression.function.LowerFunction;
-import org.apache.phoenix.expression.function.LpadFunction;
-import org.apache.phoenix.expression.function.MD5Function;
-import org.apache.phoenix.expression.function.MaxAggregateFunction;
-import org.apache.phoenix.expression.function.MinAggregateFunction;
-import org.apache.phoenix.expression.function.MinuteFunctio

phoenix git commit: PHOENIX-4605 Support running multiple transaction providers (ammendment)

2018-04-12 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.3 7f37966c1 -> e79228c4f


PHOENIX-4605 Support running multiple transaction providers (ammendment)


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

Branch: refs/heads/4.x-HBase-1.3
Commit: e79228c4f35fafedd7a5c7ded0c053fc561846e7
Parents: 7f37966
Author: James Taylor 
Authored: Thu Apr 12 18:11:19 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 12 18:11:19 2018 -0700

--
 .../src/main/java/org/apache/phoenix/schema/MetaDataClient.java| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e79228c4/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 3c7a837..1fb668e 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
@@ -3276,7 +3276,7 @@ public class MetaDataClient {
 // If changing isImmutableRows to true or it's not being 
changed and is already true
 boolean willBeImmutableRows = 
Boolean.TRUE.equals(metaPropertiesEvaluated.getIsImmutableRows()) || 
(metaPropertiesEvaluated.getIsImmutableRows() == null && 
table.isImmutableRows());
 boolean willBeTxnl = metaProperties.getNonTxToTx();
-Long timeStamp = TransactionUtil.getTableTimestamp(connection, 
table.isTransactional() || willBeTxnl, 
metaPropertiesEvaluated.getTransactionProvider());
+Long timeStamp = TransactionUtil.getTableTimestamp(connection, 
table.isTransactional() || willBeTxnl, table.isTransactional() ? 
table.getTransactionProvider() : 
metaPropertiesEvaluated.getTransactionProvider());
 int numPkColumnsAdded = 0;
 List columns = 
Lists.newArrayListWithExpectedSize(numCols);
 Set colFamiliesForPColumnsToBeAdded = new 
LinkedHashSet<>();



phoenix git commit: PHOENIX-4605 Support running multiple transaction providers (ammendment)

2018-04-12 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/master de83b8d5d -> a0b0c1e1e


PHOENIX-4605 Support running multiple transaction providers (ammendment)


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

Branch: refs/heads/master
Commit: a0b0c1e1e0597024e7f94c6d62e238ce1f2778ef
Parents: de83b8d
Author: James Taylor 
Authored: Thu Apr 12 18:11:19 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 12 18:12:58 2018 -0700

--
 .../src/main/java/org/apache/phoenix/schema/MetaDataClient.java| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0b0c1e1/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 3c7a837..1fb668e 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
@@ -3276,7 +3276,7 @@ public class MetaDataClient {
 // If changing isImmutableRows to true or it's not being 
changed and is already true
 boolean willBeImmutableRows = 
Boolean.TRUE.equals(metaPropertiesEvaluated.getIsImmutableRows()) || 
(metaPropertiesEvaluated.getIsImmutableRows() == null && 
table.isImmutableRows());
 boolean willBeTxnl = metaProperties.getNonTxToTx();
-Long timeStamp = TransactionUtil.getTableTimestamp(connection, 
table.isTransactional() || willBeTxnl, 
metaPropertiesEvaluated.getTransactionProvider());
+Long timeStamp = TransactionUtil.getTableTimestamp(connection, 
table.isTransactional() || willBeTxnl, table.isTransactional() ? 
table.getTransactionProvider() : 
metaPropertiesEvaluated.getTransactionProvider());
 int numPkColumnsAdded = 0;
 List columns = 
Lists.newArrayListWithExpectedSize(numCols);
 Set colFamiliesForPColumnsToBeAdded = new 
LinkedHashSet<>();



phoenix git commit: PHOENIX-4605 Support running multiple transaction providers (ammendment)

2018-04-12 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.2 6d9140ed6 -> b7e358242


PHOENIX-4605 Support running multiple transaction providers (ammendment)


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

Branch: refs/heads/4.x-HBase-1.2
Commit: b7e35824249e2677e4bf7062f7434e10355f0baa
Parents: 6d9140e
Author: James Taylor 
Authored: Thu Apr 12 18:11:19 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 12 18:14:25 2018 -0700

--
 .../src/main/java/org/apache/phoenix/schema/MetaDataClient.java| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b7e35824/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 3c7a837..1fb668e 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
@@ -3276,7 +3276,7 @@ public class MetaDataClient {
 // If changing isImmutableRows to true or it's not being 
changed and is already true
 boolean willBeImmutableRows = 
Boolean.TRUE.equals(metaPropertiesEvaluated.getIsImmutableRows()) || 
(metaPropertiesEvaluated.getIsImmutableRows() == null && 
table.isImmutableRows());
 boolean willBeTxnl = metaProperties.getNonTxToTx();
-Long timeStamp = TransactionUtil.getTableTimestamp(connection, 
table.isTransactional() || willBeTxnl, 
metaPropertiesEvaluated.getTransactionProvider());
+Long timeStamp = TransactionUtil.getTableTimestamp(connection, 
table.isTransactional() || willBeTxnl, table.isTransactional() ? 
table.getTransactionProvider() : 
metaPropertiesEvaluated.getTransactionProvider());
 int numPkColumnsAdded = 0;
 List columns = 
Lists.newArrayListWithExpectedSize(numCols);
 Set colFamiliesForPColumnsToBeAdded = new 
LinkedHashSet<>();



phoenix git commit: PHOENIX-4605 Support running multiple transaction providers (ammendment)

2018-04-12 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 06e9c1f03 -> 6a88cfed3


PHOENIX-4605 Support running multiple transaction providers (ammendment)


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 6a88cfed32b12781674d78057f83b7d3f4954b37
Parents: 06e9c1f
Author: James Taylor 
Authored: Thu Apr 12 18:11:19 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 12 18:15:51 2018 -0700

--
 .../src/main/java/org/apache/phoenix/schema/MetaDataClient.java| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6a88cfed/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 1f6cab3..d70b1bc 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
@@ -3271,7 +3271,7 @@ public class MetaDataClient {
 // If changing isImmutableRows to true or it's not being 
changed and is already true
 boolean willBeImmutableRows = 
Boolean.TRUE.equals(metaPropertiesEvaluated.getIsImmutableRows()) || 
(metaPropertiesEvaluated.getIsImmutableRows() == null && 
table.isImmutableRows());
 boolean willBeTxnl = metaProperties.getNonTxToTx();
-Long timeStamp = TransactionUtil.getTableTimestamp(connection, 
table.isTransactional() || willBeTxnl, 
metaPropertiesEvaluated.getTransactionProvider());
+Long timeStamp = TransactionUtil.getTableTimestamp(connection, 
table.isTransactional() || willBeTxnl, table.isTransactional() ? 
table.getTransactionProvider() : 
metaPropertiesEvaluated.getTransactionProvider());
 int numPkColumnsAdded = 0;
 List columns = 
Lists.newArrayListWithExpectedSize(numCols);
 Set colFamiliesForPColumnsToBeAdded = new 
LinkedHashSet<>();



[phoenix] Git Push Summary

2018-04-12 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/remotes/origin/4.x-cdh5.11 [created] 382307a7d


[4/5] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java 
b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index f05c74b..2759c9f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -70,6 +70,8 @@ import 
org.apache.phoenix.monitoring.MutationMetricQueue.MutationMetric;
 import 
org.apache.phoenix.monitoring.MutationMetricQueue.NoOpMutationMetricsQueue;
 import org.apache.phoenix.monitoring.ReadMetricQueue;
 import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.IllegalDataException;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PColumn;
@@ -89,8 +91,8 @@ import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.trace.util.Tracing;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
 import 
org.apache.phoenix.transaction.PhoenixTransactionContext.PhoenixVisibilityLevel;
-import org.apache.phoenix.transaction.PhoenixTransactionalTable;
 import org.apache.phoenix.transaction.TransactionFactory;
+import org.apache.phoenix.transaction.TransactionFactory.Provider;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.LogUtil;
@@ -137,7 +139,7 @@ public class MutationState implements SQLCloseable {
 private boolean isExternalTxContext = false;
 private Map txMutations = 
Collections.emptyMap();
 
-final PhoenixTransactionContext phoenixTransactionContext;
+private PhoenixTransactionContext phoenixTransactionContext = 
PhoenixTransactionContext.NULL_CONTEXT;
 
 private final MutationMetricQueue mutationMetricQueue;
 private ReadMetricQueue readMetricQueue;
@@ -179,17 +181,13 @@ public class MutationState implements SQLCloseable {
 boolean isMetricsEnabled = connection.isRequestLevelMetricsEnabled();
 this.mutationMetricQueue = isMetricsEnabled ? new MutationMetricQueue()
 : NoOpMutationMetricsQueue.NO_OP_MUTATION_METRICS_QUEUE;
-if (!subTask) {
-if (txContext == null) {
-phoenixTransactionContext = 
TransactionFactory.getTransactionProvider().getTransactionContext(connection);
-} else {
-isExternalTxContext = true;
-phoenixTransactionContext = 
TransactionFactory.getTransactionProvider().getTransactionContext(txContext, 
connection, subTask);
-}
-} else {
+if (subTask) {
 // this code path is only used while running child scans, we can't 
pass the txContext to child scans
 // as it is not thread safe, so we use the tx member variable
-phoenixTransactionContext = 
TransactionFactory.getTransactionProvider().getTransactionContext(txContext, 
connection, subTask);
+phoenixTransactionContext = 
txContext.newTransactionContext(txContext, subTask);
+} else if (txContext != null) {
+isExternalTxContext = true;
+phoenixTransactionContext = 
txContext.newTransactionContext(txContext, subTask);
 }
 }
 
@@ -232,7 +230,7 @@ public class MutationState implements SQLCloseable {
 public void commitDDLFence(PTable dataTable) throws SQLException {
 if (dataTable.isTransactional()) {
 try {
-phoenixTransactionContext.commitDDLFence(dataTable, logger);
+phoenixTransactionContext.commitDDLFence(dataTable);
 } finally {
 // The client expects a transaction to be in progress on the 
txContext while the
 // VisibilityFence.prepareWait() starts a new tx and 
finishes/aborts it. After it's
@@ -298,14 +296,12 @@ public class MutationState implements SQLCloseable {
 // Though MutationState is not thread safe in general, this method should 
be because it may
 // be called by TableResultIterator in a multi-threaded manner. Since we 
do not want to expose
 // the Transaction outside of MutationState, this seems reasonable, as the 
member variables
-// would not change as these threads are running.
+// would not change as these threads are running. We also clone 
mutationState to ensure that
+// the transaction context won't change due to a commit when auto commit 
is true.
 public Table getHTable(PTable table) throws SQLException {
 Table htable = 
this.getConnection().getQueryServices().getTable(table.getPhysicalName().getBytes());
 if (table.isTransactional() && 
phoenixTransactionContext.isTransactionRunning()) {
-   

[5/5] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
PHOENIX-4605 Support running multiple transaction providers


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 15fa00fa59472c5f61a211af5c723b4803a3cac3
Parents: 5d4cb80
Author: James Taylor 
Authored: Wed Apr 11 20:06:35 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 12 22:30:56 2018 -0700

--
 .../phoenix/end2end/AlterTableWithViewsIT.java  |   6 +-
 .../ConnectionQueryServicesTestImpl.java|  34 +-
 .../phoenix/tx/FlappingTransactionIT.java   |  11 +-
 .../phoenix/tx/ParameterizedTransactionIT.java  |  12 +-
 .../org/apache/phoenix/tx/TransactionIT.java|  12 +
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |   5 +-
 .../phoenix/cache/IndexMetaDataCache.java   |   5 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |   3 +-
 .../apache/phoenix/compile/FromCompiler.java|   2 +-
 .../apache/phoenix/compile/JoinCompiler.java|   2 +-
 .../compile/TupleProjectionCompiler.java|   4 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   6 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   2 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   3 +-
 .../coprocessor/MetaDataEndpointImpl.java   |  72 +-
 .../phoenix/coprocessor/MetaDataProtocol.java   |   3 +-
 .../coprocessor/MetaDataRegionObserver.java |   4 +-
 .../PhoenixTransactionalProcessor.java  |  52 --
 .../coprocessor/ServerCachingEndpointImpl.java  |   4 +-
 .../TephraTransactionalProcessor.java   |  52 ++
 .../UngroupedAggregateRegionObserver.java   |  10 +-
 .../coprocessor/generated/PTableProtos.java | 110 ++-
 .../phoenix/exception/SQLExceptionCode.java |   4 +
 .../apache/phoenix/execute/BaseQueryPlan.java   |   3 +
 .../apache/phoenix/execute/MutationState.java   |  75 +-
 .../PhoenixTxIndexMutationGenerator.java|  10 +-
 .../phoenix/expression/ExpressionType.java  | 119 +--
 .../TransactionProviderNameFunction.java|  81 ++
 .../apache/phoenix/index/IndexMaintainer.java   |   4 +-
 .../index/IndexMetaDataCacheFactory.java|   2 +-
 .../apache/phoenix/index/PhoenixIndexCodec.java |   1 -
 .../index/PhoenixIndexMetaDataBuilder.java  |   7 +-
 .../NonAggregateRegionScannerFactory.java   |   5 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  12 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   2 +-
 .../index/PhoenixIndexPartialBuildMapper.java   |   4 +-
 .../phoenix/query/ConnectionQueryServices.java  |   4 +
 .../query/ConnectionQueryServicesImpl.java  |  69 +-
 .../query/ConnectionlessQueryServicesImpl.java  |   9 +-
 .../query/DelegateConnectionQueryServices.java  |   7 +-
 .../apache/phoenix/query/QueryConstants.java| 108 ++-
 .../org/apache/phoenix/query/QueryServices.java |   1 +
 .../phoenix/query/QueryServicesOptions.java |  10 +-
 .../apache/phoenix/schema/DelegateTable.java|   8 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 129 ++-
 .../java/org/apache/phoenix/schema/PTable.java  |   2 +
 .../org/apache/phoenix/schema/PTableImpl.java   |  67 +-
 .../apache/phoenix/schema/TableProperty.java|  18 +
 .../transaction/OmidTransactionContext.java |  56 +-
 .../transaction/OmidTransactionProvider.java|  54 +-
 .../transaction/OmidTransactionTable.java   | 380 -
 .../transaction/PhoenixTransactionClient.java   |  23 +
 .../transaction/PhoenixTransactionContext.java  | 165 ++--
 .../transaction/PhoenixTransactionProvider.java |  51 ++
 .../transaction/PhoenixTransactionService.java  |  24 +
 .../transaction/PhoenixTransactionalTable.java  | 798 ---
 .../transaction/TephraTransactionContext.java   | 198 +
 .../transaction/TephraTransactionProvider.java  | 161 +++-
 .../transaction/TephraTransactionTable.java | 360 -
 .../phoenix/transaction/TransactionFactory.java |  57 +-
 .../transaction/TransactionProvider.java|  36 -
 .../org/apache/phoenix/util/PhoenixRuntime.java |   3 +-
 .../java/org/apache/phoenix/util/ScanUtil.java  |  15 +
 .../apache/phoenix/util/TransactionUtil.java|  49 +-
 .../phoenix/execute/CorrelatePlanTest.java  |   4 +-
 .../execute/LiteralResultIteratorPlanTest.java  |   5 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  26 +-
 .../phoenix/query/QueryServicesTestImpl.java|  15 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |   2 +-
 phoenix-protocol/src/main/PTable.proto  |   1 +
 70 files changed, 1320 insertions(+), 2338 deletions(-)
--


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

[1/5] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/5.x-HBase-2.0 5d4cb8041 -> 15fa00fa5


http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java 
b/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
index ce70dd9..dee02d1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/TransactionUtil.java
@@ -25,42 +25,57 @@ import java.util.Map;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Table;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.transaction.PhoenixTransactionContext;
-import org.apache.phoenix.transaction.PhoenixTransactionalTable;
-import org.apache.phoenix.transaction.TephraTransactionTable;
 import org.apache.phoenix.transaction.TransactionFactory;
-import org.apache.tephra.util.TxUtils;
 
 public class TransactionUtil {
+// All transaction providers must use an empty byte array as the family 
delete marker
+// (see TxConstants.FAMILY_DELETE_QUALIFIER)
+public static final byte[] FAMILY_DELETE_MARKER = 
HConstants.EMPTY_BYTE_ARRAY;
+// All transaction providers must multiply timestamps by this constant.
+// (see TxConstants.MAX_TX_PER_MS)
+public static final int MAX_TRANSACTIONS_PER_MILLISECOND = 100;
+// Constant used to empirically determine if a timestamp is a 
transactional or
+// non transactional timestamp (see TxUtils.MAX_NON_TX_TIMESTAMP)
+private static final long MAX_NON_TX_TIMESTAMP = (long) 
(System.currentTimeMillis() * 1.1);
+
 private TransactionUtil() {
+
 }
 
 public static boolean isTransactionalTimestamp(long ts) {
-return !TxUtils.isPreExistingVersion(ts);
+return ts >= MAX_NON_TX_TIMESTAMP;
 }
 
 public static boolean isDelete(Cell cell) {
-return (CellUtil.matchingValue(cell, HConstants.EMPTY_BYTE_ARRAY));
+return CellUtil.matchingValue(cell, HConstants.EMPTY_BYTE_ARRAY);
 }
 
-public static long convertToNanoseconds(long serverTimeStamp) {
-return serverTimeStamp * 
TransactionFactory.getTransactionProvider().getTransactionContext().getMaxTransactionsPerSecond();
+public static boolean isDeleteFamily(Cell cell) {
+return CellUtil.matchingQualifier(cell, FAMILY_DELETE_MARKER) && 
CellUtil.matchingValue(cell, HConstants.EMPTY_BYTE_ARRAY);
 }
 
-public static long convertToMilliseconds(long serverTimeStamp) {
-return serverTimeStamp / 
TransactionFactory.getTransactionProvider().getTransactionContext().getMaxTransactionsPerSecond();
+private static Cell newDeleteFamilyMarker(byte[] row, byte[] family, long 
timestamp) {
+return CellUtil.createCell(row, family, FAMILY_DELETE_MARKER, 
timestamp, KeyValue.Type.Put.getCode(), HConstants.EMPTY_BYTE_ARRAY);
 }
 
-public static PhoenixTransactionalTable 
getPhoenixTransactionTable(PhoenixTransactionContext phoenixTransactionContext, 
Table htable, PTable pTable) {
-return new TephraTransactionTable(phoenixTransactionContext, htable, 
pTable);
+private static Cell newDeleteColumnMarker(byte[] row, byte[] family, 
byte[] qualifier, long timestamp) {
+return CellUtil.createCell(row, family, qualifier, timestamp, 
KeyValue.Type.Put.getCode(), HConstants.EMPTY_BYTE_ARRAY);
+}
+
+public static long convertToNanoseconds(long serverTimeStamp) {
+return serverTimeStamp * MAX_TRANSACTIONS_PER_MILLISECOND;
+}
+
+public static long convertToMilliseconds(long serverTimeStamp) {
+return serverTimeStamp / MAX_TRANSACTIONS_PER_MILLISECOND;
 }
 
 // we resolve transactional tables at the txn read pointer
@@ -83,14 +98,14 @@ public class TransactionUtil {
return  txInProgress ? 
convertToMilliseconds(mutationState.getInitialWritePointer()) : 
result.getMutationTime();
}
 
-   public static Long getTableTimestamp(PhoenixConnection connection, 
boolean transactional) throws SQLException {
+   public static Long getTableTimestamp(PhoenixConnection connection, 
boolean transactional, TransactionFactory.Provider provider) throws 
SQLException {
Long timestamp = null;
if (!transactional) {

[2/5] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionalTable.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionalTable.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionalTable.java
deleted file mode 100644
index deceac6..000
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/transaction/PhoenixTransactionalTable.java
+++ /dev/null
@@ -1,798 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.phoenix.transaction;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.CompareOperator;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
-import org.apache.hadoop.hbase.client.Row;
-import org.apache.hadoop.hbase.client.RowMutations;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.client.TableBuilder;
-import org.apache.hadoop.hbase.client.TableDescriptor;
-import org.apache.hadoop.hbase.client.coprocessor.Batch;
-import org.apache.hadoop.hbase.filter.CompareFilter;
-import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
-
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
-public interface PhoenixTransactionalTable extends Table {
-/**
- * Gets the fully qualified table name instance of this table.
- */
-TableName getName();
-
-/**
- * Returns the {@link org.apache.hadoop.conf.Configuration} object used by 
this instance.
- * 
- * The reference returned is not a copy, so any change made to it will
- * affect this instance.
- */
-Configuration getConfiguration();
-
-/**
- * Gets the {@link org.apache.hadoop.hbase.HTableDescriptor table 
descriptor} for this table.
- * @throws java.io.IOException if a remote or network exception occurs.
- * @deprecated since 2.0 version and will be removed in 3.0 version.
- * use {@link #getDescriptor()}
- */
-@Deprecated
-HTableDescriptor getTableDescriptor() throws IOException;
-
-/**
- * Gets the {@link org.apache.hadoop.hbase.client.TableDescriptor table 
descriptor} for this table.
- * @throws java.io.IOException if a remote or network exception occurs.
- */
-TableDescriptor getDescriptor() throws IOException;
-
-/**
- * Test for the existence of columns in the table, as specified by the Get.
- * 
- *
- * This will return true if the Get matches one or more keys, false if not.
- * 
- *
- * This is a server-side call so it prevents any data from being 
transfered to
- * the client.
- *
- * @param get the Get
- * @return true if the specified Get matches one or more keys, false if not
- * @throws IOException e
- */
-boolean exists(Get get) throws IOException;
-
-/**
- * Test for the existence of columns in the table, as specified by the 
Gets.
- * 
- *
- * This will return an array of booleans. Each value will be true if the 
related Get matches
- * one or more keys, false if not.
- * 
- *
- * This is a server-side call so it prevents any data from being 
transferred to
- * the client.
- *
- * @param gets the Gets
- * @return Array of boolean.  True if the specified Get matches one or 
more keys, false if not.
- * @throws

[3/5] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-12 Thread jamestaylor
http://git-wip-us.apache.org/repos/asf/phoenix/blob/15fa00fa/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 1c3bff6..e062729 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
@@ -82,6 +82,7 @@ import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SEQ_NUM;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTIONAL;
+import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTION_PROVIDER;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE;
 import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY;
 import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.USE_STATS_FOR_PARALLELIZATION;
@@ -224,6 +225,9 @@ import org.apache.phoenix.schema.types.PUnsignedLong;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.transaction.PhoenixTransactionContext;
+import org.apache.phoenix.transaction.PhoenixTransactionProvider;
+import org.apache.phoenix.transaction.TransactionFactory;
+import org.apache.phoenix.transaction.TransactionFactory.Provider;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.CursorUtil;
 import org.apache.phoenix.util.EncodedColumnsUtil;
@@ -282,7 +286,7 @@ public class MetaDataClient {
 INDEX_TYPE + "," +
 STORE_NULLS + "," +
 BASE_COLUMN_COUNT + "," +
-TRANSACTIONAL + "," +
+TRANSACTION_PROVIDER + "," +
 UPDATE_CACHE_FREQUENCY + "," +
 IS_NAMESPACE_MAPPED + "," +
 AUTO_PARTITION_SEQ +  "," +
@@ -574,14 +578,11 @@ public class MetaDataClient {
 } catch (TableNotFoundException e) {
 }
 
-boolean defaultTransactional = 
connection.getQueryServices().getProps().getBoolean(
-QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
-QueryServicesOptions.DEFAULT_TRANSACTIONAL);
 // start a txn if all table are transactional by default or if we 
found the table in the cache and it is transactional
 // TODO if system tables become transactional remove the check
-boolean isTransactional = defaultTransactional || (table!=null && 
table.isTransactional());
-if (!systemTable && isTransactional && 
!connection.getMutationState().isTransactionStarted()) {
-connection.getMutationState().startTransaction();
+boolean isTransactional = (table!=null && table.isTransactional());
+if (isTransactional) {
+
connection.getMutationState().startTransaction(table.getTransactionProvider());
 }
 resolvedTimestamp = resolvedTimestamp==null ? 
TransactionUtil.getResolvedTimestamp(connection, isTransactional, 
HConstants.LATEST_TIMESTAMP) : resolvedTimestamp;
 // Do not make rpc to getTable if
@@ -634,16 +635,20 @@ public class MetaDataClient {
 result =
 queryServices.getTable(tenantId, schemaBytes, 
tableBytes, tableTimestamp,
 resolvedTimestamp);
-// if the table was assumed to be transactional, but is 
actually not transactional
-// then re-resolve as of the right timestamp (and vice versa)
-if (table == null && result.getTable() != null
-&& result.getTable().isTransactional() != 
isTransactional) {
-result =
-queryServices.getTable(tenantId, schemaBytes, 
tableBytes,
-tableTimestamp,
-
TransactionUtil.getResolvedTimestamp(connection,
-result.getTable().isTransactional(),
-HConstants.LATEST_TIMESTAMP));
+// if the table was assumed to be non transactional, but is 
actually transactional
+// then re-resolve as of the right timestamp
+if (result.getTable() != null
+&& result.getTable().isTransactional()
+&& !isTransactional) {
+long resolveTimestamp = 
TransactionUtil.getResolvedTimestamp(connection,
+result.getTable().isTransactional(),
+HConstants.LATEST_TIMESTAMP);
+// Reresolve if table timestamp is p

phoenix git commit: Revert "PHOENIX-4496 Fix RowValueConstructorIT and IndexMetadataIT"

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/master 70c9c6bb2 -> 7b2c7e135


Revert "PHOENIX-4496 Fix RowValueConstructorIT and IndexMetadataIT"

This reverts commit 70c9c6bb2d2073bb40640a0a794032b166c4b63b.


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

Branch: refs/heads/master
Commit: 7b2c7e135916d5a6087ec3e6c4b6c261779795ac
Parents: 70c9c6b
Author: James Taylor 
Authored: Fri Apr 13 10:10:07 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 10:10:07 2018 -0700

--
 .../phoenix/hbase/index/scanner/ScannerBuilder.java   | 10 --
 1 file changed, 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7b2c7e13/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
index 4c42fe4..ad09c0c 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/scanner/ScannerBuilder.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.filter.BinaryComparator;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.FamilyFilter;
 import org.apache.hadoop.hbase.filter.Filter;
-import org.apache.hadoop.hbase.filter.FilterBase;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.QualifierFilter;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -99,15 +98,6 @@ public class ScannerBuilder {
   }
   columnFilters.addFilter(columnFilter);
 }
-
-if(columns.isEmpty()){
-columnFilters.addFilter(new FilterBase(){
-@Override
-public boolean filterAllRemaining() throws IOException {
-return true;
-}
-});
-}
 return columnFilters;
   }
 



[1/2] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/master 7b2c7e135 -> 87fdda8b1


http://git-wip-us.apache.org/repos/asf/phoenix/blob/87fdda8b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 914f62a..8b328d3 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -170,6 +170,7 @@ import 
org.apache.phoenix.exception.RetriableUpgradeException;
 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.execute.MutationState;
 import org.apache.phoenix.hbase.index.IndexRegionSplitPolicy;
@@ -1041,23 +1042,69 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 
 /**
  *
- * @param tableName
+ * @param physicalTableName
+ * @param tableType
+ * @param props
+ * @param families
  * @param splits
- * @param modifyExistingMetaData TODO
+ * @param modifyExistingMetaData
+ * @param isNamespaceMapped
+ * @param isDoNotUpgradePropSet
  * @return true if table was created and false if it already exists
  * @throws SQLException
  */
 private HTableDescriptor ensureTableCreated(byte[] physicalTableName, 
PTableType tableType, Map props,
 List>> families, byte[][] splits, 
boolean modifyExistingMetaData,
-boolean isNamespaceMapped) throws SQLException {
+boolean isNamespaceMapped, boolean isDoNotUpgradePropSet) throws 
SQLException {
 SQLException sqlE = null;
 HTableDescriptor existingDesc = null;
 boolean isMetaTable = SchemaUtil.isMetaTable(physicalTableName);
 boolean tableExist = true;
 try (HBaseAdmin admin = getAdmin()) {
 final String quorum = ZKConfig.getZKQuorumServersString(config);
-final String znode = 
this.props.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
+final String znode = 
this.getProps().get(HConstants.ZOOKEEPER_ZNODE_PARENT);
 logger.debug("Found quorum: " + quorum + ":" + znode);
+
+if (isMetaTable) {
+if(SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, 
this.getProps())) {
+try {
+// SYSTEM namespace needs to be created via HBase APIs 
because "CREATE SCHEMA" statement tries to write
+// its metadata in SYSTEM:CATALOG table. Without 
SYSTEM namespace, SYSTEM:CATALOG table cannot be created
+
ensureNamespaceCreated(QueryConstants.SYSTEM_SCHEMA_NAME);
+} catch (PhoenixIOException e) {
+// We could either:
+// 1) Not access the NS descriptor. The NS may or may 
not exist at this point
+// 2) We could not create the NS
+// Regardless of the case 1 or 2, if we eventually try 
to migrate SYSTEM tables to the SYSTEM
+// namespace using the {@link 
ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method,
+// if the NS does not exist, we will error as 
expected, or
+// if the NS does exist and tables are already mapped, 
the check will exit gracefully
+}
+if 
(admin.tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, 
false))) {
+// SYSTEM.CATALOG exists, so at this point, we have 3 
cases:
+// 1) If server-side namespace mapping is disabled, 
throw Inconsistent namespace mapping exception
+// 2) If server-side namespace mapping is enabled and 
SYSCAT needs to be upgraded, upgrade SYSCAT
+//and also migrate SYSTEM tables to the SYSTEM 
namespace
+// 3. If server-side namespace mapping is enabled and 
SYSCAT doesn't need to be upgraded, we still
+//need to migrate SYSTEM tables to the SYSTEM 
namespace using the
+//{@link 
ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method (as part of
+//{@link upgradeSystemTables(String, Properties)})
+
checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
+// Thrown so we can force an upgrade which will just 
m

[2/2] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first 
client connection (Chinmay Kulkarni)


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

Branch: refs/heads/master
Commit: 87fdda8b180438e57d0e2f6082e5a9e988220245
Parents: 7b2c7e1
Author: James Taylor 
Authored: Fri Apr 13 10:30:30 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 10:30:30 2018 -0700

--
 .../phoenix/end2end/AppendOnlySchemaIT.java |   2 +-
 .../MigrateSystemTablesToSystemNamespaceIT.java |   9 +-
 .../SystemCatalogCreationOnConnectionIT.java| 626 
 .../coprocessor/MetaDataEndpointImpl.java   |  21 +
 .../phoenix/coprocessor/MetaDataProtocol.java   |   4 +
 .../coprocessor/generated/MetaDataProtos.java   | 183 +++--
 .../exception/UpgradeRequiredException.java |  13 +-
 .../phoenix/query/ConnectionQueryServices.java  |   2 +-
 .../query/ConnectionQueryServicesImpl.java  | 750 +++
 .../query/ConnectionlessQueryServicesImpl.java  |   2 +-
 .../query/DelegateConnectionQueryServices.java  |   4 +-
 .../apache/phoenix/schema/MetaDataClient.java   |   3 +-
 .../query/ConnectionQueryServicesImplTest.java  |   7 +-
 phoenix-protocol/src/main/MetaDataService.proto |   1 +
 14 files changed, 1236 insertions(+), 391 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/87fdda8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
index 7ed64ff..d601beb 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
@@ -114,7 +114,7 @@ public class AppendOnlySchemaIT extends 
ParallelStatsDisabledIT {
 // verify no create table rpcs
 verify(connectionQueryServices, 
never()).createTable(anyListOf(Mutation.class),
 any(byte[].class), any(PTableType.class), anyMap(), anyList(), 
any(byte[][].class),
-eq(false), eq(false));
+eq(false), eq(false), eq(false));
 reset(connectionQueryServices);
 
 // execute alter table ddl that adds the same column

http://git-wip-us.apache.org/repos/asf/phoenix/blob/87fdda8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
index 7cb5857..627e453 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
@@ -376,8 +376,10 @@ public class MigrateSystemTablesToSystemNamespaceIT 
extends BaseTest {
 while(rs.next()) {
 
 if(rs.getString("IS_NAMESPACE_MAPPED") == null) {
+// Check that entry for SYSTEM namespace exists in SYSCAT
 systemSchemaExists = 
rs.getString("TABLE_SCHEM").equals(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME) 
? true : systemSchemaExists;
 } else if (rs.getString("COLUMN_NAME") == null) {
+// Found the intial entry for a table in SYSCAT
 String schemaName = rs.getString("TABLE_SCHEM");
 String tableName = rs.getString("TABLE_NAME");
 
@@ -395,12 +397,11 @@ public class MigrateSystemTablesToSystemNamespaceIT 
extends BaseTest {
 }
 }
 
-if(!systemSchemaExists) {
-fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry doesn't 
exist in SYSTEM.CATALOG table.");
-}
-
 // The set will contain SYSMUTEX table since that table is not exposed 
in SYSCAT
 if (systemTablesMapped) {
+if (!systemSchemaExists) {
+fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry 
doesn't exist in SYSTEM.CATALOG table.");
+}
 assertTrue(namespaceMappedSystemTablesSet.size() == 1);
 } else {
 assertTrue(systemTablesSet.size() == 1);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/87fdda8b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java

[1/2] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.3 f543d99a3 -> 5f3ef7c5d


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5f3ef7c5/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 914f62a..8b328d3 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -170,6 +170,7 @@ import 
org.apache.phoenix.exception.RetriableUpgradeException;
 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.execute.MutationState;
 import org.apache.phoenix.hbase.index.IndexRegionSplitPolicy;
@@ -1041,23 +1042,69 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 
 /**
  *
- * @param tableName
+ * @param physicalTableName
+ * @param tableType
+ * @param props
+ * @param families
  * @param splits
- * @param modifyExistingMetaData TODO
+ * @param modifyExistingMetaData
+ * @param isNamespaceMapped
+ * @param isDoNotUpgradePropSet
  * @return true if table was created and false if it already exists
  * @throws SQLException
  */
 private HTableDescriptor ensureTableCreated(byte[] physicalTableName, 
PTableType tableType, Map props,
 List>> families, byte[][] splits, 
boolean modifyExistingMetaData,
-boolean isNamespaceMapped) throws SQLException {
+boolean isNamespaceMapped, boolean isDoNotUpgradePropSet) throws 
SQLException {
 SQLException sqlE = null;
 HTableDescriptor existingDesc = null;
 boolean isMetaTable = SchemaUtil.isMetaTable(physicalTableName);
 boolean tableExist = true;
 try (HBaseAdmin admin = getAdmin()) {
 final String quorum = ZKConfig.getZKQuorumServersString(config);
-final String znode = 
this.props.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
+final String znode = 
this.getProps().get(HConstants.ZOOKEEPER_ZNODE_PARENT);
 logger.debug("Found quorum: " + quorum + ":" + znode);
+
+if (isMetaTable) {
+if(SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, 
this.getProps())) {
+try {
+// SYSTEM namespace needs to be created via HBase APIs 
because "CREATE SCHEMA" statement tries to write
+// its metadata in SYSTEM:CATALOG table. Without 
SYSTEM namespace, SYSTEM:CATALOG table cannot be created
+
ensureNamespaceCreated(QueryConstants.SYSTEM_SCHEMA_NAME);
+} catch (PhoenixIOException e) {
+// We could either:
+// 1) Not access the NS descriptor. The NS may or may 
not exist at this point
+// 2) We could not create the NS
+// Regardless of the case 1 or 2, if we eventually try 
to migrate SYSTEM tables to the SYSTEM
+// namespace using the {@link 
ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method,
+// if the NS does not exist, we will error as 
expected, or
+// if the NS does exist and tables are already mapped, 
the check will exit gracefully
+}
+if 
(admin.tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, 
false))) {
+// SYSTEM.CATALOG exists, so at this point, we have 3 
cases:
+// 1) If server-side namespace mapping is disabled, 
throw Inconsistent namespace mapping exception
+// 2) If server-side namespace mapping is enabled and 
SYSCAT needs to be upgraded, upgrade SYSCAT
+//and also migrate SYSTEM tables to the SYSTEM 
namespace
+// 3. If server-side namespace mapping is enabled and 
SYSCAT doesn't need to be upgraded, we still
+//need to migrate SYSTEM tables to the SYSTEM 
namespace using the
+//{@link 
ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method (as part of
+//{@link upgradeSystemTables(String, Properties)})
+
checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
+// Thrown so we can force an upgrade which will 

[2/2] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first 
client connection (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-HBase-1.3
Commit: 5f3ef7c5dd7e276f9185a90f81b54729cf632349
Parents: f543d99
Author: James Taylor 
Authored: Fri Apr 13 10:30:30 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 10:33:58 2018 -0700

--
 .../phoenix/end2end/AppendOnlySchemaIT.java |   2 +-
 .../MigrateSystemTablesToSystemNamespaceIT.java |   9 +-
 .../SystemCatalogCreationOnConnectionIT.java| 626 
 .../coprocessor/MetaDataEndpointImpl.java   |  21 +
 .../phoenix/coprocessor/MetaDataProtocol.java   |   4 +
 .../coprocessor/generated/MetaDataProtos.java   | 183 +++--
 .../exception/UpgradeRequiredException.java |  13 +-
 .../phoenix/query/ConnectionQueryServices.java  |   2 +-
 .../query/ConnectionQueryServicesImpl.java  | 750 +++
 .../query/ConnectionlessQueryServicesImpl.java  |   2 +-
 .../query/DelegateConnectionQueryServices.java  |   4 +-
 .../apache/phoenix/schema/MetaDataClient.java   |   3 +-
 .../query/ConnectionQueryServicesImplTest.java  |   7 +-
 phoenix-protocol/src/main/MetaDataService.proto |   1 +
 14 files changed, 1236 insertions(+), 391 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5f3ef7c5/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
index 7ed64ff..d601beb 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
@@ -114,7 +114,7 @@ public class AppendOnlySchemaIT extends 
ParallelStatsDisabledIT {
 // verify no create table rpcs
 verify(connectionQueryServices, 
never()).createTable(anyListOf(Mutation.class),
 any(byte[].class), any(PTableType.class), anyMap(), anyList(), 
any(byte[][].class),
-eq(false), eq(false));
+eq(false), eq(false), eq(false));
 reset(connectionQueryServices);
 
 // execute alter table ddl that adds the same column

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5f3ef7c5/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
index 7cb5857..627e453 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
@@ -376,8 +376,10 @@ public class MigrateSystemTablesToSystemNamespaceIT 
extends BaseTest {
 while(rs.next()) {
 
 if(rs.getString("IS_NAMESPACE_MAPPED") == null) {
+// Check that entry for SYSTEM namespace exists in SYSCAT
 systemSchemaExists = 
rs.getString("TABLE_SCHEM").equals(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME) 
? true : systemSchemaExists;
 } else if (rs.getString("COLUMN_NAME") == null) {
+// Found the intial entry for a table in SYSCAT
 String schemaName = rs.getString("TABLE_SCHEM");
 String tableName = rs.getString("TABLE_NAME");
 
@@ -395,12 +397,11 @@ public class MigrateSystemTablesToSystemNamespaceIT 
extends BaseTest {
 }
 }
 
-if(!systemSchemaExists) {
-fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry doesn't 
exist in SYSTEM.CATALOG table.");
-}
-
 // The set will contain SYSMUTEX table since that table is not exposed 
in SYSCAT
 if (systemTablesMapped) {
+if (!systemSchemaExists) {
+fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry 
doesn't exist in SYSTEM.CATALOG table.");
+}
 assertTrue(namespaceMappedSystemTablesSet.size() == 1);
 } else {
 assertTrue(systemTablesSet.size() == 1);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5f3ef7c5/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
-

[1/2] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.2 c7f0a55e9 -> 6fb7d45a7


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6fb7d45a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 914f62a..8b328d3 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -170,6 +170,7 @@ import 
org.apache.phoenix.exception.RetriableUpgradeException;
 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.execute.MutationState;
 import org.apache.phoenix.hbase.index.IndexRegionSplitPolicy;
@@ -1041,23 +1042,69 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 
 /**
  *
- * @param tableName
+ * @param physicalTableName
+ * @param tableType
+ * @param props
+ * @param families
  * @param splits
- * @param modifyExistingMetaData TODO
+ * @param modifyExistingMetaData
+ * @param isNamespaceMapped
+ * @param isDoNotUpgradePropSet
  * @return true if table was created and false if it already exists
  * @throws SQLException
  */
 private HTableDescriptor ensureTableCreated(byte[] physicalTableName, 
PTableType tableType, Map props,
 List>> families, byte[][] splits, 
boolean modifyExistingMetaData,
-boolean isNamespaceMapped) throws SQLException {
+boolean isNamespaceMapped, boolean isDoNotUpgradePropSet) throws 
SQLException {
 SQLException sqlE = null;
 HTableDescriptor existingDesc = null;
 boolean isMetaTable = SchemaUtil.isMetaTable(physicalTableName);
 boolean tableExist = true;
 try (HBaseAdmin admin = getAdmin()) {
 final String quorum = ZKConfig.getZKQuorumServersString(config);
-final String znode = 
this.props.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
+final String znode = 
this.getProps().get(HConstants.ZOOKEEPER_ZNODE_PARENT);
 logger.debug("Found quorum: " + quorum + ":" + znode);
+
+if (isMetaTable) {
+if(SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, 
this.getProps())) {
+try {
+// SYSTEM namespace needs to be created via HBase APIs 
because "CREATE SCHEMA" statement tries to write
+// its metadata in SYSTEM:CATALOG table. Without 
SYSTEM namespace, SYSTEM:CATALOG table cannot be created
+
ensureNamespaceCreated(QueryConstants.SYSTEM_SCHEMA_NAME);
+} catch (PhoenixIOException e) {
+// We could either:
+// 1) Not access the NS descriptor. The NS may or may 
not exist at this point
+// 2) We could not create the NS
+// Regardless of the case 1 or 2, if we eventually try 
to migrate SYSTEM tables to the SYSTEM
+// namespace using the {@link 
ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method,
+// if the NS does not exist, we will error as 
expected, or
+// if the NS does exist and tables are already mapped, 
the check will exit gracefully
+}
+if 
(admin.tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, 
false))) {
+// SYSTEM.CATALOG exists, so at this point, we have 3 
cases:
+// 1) If server-side namespace mapping is disabled, 
throw Inconsistent namespace mapping exception
+// 2) If server-side namespace mapping is enabled and 
SYSCAT needs to be upgraded, upgrade SYSCAT
+//and also migrate SYSTEM tables to the SYSTEM 
namespace
+// 3. If server-side namespace mapping is enabled and 
SYSCAT doesn't need to be upgraded, we still
+//need to migrate SYSTEM tables to the SYSTEM 
namespace using the
+//{@link 
ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method (as part of
+//{@link upgradeSystemTables(String, Properties)})
+
checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
+// Thrown so we can force an upgrade which will 

[2/2] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first 
client connection (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-HBase-1.2
Commit: 6fb7d45a7aff3212e15a552dc5d23436c284b2f9
Parents: c7f0a55
Author: James Taylor 
Authored: Fri Apr 13 10:30:30 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 10:36:56 2018 -0700

--
 .../phoenix/end2end/AppendOnlySchemaIT.java |   2 +-
 .../MigrateSystemTablesToSystemNamespaceIT.java |   9 +-
 .../SystemCatalogCreationOnConnectionIT.java| 626 
 .../coprocessor/MetaDataEndpointImpl.java   |  21 +
 .../phoenix/coprocessor/MetaDataProtocol.java   |   4 +
 .../coprocessor/generated/MetaDataProtos.java   | 183 +++--
 .../exception/UpgradeRequiredException.java |  13 +-
 .../phoenix/query/ConnectionQueryServices.java  |   2 +-
 .../query/ConnectionQueryServicesImpl.java  | 750 +++
 .../query/ConnectionlessQueryServicesImpl.java  |   2 +-
 .../query/DelegateConnectionQueryServices.java  |   4 +-
 .../apache/phoenix/schema/MetaDataClient.java   |   3 +-
 .../query/ConnectionQueryServicesImplTest.java  |   7 +-
 phoenix-protocol/src/main/MetaDataService.proto |   1 +
 14 files changed, 1236 insertions(+), 391 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6fb7d45a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
index 7ed64ff..d601beb 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
@@ -114,7 +114,7 @@ public class AppendOnlySchemaIT extends 
ParallelStatsDisabledIT {
 // verify no create table rpcs
 verify(connectionQueryServices, 
never()).createTable(anyListOf(Mutation.class),
 any(byte[].class), any(PTableType.class), anyMap(), anyList(), 
any(byte[][].class),
-eq(false), eq(false));
+eq(false), eq(false), eq(false));
 reset(connectionQueryServices);
 
 // execute alter table ddl that adds the same column

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6fb7d45a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
index 7cb5857..627e453 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
@@ -376,8 +376,10 @@ public class MigrateSystemTablesToSystemNamespaceIT 
extends BaseTest {
 while(rs.next()) {
 
 if(rs.getString("IS_NAMESPACE_MAPPED") == null) {
+// Check that entry for SYSTEM namespace exists in SYSCAT
 systemSchemaExists = 
rs.getString("TABLE_SCHEM").equals(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME) 
? true : systemSchemaExists;
 } else if (rs.getString("COLUMN_NAME") == null) {
+// Found the intial entry for a table in SYSCAT
 String schemaName = rs.getString("TABLE_SCHEM");
 String tableName = rs.getString("TABLE_NAME");
 
@@ -395,12 +397,11 @@ public class MigrateSystemTablesToSystemNamespaceIT 
extends BaseTest {
 }
 }
 
-if(!systemSchemaExists) {
-fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry doesn't 
exist in SYSTEM.CATALOG table.");
-}
-
 // The set will contain SYSMUTEX table since that table is not exposed 
in SYSCAT
 if (systemTablesMapped) {
+if (!systemSchemaExists) {
+fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry 
doesn't exist in SYSTEM.CATALOG table.");
+}
 assertTrue(namespaceMappedSystemTablesSet.size() == 1);
 } else {
 assertTrue(systemTablesSet.size() == 1);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/6fb7d45a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
-

[1/6] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-cdh5.11 382307a7d -> 2a3c9c94e


http://git-wip-us.apache.org/repos/asf/phoenix/blob/02715adc/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
deleted file mode 100644
index c191d8d..000
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TephraTransactionTable.java
+++ /dev/null
@@ -1,350 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.phoenix.transaction;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Append;
-import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.Durability;
-import org.apache.hadoop.hbase.client.Get;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Increment;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Row;
-import org.apache.hadoop.hbase.client.RowMutations;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.Call;
-import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
-import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
-import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
-import org.apache.tephra.TxConstants;
-import org.apache.tephra.hbase.TransactionAwareHTable;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableType;
-
-import com.google.protobuf.Descriptors.MethodDescriptor;
-import com.google.protobuf.Message;
-import com.google.protobuf.Service;
-import com.google.protobuf.ServiceException;
-
-public class TephraTransactionTable implements PhoenixTransactionalTable {
-
-private TransactionAwareHTable transactionAwareHTable;
-
-private TephraTransactionContext tephraTransactionContext;
-
-public TephraTransactionTable(PhoenixTransactionContext ctx, 
HTableInterface hTable) {
-this(ctx, hTable, null);
-}
-
-public TephraTransactionTable(PhoenixTransactionContext ctx, 
HTableInterface hTable, PTable pTable) {
-
-assert(ctx instanceof TephraTransactionContext);
-
-tephraTransactionContext = (TephraTransactionContext) ctx;
-
-transactionAwareHTable = new TransactionAwareHTable(hTable, (pTable != 
null && pTable.isImmutableRows()) ? TxConstants.ConflictDetection.NONE : 
TxConstants.ConflictDetection.ROW);
-
-tephraTransactionContext.addTransactionAware(transactionAwareHTable);
-
-if (pTable != null && pTable.getType() != PTableType.INDEX) {
-tephraTransactionContext.markDMLFence(pTable);
-}
-}
-
-@Override
-public Result get(Get get) throws IOException {
-return transactionAwareHTable.get(get);
-}
-
-@Override
-public void put(Put put) throws IOException {
-transactionAwareHTable.put(put);
-}
-
-@Override
-public void delete(Delete delete) throws IOException {
-transactionAwareHTable.delete(delete);
-}
-
-@Override
-public ResultScanner getScanner(Scan scan) throws IOException {
-return transactionAwareHTable.getScanner(scan);
-}
-
-@Override
-public byte[] getTableName() {
-return transactionAwareHTable.getTableName();
-}
-
-@Override
-public Configuration getConfiguration() {
-return transactionAwareHTable.getConfiguration();
-}
-
-@Override
-public HTableDescriptor getTableDescriptor() throws IOException {
-return transactionAwareHTable.getTableDescriptor();
-}
-
-@Override
-public boolean exists(Get get) throws IOException {
-return transactionAwareHTable.exists(

[5/6] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a3c9c94/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 5f42e72..a94c083 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -169,6 +169,7 @@ import 
org.apache.phoenix.exception.RetriableUpgradeException;
 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.execute.MutationState;
 import org.apache.phoenix.hbase.index.IndexRegionSplitPolicy;
@@ -1023,23 +1024,69 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 
 /**
  *
- * @param tableName
+ * @param physicalTableName
+ * @param tableType
+ * @param props
+ * @param families
  * @param splits
- * @param modifyExistingMetaData TODO
+ * @param modifyExistingMetaData
+ * @param isNamespaceMapped
+ * @param isDoNotUpgradePropSet
  * @return true if table was created and false if it already exists
  * @throws SQLException
  */
 private HTableDescriptor ensureTableCreated(byte[] physicalTableName, 
PTableType tableType, Map props,
 List>> families, byte[][] splits, 
boolean modifyExistingMetaData,
-boolean isNamespaceMapped) throws SQLException {
+boolean isNamespaceMapped, boolean isDoNotUpgradePropSet) throws 
SQLException {
 SQLException sqlE = null;
 HTableDescriptor existingDesc = null;
 boolean isMetaTable = SchemaUtil.isMetaTable(physicalTableName);
 boolean tableExist = true;
 try (HBaseAdmin admin = getAdmin()) {
 final String quorum = ZKConfig.getZKQuorumServersString(config);
-final String znode = 
this.props.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
+final String znode = 
this.getProps().get(HConstants.ZOOKEEPER_ZNODE_PARENT);
 logger.debug("Found quorum: " + quorum + ":" + znode);
+
+if (isMetaTable) {
+if(SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, 
this.getProps())) {
+try {
+// SYSTEM namespace needs to be created via HBase APIs 
because "CREATE SCHEMA" statement tries to write
+// its metadata in SYSTEM:CATALOG table. Without 
SYSTEM namespace, SYSTEM:CATALOG table cannot be created
+
ensureNamespaceCreated(QueryConstants.SYSTEM_SCHEMA_NAME);
+} catch (PhoenixIOException e) {
+// We could either:
+// 1) Not access the NS descriptor. The NS may or may 
not exist at this point
+// 2) We could not create the NS
+// Regardless of the case 1 or 2, if we eventually try 
to migrate SYSTEM tables to the SYSTEM
+// namespace using the {@link 
ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method,
+// if the NS does not exist, we will error as 
expected, or
+// if the NS does exist and tables are already mapped, 
the check will exit gracefully
+}
+if 
(admin.tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, 
false))) {
+// SYSTEM.CATALOG exists, so at this point, we have 3 
cases:
+// 1) If server-side namespace mapping is disabled, 
throw Inconsistent namespace mapping exception
+// 2) If server-side namespace mapping is enabled and 
SYSCAT needs to be upgraded, upgrade SYSCAT
+//and also migrate SYSTEM tables to the SYSTEM 
namespace
+// 3. If server-side namespace mapping is enabled and 
SYSCAT doesn't need to be upgraded, we still
+//need to migrate SYSTEM tables to the SYSTEM 
namespace using the
+//{@link 
ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method (as part of
+//{@link upgradeSystemTables(String, Properties)})
+
checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
+// Thrown so we can force an upgrade which will just 
migrate SYSTEM tables to the SYSTEM namespace
+throw new 
Up

[2/6] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-13 Thread jamestaylor
http://git-wip-us.apache.org/repos/asf/phoenix/blob/02715adc/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index a7b31e8..1a11427 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -69,6 +69,7 @@ import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.types.PFloat;
 import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -134,7 +135,7 @@ public class PTableImpl implements PTable {
 private boolean disableWAL;
 private boolean multiTenant;
 private boolean storeNulls;
-private boolean isTransactional;
+private TransactionFactory.Provider transactionProvider;
 private ViewType viewType;
 private Short viewIndexId;
 private int estimatedSize;
@@ -227,7 +228,7 @@ public class PTableImpl implements PTable {
 init(tenantId, this.schemaName, this.tableName, PTableType.INDEX, 
state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
 this.schemaName, parentTableName, indexes, isImmutableRows, 
physicalNames, defaultFamilyName,
 null, disableWAL, multiTenant, storeNulls, viewType, viewIndexId, 
indexType, baseColumnCount, rowKeyOrderOptimizable,
-isTransactional, updateCacheFrequency, indexDisableTimestamp, 
isNamespaceMpped, null, false, storageScheme, qualifierEncodingScheme, 
encodedCQCounter, useStatsForParallelization);
+transactionProvider, updateCacheFrequency, indexDisableTimestamp, 
isNamespaceMpped, null, false, storageScheme, qualifierEncodingScheme, 
encodedCQCounter, useStatsForParallelization);
 }
 
 public PTableImpl(long timeStamp) { // For delete marker
@@ -270,7 +271,7 @@ public class PTableImpl implements PTable {
 table.getSequenceNumber(), table.getPKName(), 
table.getBucketNum(), getColumnsToClone(table), table.getParentSchemaName(), 
table.getParentTableName(),
 indexes, table.isImmutableRows(), physicalNames, 
table.getDefaultFamilyName(), viewStatement,
 table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), 
table.getIndexType(),
-table.getBaseColumnCount(), 
table.rowKeyOrderOptimizable(), table.isTransactional(), updateCacheFrequency,
+table.getBaseColumnCount(), 
table.rowKeyOrderOptimizable(), table.getTransactionProvider(), 
updateCacheFrequency,
 table.getIndexDisableTimestamp(), 
table.isNamespaceMapped(), table.getAutoPartitionSeqName(), 
table.isAppendOnlySchema(), table.getImmutableStorageScheme(), 
table.getEncodingScheme(), table.getEncodedCQCounter(), 
table.useStatsForParallelization());
 }
 
@@ -280,7 +281,7 @@ public class PTableImpl implements PTable {
 table.getSequenceNumber(), table.getPKName(), 
table.getBucketNum(), getColumnsToClone(table), parentSchemaName, 
table.getParentTableName(),
 indexes, table.isImmutableRows(), table.getPhysicalNames(), 
table.getDefaultFamilyName(), viewStatement,
 table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), 
table.getIndexType(),
-table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), 
table.isTransactional(), table.getUpdateCacheFrequency(),
+table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), 
table.getTransactionProvider(), table.getUpdateCacheFrequency(),
 table.getIndexDisableTimestamp(), table.isNamespaceMapped(), 
table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), 
table.getImmutableStorageScheme(), table.getEncodingScheme(), 
table.getEncodedCQCounter(), table.useStatsForParallelization());
 }
 
@@ -290,7 +291,7 @@ public class PTableImpl implements PTable {
 table.getSequenceNumber(), table.getPKName(), 
table.getBucketNum(), columns, table.getParentSchemaName(), 
table.getParentTableName(),
 table.getIndexes(), table.isImmutableRows(), 
table.getPhysicalNames(), table.getDefaultFamilyName(), 
table.getViewStatement(),
 table.isWALDisabled(), table.isMultiTenant(), 
table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), 
table.getIndexType(),
-table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), 
table.isTransactional(), table.getUpdateCacheFrequency(),

[6/6] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first 
client connection (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-cdh5.11
Commit: 2a3c9c94eb94ce53cf164066df2cb3b62187a1cf
Parents: 02715ad
Author: James Taylor 
Authored: Fri Apr 13 10:30:30 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 10:41:15 2018 -0700

--
 .../phoenix/end2end/AppendOnlySchemaIT.java |   2 +-
 .../MigrateSystemTablesToSystemNamespaceIT.java |   9 +-
 .../SystemCatalogCreationOnConnectionIT.java| 626 
 .../coprocessor/MetaDataEndpointImpl.java   |  21 +
 .../phoenix/coprocessor/MetaDataProtocol.java   |   4 +
 .../coprocessor/generated/MetaDataProtos.java   | 183 +++--
 .../exception/UpgradeRequiredException.java |  13 +-
 .../phoenix/query/ConnectionQueryServices.java  |   2 +-
 .../query/ConnectionQueryServicesImpl.java  | 751 ++-
 .../query/ConnectionlessQueryServicesImpl.java  |   2 +-
 .../query/DelegateConnectionQueryServices.java  |   4 +-
 .../apache/phoenix/schema/MetaDataClient.java   |   3 +-
 .../query/ConnectionQueryServicesImplTest.java  |   7 +-
 phoenix-protocol/src/main/MetaDataService.proto |   1 +
 14 files changed, 1231 insertions(+), 397 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a3c9c94/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
index 7ed64ff..d601beb 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
@@ -114,7 +114,7 @@ public class AppendOnlySchemaIT extends 
ParallelStatsDisabledIT {
 // verify no create table rpcs
 verify(connectionQueryServices, 
never()).createTable(anyListOf(Mutation.class),
 any(byte[].class), any(PTableType.class), anyMap(), anyList(), 
any(byte[][].class),
-eq(false), eq(false));
+eq(false), eq(false), eq(false));
 reset(connectionQueryServices);
 
 // execute alter table ddl that adds the same column

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a3c9c94/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
index 0d64ca9..9047c6b 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
@@ -373,8 +373,10 @@ public class MigrateSystemTablesToSystemNamespaceIT 
extends BaseTest {
 while(rs.next()) {
 
 if(rs.getString("IS_NAMESPACE_MAPPED") == null) {
+// Check that entry for SYSTEM namespace exists in SYSCAT
 systemSchemaExists = 
rs.getString("TABLE_SCHEM").equals(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME) 
? true : systemSchemaExists;
 } else if (rs.getString("COLUMN_NAME") == null) {
+// Found the intial entry for a table in SYSCAT
 String schemaName = rs.getString("TABLE_SCHEM");
 String tableName = rs.getString("TABLE_NAME");
 
@@ -392,12 +394,11 @@ public class MigrateSystemTablesToSystemNamespaceIT 
extends BaseTest {
 }
 }
 
-if(!systemSchemaExists) {
-fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry doesn't 
exist in SYSTEM.CATALOG table.");
-}
-
 // The set will contain SYSMUTEX table since that table is not exposed 
in SYSCAT
 if (systemTablesMapped) {
+if (!systemSchemaExists) {
+fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry 
doesn't exist in SYSTEM.CATALOG table.");
+}
 assertTrue(namespaceMappedSystemTablesSet.size() == 1);
 } else {
 assertTrue(systemTablesSet.size() == 1);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a3c9c94/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
---

[4/6] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-13 Thread jamestaylor
PHOENIX-4605 Support running multiple transaction providers


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

Branch: refs/heads/4.x-cdh5.11
Commit: 02715adc147db3684b8f1c82e21a6033af2d71e4
Parents: 382307a
Author: James Taylor 
Authored: Wed Apr 11 20:06:35 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 12 18:23:37 2018 -0700

--
 .../phoenix/end2end/AlterTableWithViewsIT.java  |   6 +-
 .../ConnectionQueryServicesTestImpl.java|  34 +-
 .../phoenix/tx/FlappingTransactionIT.java   |  11 +-
 .../phoenix/tx/ParameterizedTransactionIT.java  |  14 +-
 .../org/apache/phoenix/tx/TransactionIT.java|  12 +
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |   5 +-
 .../phoenix/cache/IndexMetaDataCache.java   |   5 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |   3 +-
 .../apache/phoenix/compile/FromCompiler.java|   2 +-
 .../apache/phoenix/compile/JoinCompiler.java|   2 +-
 .../compile/TupleProjectionCompiler.java|   4 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   6 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   2 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   4 +-
 .../coprocessor/MetaDataEndpointImpl.java   |  26 +-
 .../phoenix/coprocessor/MetaDataProtocol.java   |   2 +-
 .../coprocessor/MetaDataRegionObserver.java |   4 +-
 .../PhoenixTransactionalProcessor.java  |  28 --
 .../coprocessor/ServerCachingEndpointImpl.java  |   4 +-
 .../TephraTransactionalProcessor.java   |  29 ++
 .../UngroupedAggregateRegionObserver.java   |  10 +-
 .../coprocessor/generated/PTableProtos.java | 110 +-
 .../phoenix/exception/SQLExceptionCode.java |   4 +
 .../apache/phoenix/execute/BaseQueryPlan.java   |   3 +
 .../apache/phoenix/execute/MutationState.java   |  74 ++--
 .../PhoenixTxIndexMutationGenerator.java|  10 +-
 .../phoenix/expression/ExpressionType.java  | 119 +-
 .../TransactionProviderNameFunction.java|  81 +
 .../apache/phoenix/index/IndexMaintainer.java   |   4 +-
 .../index/IndexMetaDataCacheFactory.java|   2 +-
 .../apache/phoenix/index/PhoenixIndexCodec.java |   1 -
 .../index/PhoenixIndexMetaDataBuilder.java  |   7 +-
 .../NonAggregateRegionScannerFactory.java   |   5 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  12 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   2 +-
 .../index/PhoenixIndexPartialBuildMapper.java   |   4 +-
 .../phoenix/query/ConnectionQueryServices.java  |   6 +-
 .../query/ConnectionQueryServicesImpl.java  |  69 ++--
 .../query/ConnectionlessQueryServicesImpl.java  |   9 +-
 .../query/DelegateConnectionQueryServices.java  |   9 +-
 .../apache/phoenix/query/QueryConstants.java|   2 +
 .../org/apache/phoenix/query/QueryServices.java |   1 +
 .../phoenix/query/QueryServicesOptions.java |  10 +-
 .../apache/phoenix/schema/DelegateTable.java|   8 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 129 +--
 .../java/org/apache/phoenix/schema/PTable.java  |   2 +
 .../org/apache/phoenix/schema/PTableImpl.java   |  68 ++--
 .../apache/phoenix/schema/TableProperty.java|  18 +
 .../transaction/OmidTransactionContext.java |  57 +--
 .../transaction/OmidTransactionProvider.java|  54 +--
 .../transaction/OmidTransactionTable.java   | 363 ---
 .../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  | 161 +++-
 .../transaction/TephraTransactionTable.java | 350 --
 .../phoenix/transaction/TransactionFactory.java |  57 ++-
 .../transaction/TransactionProvider.java|  36 --
 .../org/apache/phoenix/util/PhoenixRuntime.java |   3 +-
 .../java/org/apache/phoenix/util/ScanUtil.java  |  15 +
 .../apache/phoenix/util/TransactionUtil.java|  93 -
 .../phoenix/execute/CorrelatePlanTest.java  |   5 +-
 .../execute/LiteralResultIteratorPlanTest.java  |   5 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  26 +-
 .../phoenix/query/QueryServicesTestImpl.java|  15 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |   2 +-
 phoenix-protocol/src/main/PTable.proto  |   1 +
 70 files changed, 1225 insertions(+), 1621 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/02715adc/phoenix-core/src/it/java/org/apache/phoe

[3/6] phoenix git commit: PHOENIX-4605 Support running multiple transaction providers

2018-04-13 Thread jamestaylor
http://git-wip-us.apache.org/repos/asf/phoenix/blob/02715adc/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index c5065e0..59c10ad 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -19,120 +19,7 @@ package org.apache.phoenix.expression;
 
 import java.util.Map;
 
-import org.apache.phoenix.expression.function.AbsFunction;
-import org.apache.phoenix.expression.function.ArrayAllComparisonExpression;
-import org.apache.phoenix.expression.function.ArrayAnyComparisonExpression;
-import org.apache.phoenix.expression.function.ArrayAppendFunction;
-import org.apache.phoenix.expression.function.ArrayConcatFunction;
-import org.apache.phoenix.expression.function.ArrayElemRefExpression;
-import org.apache.phoenix.expression.function.ArrayFillFunction;
-import org.apache.phoenix.expression.function.ArrayIndexFunction;
-import org.apache.phoenix.expression.function.ArrayLengthFunction;
-import org.apache.phoenix.expression.function.ArrayPrependFunction;
-import org.apache.phoenix.expression.function.ArrayRemoveFunction;
-import org.apache.phoenix.expression.function.ArrayToStringFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpReplaceFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpSplitFunction;
-import org.apache.phoenix.expression.function.ByteBasedRegexpSubstrFunction;
-import org.apache.phoenix.expression.function.CbrtFunction;
-import org.apache.phoenix.expression.function.CeilDateExpression;
-import org.apache.phoenix.expression.function.CeilDecimalExpression;
-import org.apache.phoenix.expression.function.CeilFunction;
-import org.apache.phoenix.expression.function.CeilMonthExpression;
-import org.apache.phoenix.expression.function.CeilTimestampExpression;
-import org.apache.phoenix.expression.function.CeilWeekExpression;
-import org.apache.phoenix.expression.function.CeilYearExpression;
-import org.apache.phoenix.expression.function.CoalesceFunction;
-import org.apache.phoenix.expression.function.CollationKeyFunction;
-import org.apache.phoenix.expression.function.ConvertTimezoneFunction;
-import org.apache.phoenix.expression.function.CountAggregateFunction;
-import org.apache.phoenix.expression.function.DayOfMonthFunction;
-import org.apache.phoenix.expression.function.DayOfWeekFunction;
-import org.apache.phoenix.expression.function.DayOfYearFunction;
-import org.apache.phoenix.expression.function.DecodeFunction;
-import org.apache.phoenix.expression.function.DefaultValueExpression;
-import org.apache.phoenix.expression.function.DistinctCountAggregateFunction;
-import 
org.apache.phoenix.expression.function.DistinctCountHyperLogLogAggregateFunction;
-import org.apache.phoenix.expression.function.DistinctValueAggregateFunction;
-import org.apache.phoenix.expression.function.EncodeFunction;
-import org.apache.phoenix.expression.function.ExpFunction;
-import org.apache.phoenix.expression.function.ExternalSqlTypeIdFunction;
-import org.apache.phoenix.expression.function.FirstValueFunction;
-import org.apache.phoenix.expression.function.FirstValuesFunction;
-import org.apache.phoenix.expression.function.FloorDateExpression;
-import org.apache.phoenix.expression.function.FloorDecimalExpression;
-import org.apache.phoenix.expression.function.FloorFunction;
-import org.apache.phoenix.expression.function.FloorMonthExpression;
-import org.apache.phoenix.expression.function.FloorWeekExpression;
-import org.apache.phoenix.expression.function.FloorYearExpression;
-import org.apache.phoenix.expression.function.GetBitFunction;
-import org.apache.phoenix.expression.function.GetByteFunction;
-import org.apache.phoenix.expression.function.HourFunction;
-import org.apache.phoenix.expression.function.IndexStateNameFunction;
-import org.apache.phoenix.expression.function.InstrFunction;
-import org.apache.phoenix.expression.function.InvertFunction;
-import org.apache.phoenix.expression.function.LTrimFunction;
-import org.apache.phoenix.expression.function.LastValueFunction;
-import org.apache.phoenix.expression.function.LastValuesFunction;
-import org.apache.phoenix.expression.function.LengthFunction;
-import org.apache.phoenix.expression.function.LnFunction;
-import org.apache.phoenix.expression.function.LogFunction;
-import org.apache.phoenix.expression.function.LowerFunction;
-import org.apache.phoenix.expression.function.LpadFunction;
-import org.apache.phoenix.expression.function.MD5Function;
-import org.apache.phoenix.expression.function.MaxAggregateFunction;
-import org.apache.phoenix.expression.function.MinAggregateFunction;
-import org.apache.phoenix.expression.function.MinuteFunctio

[1/2] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 b8c42054e -> 0d2118fc0


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0d2118fc/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index e071ea6..c260845 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -169,6 +169,7 @@ import 
org.apache.phoenix.exception.RetriableUpgradeException;
 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.execute.MutationState;
 import org.apache.phoenix.hbase.index.IndexRegionSplitPolicy;
@@ -194,6 +195,7 @@ import 
org.apache.phoenix.schema.EmptySequenceCacheException;
 import org.apache.phoenix.schema.FunctionNotFoundException;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.MetaDataSplitPolicy;
+import org.apache.phoenix.schema.NewerSchemaAlreadyExistsException;
 import org.apache.phoenix.schema.NewerTableAlreadyExistsException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
@@ -1038,23 +1040,69 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 
 /**
  *
- * @param tableName
+ * @param physicalTableName
+ * @param tableType
+ * @param props
+ * @param families
  * @param splits
- * @param modifyExistingMetaData TODO
+ * @param modifyExistingMetaData
+ * @param isNamespaceMapped
+ * @param isDoNotUpgradePropSet
  * @return true if table was created and false if it already exists
  * @throws SQLException
  */
 private HTableDescriptor ensureTableCreated(byte[] physicalTableName, 
PTableType tableType, Map props,
 List>> families, byte[][] splits, 
boolean modifyExistingMetaData,
-boolean isNamespaceMapped) throws SQLException {
+boolean isNamespaceMapped, boolean isDoNotUpgradePropSet) throws 
SQLException {
 SQLException sqlE = null;
 HTableDescriptor existingDesc = null;
 boolean isMetaTable = SchemaUtil.isMetaTable(physicalTableName);
 boolean tableExist = true;
 try (HBaseAdmin admin = getAdmin()) {
 final String quorum = ZKConfig.getZKQuorumServersString(config);
-final String znode = 
this.props.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
+final String znode = 
this.getProps().get(HConstants.ZOOKEEPER_ZNODE_PARENT);
 logger.debug("Found quorum: " + quorum + ":" + znode);
+
+if (isMetaTable) {
+if(SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, 
this.getProps())) {
+try {
+// SYSTEM namespace needs to be created via HBase APIs 
because "CREATE SCHEMA" statement tries to write
+// its metadata in SYSTEM:CATALOG table. Without 
SYSTEM namespace, SYSTEM:CATALOG table cannot be created
+
ensureNamespaceCreated(QueryConstants.SYSTEM_SCHEMA_NAME);
+} catch (PhoenixIOException e) {
+// We could either:
+// 1) Not access the NS descriptor. The NS may or may 
not exist at this point
+// 2) We could not create the NS
+// Regardless of the case 1 or 2, if we eventually try 
to migrate SYSTEM tables to the SYSTEM
+// namespace using the {@link 
ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method,
+// if the NS does not exist, we will error as 
expected, or
+// if the NS does exist and tables are already mapped, 
the check will exit gracefully
+}
+if 
(admin.tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, 
false))) {
+// SYSTEM.CATALOG exists, so at this point, we have 3 
cases:
+// 1) If server-side namespace mapping is disabled, 
throw Inconsistent namespace mapping exception
+// 2) If server-side namespace mapping is enabled and 
SYSCAT needs to be upgraded, upgrade SYSCAT
+//and also migrate SYSTEM tables to the SYSTEM 
namespace
+// 3. If server-side namespace mapping is enabled and 
SYSCAT doesn't need

[2/2] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first 
client connection (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 0d2118fc092f7c4afd1c59a0ec959c4b4777a1e5
Parents: b8c4205
Author: James Taylor 
Authored: Fri Apr 13 10:30:30 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 10:46:21 2018 -0700

--
 .../phoenix/end2end/AppendOnlySchemaIT.java |   2 +-
 .../MigrateSystemTablesToSystemNamespaceIT.java |   9 +-
 .../SystemCatalogCreationOnConnectionIT.java| 626 
 .../coprocessor/MetaDataEndpointImpl.java   |  21 +
 .../phoenix/coprocessor/MetaDataProtocol.java   |   4 +
 .../coprocessor/generated/MetaDataProtos.java   | 183 +++--
 .../exception/UpgradeRequiredException.java |  13 +-
 .../phoenix/query/ConnectionQueryServices.java  |   2 +-
 .../query/ConnectionQueryServicesImpl.java  | 740 +++
 .../query/ConnectionlessQueryServicesImpl.java  |   2 +-
 .../query/DelegateConnectionQueryServices.java  |   4 +-
 .../apache/phoenix/schema/MetaDataClient.java   |   3 +-
 .../query/ConnectionQueryServicesImplTest.java  |   7 +-
 phoenix-protocol/src/main/MetaDataService.proto |   1 +
 14 files changed, 1237 insertions(+), 380 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0d2118fc/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
index 7ed64ff..d601beb 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
@@ -114,7 +114,7 @@ public class AppendOnlySchemaIT extends 
ParallelStatsDisabledIT {
 // verify no create table rpcs
 verify(connectionQueryServices, 
never()).createTable(anyListOf(Mutation.class),
 any(byte[].class), any(PTableType.class), anyMap(), anyList(), 
any(byte[][].class),
-eq(false), eq(false));
+eq(false), eq(false), eq(false));
 reset(connectionQueryServices);
 
 // execute alter table ddl that adds the same column

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0d2118fc/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
index 7cb5857..627e453 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
@@ -376,8 +376,10 @@ public class MigrateSystemTablesToSystemNamespaceIT 
extends BaseTest {
 while(rs.next()) {
 
 if(rs.getString("IS_NAMESPACE_MAPPED") == null) {
+// Check that entry for SYSTEM namespace exists in SYSCAT
 systemSchemaExists = 
rs.getString("TABLE_SCHEM").equals(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME) 
? true : systemSchemaExists;
 } else if (rs.getString("COLUMN_NAME") == null) {
+// Found the intial entry for a table in SYSCAT
 String schemaName = rs.getString("TABLE_SCHEM");
 String tableName = rs.getString("TABLE_NAME");
 
@@ -395,12 +397,11 @@ public class MigrateSystemTablesToSystemNamespaceIT 
extends BaseTest {
 }
 }
 
-if(!systemSchemaExists) {
-fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry doesn't 
exist in SYSTEM.CATALOG table.");
-}
-
 // The set will contain SYSMUTEX table since that table is not exposed 
in SYSCAT
 if (systemTablesMapped) {
+if (!systemSchemaExists) {
+fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry 
doesn't exist in SYSTEM.CATALOG table.");
+}
 assertTrue(namespaceMappedSystemTablesSet.size() == 1);
 } else {
 assertTrue(systemTablesSet.size() == 1);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0d2118fc/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
-

[1/2] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 6a88cfed3 -> c0409f815


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c0409f81/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 99612af..6c316de 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -169,6 +169,7 @@ import 
org.apache.phoenix.exception.RetriableUpgradeException;
 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.execute.MutationState;
 import org.apache.phoenix.hbase.index.IndexRegionSplitPolicy;
@@ -193,6 +194,7 @@ import 
org.apache.phoenix.schema.EmptySequenceCacheException;
 import org.apache.phoenix.schema.FunctionNotFoundException;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.MetaDataSplitPolicy;
+import org.apache.phoenix.schema.NewerSchemaAlreadyExistsException;
 import org.apache.phoenix.schema.NewerTableAlreadyExistsException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
@@ -1022,23 +1024,69 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 
 /**
  *
- * @param tableName
+ * @param physicalTableName
+ * @param tableType
+ * @param props
+ * @param families
  * @param splits
- * @param modifyExistingMetaData TODO
+ * @param modifyExistingMetaData
+ * @param isNamespaceMapped
+ * @param isDoNotUpgradePropSet
  * @return true if table was created and false if it already exists
  * @throws SQLException
  */
 private HTableDescriptor ensureTableCreated(byte[] physicalTableName, 
PTableType tableType, Map props,
 List>> families, byte[][] splits, 
boolean modifyExistingMetaData,
-boolean isNamespaceMapped) throws SQLException {
+boolean isNamespaceMapped, boolean isDoNotUpgradePropSet) throws 
SQLException {
 SQLException sqlE = null;
 HTableDescriptor existingDesc = null;
 boolean isMetaTable = SchemaUtil.isMetaTable(physicalTableName);
 boolean tableExist = true;
 try (HBaseAdmin admin = getAdmin()) {
 final String quorum = ZKConfig.getZKQuorumServersString(config);
-final String znode = 
this.props.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
+final String znode = 
this.getProps().get(HConstants.ZOOKEEPER_ZNODE_PARENT);
 logger.debug("Found quorum: " + quorum + ":" + znode);
+
+if (isMetaTable) {
+if(SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, 
this.getProps())) {
+try {
+// SYSTEM namespace needs to be created via HBase APIs 
because "CREATE SCHEMA" statement tries to write
+// its metadata in SYSTEM:CATALOG table. Without 
SYSTEM namespace, SYSTEM:CATALOG table cannot be created
+
ensureNamespaceCreated(QueryConstants.SYSTEM_SCHEMA_NAME);
+} catch (PhoenixIOException e) {
+// We could either:
+// 1) Not access the NS descriptor. The NS may or may 
not exist at this point
+// 2) We could not create the NS
+// Regardless of the case 1 or 2, if we eventually try 
to migrate SYSTEM tables to the SYSTEM
+// namespace using the {@link 
ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method,
+// if the NS does not exist, we will error as 
expected, or
+// if the NS does exist and tables are already mapped, 
the check will exit gracefully
+}
+if 
(admin.tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, 
false))) {
+// SYSTEM.CATALOG exists, so at this point, we have 3 
cases:
+// 1) If server-side namespace mapping is disabled, 
throw Inconsistent namespace mapping exception
+// 2) If server-side namespace mapping is enabled and 
SYSCAT needs to be upgraded, upgrade SYSCAT
+//and also migrate SYSTEM tables to the SYSTEM 
namespace
+// 3. If server-side namespace mapping is enabled and 
SYSCAT doesn't nee

[2/2] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first 
client connection (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-HBase-0.98
Commit: c0409f815816ba6b81e4fa08647586e8ad0e3a54
Parents: 6a88cfe
Author: James Taylor 
Authored: Fri Apr 13 10:30:30 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 10:51:44 2018 -0700

--
 .../phoenix/end2end/AppendOnlySchemaIT.java |   2 +-
 .../MigrateSystemTablesToSystemNamespaceIT.java |   9 +-
 .../SystemCatalogCreationOnConnectionIT.java| 626 
 .../coprocessor/MetaDataEndpointImpl.java   |  21 +
 .../phoenix/coprocessor/MetaDataProtocol.java   |   4 +
 .../coprocessor/generated/MetaDataProtos.java   | 183 +++--
 .../exception/UpgradeRequiredException.java |  13 +-
 .../phoenix/query/ConnectionQueryServices.java  |   2 +-
 .../query/ConnectionQueryServicesImpl.java  | 740 +++
 .../query/ConnectionlessQueryServicesImpl.java  |   2 +-
 .../query/DelegateConnectionQueryServices.java  |   4 +-
 .../apache/phoenix/schema/MetaDataClient.java   |   3 +-
 .../query/ConnectionQueryServicesImplTest.java  |   7 +-
 phoenix-protocol/src/main/MetaDataService.proto |   1 +
 14 files changed, 1237 insertions(+), 380 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c0409f81/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
index e9a20b3..e7becc2 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
@@ -128,7 +128,7 @@ public class AppendOnlySchemaIT extends 
ParallelStatsDisabledIT {
 // verify no create table rpcs
 verify(connectionQueryServices, 
never()).createTable(anyListOf(Mutation.class),
 any(byte[].class), any(PTableType.class), anyMap(), anyList(), 
any(byte[][].class),
-eq(false), eq(false));
+eq(false), eq(false), eq(false));
 reset(connectionQueryServices);
 
 // execute alter table ddl that adds the same column

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c0409f81/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
index c36cdd3..e7187d6 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
@@ -372,8 +372,10 @@ public class MigrateSystemTablesToSystemNamespaceIT 
extends BaseTest {
 while(rs.next()) {
 
 if(rs.getString("IS_NAMESPACE_MAPPED") == null) {
+// Check that entry for SYSTEM namespace exists in SYSCAT
 systemSchemaExists = 
rs.getString("TABLE_SCHEM").equals(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME) 
? true : systemSchemaExists;
 } else if (rs.getString("COLUMN_NAME") == null) {
+// Found the intial entry for a table in SYSCAT
 String schemaName = rs.getString("TABLE_SCHEM");
 String tableName = rs.getString("TABLE_NAME");
 
@@ -391,12 +393,11 @@ public class MigrateSystemTablesToSystemNamespaceIT 
extends BaseTest {
 }
 }
 
-if(!systemSchemaExists) {
-fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry doesn't 
exist in SYSTEM.CATALOG table.");
-}
-
 // The set will contain SYSMUTEX table since that table is not exposed 
in SYSCAT
 if (systemTablesMapped) {
+if (!systemSchemaExists) {
+fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry 
doesn't exist in SYSTEM.CATALOG table.");
+}
 assertTrue(namespaceMappedSystemTablesSet.size() == 1);
 } else {
 assertTrue(systemTablesSet.size() == 1);

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

[2/2] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first 
client connection (Chinmay Kulkarni)


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

Branch: refs/heads/5.x-HBase-2.0
Commit: d30d771860923abfc1e6efaaef55e24a1f2700c7
Parents: 15fa00f
Author: James Taylor 
Authored: Fri Apr 13 10:30:30 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 11:27:54 2018 -0700

--
 .../phoenix/end2end/AppendOnlySchemaIT.java |   2 +-
 .../MigrateSystemTablesToSystemNamespaceIT.java |   9 +-
 .../SystemCatalogCreationOnConnectionIT.java| 626 +++
 .../coprocessor/MetaDataEndpointImpl.java   |  21 +
 .../phoenix/coprocessor/MetaDataProtocol.java   |   4 +
 .../coprocessor/generated/MetaDataProtos.java   | 183 +++--
 .../exception/UpgradeRequiredException.java |  13 +-
 .../phoenix/query/ConnectionQueryServices.java  |   2 +-
 .../query/ConnectionQueryServicesImpl.java  | 756 +++
 .../query/ConnectionlessQueryServicesImpl.java  |   2 +-
 .../query/DelegateConnectionQueryServices.java  |   4 +-
 .../apache/phoenix/schema/MetaDataClient.java   |   3 +-
 .../query/ConnectionQueryServicesImplTest.java  |   7 +-
 phoenix-protocol/src/main/MetaDataService.proto |   1 +
 14 files changed, 1238 insertions(+), 395 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d30d7718/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
index 7ed64ff..d601beb 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
@@ -114,7 +114,7 @@ public class AppendOnlySchemaIT extends 
ParallelStatsDisabledIT {
 // verify no create table rpcs
 verify(connectionQueryServices, 
never()).createTable(anyListOf(Mutation.class),
 any(byte[].class), any(PTableType.class), anyMap(), anyList(), 
any(byte[][].class),
-eq(false), eq(false));
+eq(false), eq(false), eq(false));
 reset(connectionQueryServices);
 
 // execute alter table ddl that adds the same column

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d30d7718/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
index 7cb5857..627e453 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
@@ -376,8 +376,10 @@ public class MigrateSystemTablesToSystemNamespaceIT 
extends BaseTest {
 while(rs.next()) {
 
 if(rs.getString("IS_NAMESPACE_MAPPED") == null) {
+// Check that entry for SYSTEM namespace exists in SYSCAT
 systemSchemaExists = 
rs.getString("TABLE_SCHEM").equals(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME) 
? true : systemSchemaExists;
 } else if (rs.getString("COLUMN_NAME") == null) {
+// Found the intial entry for a table in SYSCAT
 String schemaName = rs.getString("TABLE_SCHEM");
 String tableName = rs.getString("TABLE_NAME");
 
@@ -395,12 +397,11 @@ public class MigrateSystemTablesToSystemNamespaceIT 
extends BaseTest {
 }
 }
 
-if(!systemSchemaExists) {
-fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry doesn't 
exist in SYSTEM.CATALOG table.");
-}
-
 // The set will contain SYSMUTEX table since that table is not exposed 
in SYSCAT
 if (systemTablesMapped) {
+if (!systemSchemaExists) {
+fail(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + " entry 
doesn't exist in SYSTEM.CATALOG table.");
+}
 assertTrue(namespaceMappedSystemTablesSet.size() == 1);
 } else {
 assertTrue(systemTablesSet.size() == 1);

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

[1/2] phoenix git commit: PHOENIX-4579 Add a config to conditionally create Phoenix meta tables on first client connection (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/5.x-HBase-2.0 15fa00fa5 -> d30d77186


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d30d7718/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index dc9ef04..446bbcf 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -177,6 +177,7 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.exception.UpgradeInProgressException;
 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;
@@ -1051,23 +1052,69 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 
 /**
  *
- * @param tableName
+ * @param physicalTableName
+ * @param tableType
+ * @param props
+ * @param families
  * @param splits
- * @param modifyExistingMetaData TODO
+ * @param modifyExistingMetaData
+ * @param isNamespaceMapped
+ * @param isDoNotUpgradePropSet
  * @return true if table was created and false if it already exists
  * @throws SQLException
  */
 private TableDescriptor ensureTableCreated(byte[] physicalTableName, 
PTableType tableType, Map props,
 List>> families, byte[][] splits, 
boolean modifyExistingMetaData,
-boolean isNamespaceMapped) throws SQLException {
+boolean isNamespaceMapped, boolean isDoNotUpgradePropSet) throws 
SQLException {
 SQLException sqlE = null;
 TableDescriptor existingDesc = null;
 boolean isMetaTable = SchemaUtil.isMetaTable(physicalTableName);
 boolean tableExist = true;
 try (Admin admin = getAdmin()) {
 final String quorum = ZKConfig.getZKQuorumServersString(config);
-final String znode = 
this.props.get(HConstants.ZOOKEEPER_ZNODE_PARENT);
+final String znode = 
this.getProps().get(HConstants.ZOOKEEPER_ZNODE_PARENT);
 logger.debug("Found quorum: " + quorum + ":" + znode);
+
+if (isMetaTable) {
+if(SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, 
this.getProps())) {
+try {
+// SYSTEM namespace needs to be created via HBase APIs 
because "CREATE SCHEMA" statement tries to write
+// its metadata in SYSTEM:CATALOG table. Without 
SYSTEM namespace, SYSTEM:CATALOG table cannot be created
+
ensureNamespaceCreated(QueryConstants.SYSTEM_SCHEMA_NAME);
+} catch (PhoenixIOException e) {
+// We could either:
+// 1) Not access the NS descriptor. The NS may or may 
not exist at this point
+// 2) We could not create the NS
+// Regardless of the case 1 or 2, if we eventually try 
to migrate SYSTEM tables to the SYSTEM
+// namespace using the {@link 
ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method,
+// if the NS does not exist, we will error as 
expected, or
+// if the NS does exist and tables are already mapped, 
the check will exit gracefully
+}
+if 
(admin.tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, 
false))) {
+// SYSTEM.CATALOG exists, so at this point, we have 3 
cases:
+// 1) If server-side namespace mapping is disabled, 
throw Inconsistent namespace mapping exception
+// 2) If server-side namespace mapping is enabled and 
SYSCAT needs to be upgraded, upgrade SYSCAT
+//and also migrate SYSTEM tables to the SYSTEM 
namespace
+// 3. If server-side namespace mapping is enabled and 
SYSCAT doesn't need to be upgraded, we still
+//need to migrate SYSTEM tables to the SYSTEM 
namespace using the
+//{@link 
ensureSystemTablesMigratedToSystemNamespace(ReadOnlyProps)} method (as part of
+//{@link upgradeSystemTables(String, Properties)})
+
checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
+// Thrown so we can force an upgrade which will just 
migrate SYSTEM tab

phoenix git commit: PHOENIX-4579 Remove SYSTEM.LOG from list of Phoenix system tables since that JIRA hasn't been backported

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 c0409f815 -> 3e8049c91


PHOENIX-4579 Remove SYSTEM.LOG from list of Phoenix system tables since that 
JIRA hasn't been backported


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 3e8049c91d1730f5dd81c8bdd8430d20d1885ec4
Parents: c0409f8
Author: James Taylor 
Authored: Fri Apr 13 13:57:03 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 13:57:03 2018 -0700

--
 .../SystemCatalogCreationOnConnectionIT.java| 34 +---
 1 file changed, 23 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3e8049c9/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
index 689eb20..a2bc272 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
@@ -17,6 +17,22 @@
  */
 package org.apache.phoenix.end2end;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -28,7 +44,11 @@ import org.apache.phoenix.exception.UpgradeRequiredException;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
 import org.apache.phoenix.jdbc.PhoenixTestDriver;
-import org.apache.phoenix.query.*;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.ConnectionQueryServicesImpl;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesTestImpl;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.UpgradeUtil;
 import org.junit.After;
@@ -36,14 +56,6 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.util.*;
-import java.util.concurrent.TimeoutException;
-
-import static org.junit.Assert.*;
-
 @Category(NeedsOwnMiniClusterTest.class)
 public class SystemCatalogCreationOnConnectionIT {
 private HBaseTestingUtility testUtil = null;
@@ -60,11 +72,11 @@ public class SystemCatalogCreationOnConnectionIT {
 
 private static final Set PHOENIX_SYSTEM_TABLES = new 
HashSet<>(Arrays.asList(
   "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION",
-  "SYSTEM.MUTEX", "SYSTEM.LOG"));
+  "SYSTEM.MUTEX"));
 
 private static final Set PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = 
new HashSet<>(
   Arrays.asList("SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", 
"SYSTEM:FUNCTION",
-"SYSTEM:MUTEX", "SYSTEM:LOG"));
+"SYSTEM:MUTEX"));
 
 private static class PhoenixSysCatCreationServices extends 
ConnectionQueryServicesImpl {
 



phoenix git commit: PHOENIX-4579 Remove SYSTEM.LOG from list of Phoenix system tables since that JIRA hasn't been backported

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 0d2118fc0 -> b28d11998


PHOENIX-4579 Remove SYSTEM.LOG from list of Phoenix system tables since that 
JIRA hasn't been backported


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

Branch: refs/heads/4.x-HBase-1.1
Commit: b28d1199879dee00686467e24e8683488818ff8e
Parents: 0d2118f
Author: James Taylor 
Authored: Fri Apr 13 13:57:03 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 13:57:52 2018 -0700

--
 .../SystemCatalogCreationOnConnectionIT.java| 34 +---
 1 file changed, 23 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b28d1199/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
index 689eb20..a2bc272 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
@@ -17,6 +17,22 @@
  */
 package org.apache.phoenix.end2end;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -28,7 +44,11 @@ import org.apache.phoenix.exception.UpgradeRequiredException;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
 import org.apache.phoenix.jdbc.PhoenixTestDriver;
-import org.apache.phoenix.query.*;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.ConnectionQueryServicesImpl;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesTestImpl;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.UpgradeUtil;
 import org.junit.After;
@@ -36,14 +56,6 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.util.*;
-import java.util.concurrent.TimeoutException;
-
-import static org.junit.Assert.*;
-
 @Category(NeedsOwnMiniClusterTest.class)
 public class SystemCatalogCreationOnConnectionIT {
 private HBaseTestingUtility testUtil = null;
@@ -60,11 +72,11 @@ public class SystemCatalogCreationOnConnectionIT {
 
 private static final Set PHOENIX_SYSTEM_TABLES = new 
HashSet<>(Arrays.asList(
   "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION",
-  "SYSTEM.MUTEX", "SYSTEM.LOG"));
+  "SYSTEM.MUTEX"));
 
 private static final Set PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = 
new HashSet<>(
   Arrays.asList("SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", 
"SYSTEM:FUNCTION",
-"SYSTEM:MUTEX", "SYSTEM:LOG"));
+"SYSTEM:MUTEX"));
 
 private static class PhoenixSysCatCreationServices extends 
ConnectionQueryServicesImpl {
 



[2/2] phoenix git commit: PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be property driven

2018-04-13 Thread jamestaylor
PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be 
property driven


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

Branch: refs/heads/master
Commit: 430e8020ce5b900d9d9a3533fc5d647926704b7f
Parents: a5770a5
Author: James Taylor 
Authored: Fri Apr 13 14:30:00 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 14:30:00 2018 -0700

--
 .../phoenix/coprocessor/MetaDataProtocol.java   |  4 --
 .../query/ConnectionQueryServicesImpl.java  | 41 ++--
 .../query/ConnectionlessQueryServicesImpl.java  | 35 +++--
 .../apache/phoenix/query/QueryConstants.java| 14 +++
 .../org/apache/phoenix/query/QueryServices.java |  5 ++-
 .../phoenix/query/QueryServicesOptions.java |  3 +-
 .../org/apache/phoenix/util/UpgradeUtil.java|  4 +-
 7 files changed, 74 insertions(+), 32 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/430e8020/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 26f8198..36d6f0d 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -72,10 +72,6 @@ public abstract class MetaDataProtocol extends 
MetaDataService {
 public static final long MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP = 0;
 public static final String MIGRATION_IN_PROGRESS = "MigrationInProgress";
 
-public static final int DEFAULT_MAX_META_DATA_VERSIONS = 1000;
-public static final boolean DEFAULT_META_DATA_KEEP_DELETED_CELLS = true;
-public static final int DEFAULT_MAX_STAT_DATA_VERSIONS = 1;
-public static final boolean DEFAULT_STATS_KEEP_DELETED_CELLS = false;
 public static final int DEFAULT_LOG_VERSIONS = 10;
 public static final int DEFAULT_LOG_TTL = 7 * 24 * 60 * 60; // 7 days 
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/430e8020/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 4c1902d..5985705 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -720,10 +720,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 private HColumnDescriptor 
generateColumnFamilyDescriptor(Pair> family, 
PTableType tableType) throws SQLException {
 HColumnDescriptor columnDesc = new 
HColumnDescriptor(family.getFirst());
 if (tableType != PTableType.VIEW) {
-if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != 
null){
-columnDesc.setKeepDeletedCells(props.getBoolean(
-QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, 
QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
-}
 
columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
 columnDesc.setBloomFilterType(BloomType.NONE);
 for (Entry entry : family.getSecond().entrySet()) {
@@ -2459,8 +2455,29 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 }
 
 // Available for testing
-protected String getSystemCatalogDML() {
-return QueryConstants.CREATE_TABLE_METADATA;
+protected String getSystemCatalogTableDDL() {
+return setSystemDDLProperties(QueryConstants.CREATE_TABLE_METADATA);
+}
+
+// Available for testing
+protected String getFunctionTableDDL() {
+return setSystemDDLProperties(QueryConstants.CREATE_FUNCTION_METADATA);
+}
+
+// Available for testing
+protected String getLogTableDDL() {
+return setSystemLogDDLProperties(QueryConstants.CREATE_LOG_METADATA);
+}
+
+private String setSystemDDLProperties(String ddl) {
+return String.format(ddl,
+  props.getInt(DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB, 
QueryServicesOptions.DEFAULT_SYSTEM_MAX_VERSIONS),
+  props.getBoolean(DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB, 
QueryServicesOptions.DEFAULT_SYSTEM_KEEP_DELETED_CELLS));
+}
+
+

[1/2] phoenix git commit: PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY column (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/master 87fdda8b1 -> 430e8020c


PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY 
column (Chinmay Kulkarni)


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

Branch: refs/heads/master
Commit: a5770a5a4d41bf786d3d8dd7ca9088a00418b666
Parents: 87fdda8
Author: James Taylor 
Authored: Fri Apr 13 14:19:15 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 14:19:15 2018 -0700

--
 .../phoenix/query/ConnectionQueryServicesImpl.java   | 11 ---
 1 file changed, 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a5770a5a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 8b328d3..4c1902d 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -1130,10 +1130,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 
PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES
 {
 newDesc.setValue(HTableDescriptor.SPLIT_POLICY, 
IndexRegionSplitPolicy.class.getName());
 }
-// Remove the splitPolicy attribute to prevent HBASE-12570
-if (isMetaTable) {
-newDesc.remove(HTableDescriptor.SPLIT_POLICY);
-}
 try {
 if (splits == null) {
 admin.createTable(newDesc);
@@ -1150,13 +1146,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 }
 if (isMetaTable && !isUpgradeRequired()) {
 
checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES,
 this.getProps()).getName());
-/*
- * Now we modify the table to add the split policy, since 
we know that the client and
- * server and compatible. This works around HBASE-12570 
which causes the cluster to be
- * brought down.
- */
-newDesc.setValue(HTableDescriptor.SPLIT_POLICY, 
MetaDataSplitPolicy.class.getName());
-modifyTable(physicalTableName, newDesc, true);
 }
 return null;
 } else {



[1/2] phoenix git commit: PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY column (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.3 5f3ef7c5d -> d525ba6a0


PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY 
column (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-HBase-1.3
Commit: 8e61260bb2df936670be836ad02018c35b6842b7
Parents: 5f3ef7c
Author: James Taylor 
Authored: Fri Apr 13 14:19:15 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 14:33:25 2018 -0700

--
 .../phoenix/query/ConnectionQueryServicesImpl.java   | 11 ---
 1 file changed, 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8e61260b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 8b328d3..4c1902d 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -1130,10 +1130,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 
PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES
 {
 newDesc.setValue(HTableDescriptor.SPLIT_POLICY, 
IndexRegionSplitPolicy.class.getName());
 }
-// Remove the splitPolicy attribute to prevent HBASE-12570
-if (isMetaTable) {
-newDesc.remove(HTableDescriptor.SPLIT_POLICY);
-}
 try {
 if (splits == null) {
 admin.createTable(newDesc);
@@ -1150,13 +1146,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 }
 if (isMetaTable && !isUpgradeRequired()) {
 
checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES,
 this.getProps()).getName());
-/*
- * Now we modify the table to add the split policy, since 
we know that the client and
- * server and compatible. This works around HBASE-12570 
which causes the cluster to be
- * brought down.
- */
-newDesc.setValue(HTableDescriptor.SPLIT_POLICY, 
MetaDataSplitPolicy.class.getName());
-modifyTable(physicalTableName, newDesc, true);
 }
 return null;
 } else {



[2/2] phoenix git commit: PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be property driven (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be 
property driven (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-HBase-1.3
Commit: d525ba6a0fda599018bc4259ba878984027ff99e
Parents: 8e61260
Author: James Taylor 
Authored: Fri Apr 13 14:30:00 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 14:35:23 2018 -0700

--
 .../phoenix/coprocessor/MetaDataProtocol.java   |  4 --
 .../query/ConnectionQueryServicesImpl.java  | 41 ++--
 .../query/ConnectionlessQueryServicesImpl.java  | 35 +++--
 .../apache/phoenix/query/QueryConstants.java| 14 +++
 .../org/apache/phoenix/query/QueryServices.java |  5 ++-
 .../phoenix/query/QueryServicesOptions.java |  3 +-
 .../org/apache/phoenix/util/UpgradeUtil.java|  4 +-
 7 files changed, 74 insertions(+), 32 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d525ba6a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 26f8198..36d6f0d 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -72,10 +72,6 @@ public abstract class MetaDataProtocol extends 
MetaDataService {
 public static final long MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP = 0;
 public static final String MIGRATION_IN_PROGRESS = "MigrationInProgress";
 
-public static final int DEFAULT_MAX_META_DATA_VERSIONS = 1000;
-public static final boolean DEFAULT_META_DATA_KEEP_DELETED_CELLS = true;
-public static final int DEFAULT_MAX_STAT_DATA_VERSIONS = 1;
-public static final boolean DEFAULT_STATS_KEEP_DELETED_CELLS = false;
 public static final int DEFAULT_LOG_VERSIONS = 10;
 public static final int DEFAULT_LOG_TTL = 7 * 24 * 60 * 60; // 7 days 
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d525ba6a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 4c1902d..5985705 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -720,10 +720,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 private HColumnDescriptor 
generateColumnFamilyDescriptor(Pair> family, 
PTableType tableType) throws SQLException {
 HColumnDescriptor columnDesc = new 
HColumnDescriptor(family.getFirst());
 if (tableType != PTableType.VIEW) {
-if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != 
null){
-columnDesc.setKeepDeletedCells(props.getBoolean(
-QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, 
QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
-}
 
columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
 columnDesc.setBloomFilterType(BloomType.NONE);
 for (Entry entry : family.getSecond().entrySet()) {
@@ -2459,8 +2455,29 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 }
 
 // Available for testing
-protected String getSystemCatalogDML() {
-return QueryConstants.CREATE_TABLE_METADATA;
+protected String getSystemCatalogTableDDL() {
+return setSystemDDLProperties(QueryConstants.CREATE_TABLE_METADATA);
+}
+
+// Available for testing
+protected String getFunctionTableDDL() {
+return setSystemDDLProperties(QueryConstants.CREATE_FUNCTION_METADATA);
+}
+
+// Available for testing
+protected String getLogTableDDL() {
+return setSystemLogDDLProperties(QueryConstants.CREATE_LOG_METADATA);
+}
+
+private String setSystemDDLProperties(String ddl) {
+return String.format(ddl,
+  props.getInt(DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB, 
QueryServicesOptions.DEFAULT_SYSTEM_MAX_VERSIONS),
+  props.getBoolean(DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB, 
QueryServicesOptions.DEFAULT_SYSTEM_KEEP_D

[2/2] phoenix git commit: PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be property driven (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be 
property driven (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-HBase-1.2
Commit: bc14ad5dc9f4f4dc2598e2e97c2a6afebdb3a096
Parents: c0e6e2a
Author: James Taylor 
Authored: Fri Apr 13 14:30:00 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 14:36:39 2018 -0700

--
 .../phoenix/coprocessor/MetaDataProtocol.java   |  4 --
 .../query/ConnectionQueryServicesImpl.java  | 41 ++--
 .../query/ConnectionlessQueryServicesImpl.java  | 35 +++--
 .../apache/phoenix/query/QueryConstants.java| 14 +++
 .../org/apache/phoenix/query/QueryServices.java |  5 ++-
 .../phoenix/query/QueryServicesOptions.java |  3 +-
 .../org/apache/phoenix/util/UpgradeUtil.java|  4 +-
 7 files changed, 74 insertions(+), 32 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bc14ad5d/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 26f8198..36d6f0d 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -72,10 +72,6 @@ public abstract class MetaDataProtocol extends 
MetaDataService {
 public static final long MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP = 0;
 public static final String MIGRATION_IN_PROGRESS = "MigrationInProgress";
 
-public static final int DEFAULT_MAX_META_DATA_VERSIONS = 1000;
-public static final boolean DEFAULT_META_DATA_KEEP_DELETED_CELLS = true;
-public static final int DEFAULT_MAX_STAT_DATA_VERSIONS = 1;
-public static final boolean DEFAULT_STATS_KEEP_DELETED_CELLS = false;
 public static final int DEFAULT_LOG_VERSIONS = 10;
 public static final int DEFAULT_LOG_TTL = 7 * 24 * 60 * 60; // 7 days 
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bc14ad5d/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 4c1902d..5985705 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -720,10 +720,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 private HColumnDescriptor 
generateColumnFamilyDescriptor(Pair> family, 
PTableType tableType) throws SQLException {
 HColumnDescriptor columnDesc = new 
HColumnDescriptor(family.getFirst());
 if (tableType != PTableType.VIEW) {
-if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != 
null){
-columnDesc.setKeepDeletedCells(props.getBoolean(
-QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, 
QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
-}
 
columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
 columnDesc.setBloomFilterType(BloomType.NONE);
 for (Entry entry : family.getSecond().entrySet()) {
@@ -2459,8 +2455,29 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 }
 
 // Available for testing
-protected String getSystemCatalogDML() {
-return QueryConstants.CREATE_TABLE_METADATA;
+protected String getSystemCatalogTableDDL() {
+return setSystemDDLProperties(QueryConstants.CREATE_TABLE_METADATA);
+}
+
+// Available for testing
+protected String getFunctionTableDDL() {
+return setSystemDDLProperties(QueryConstants.CREATE_FUNCTION_METADATA);
+}
+
+// Available for testing
+protected String getLogTableDDL() {
+return setSystemLogDDLProperties(QueryConstants.CREATE_LOG_METADATA);
+}
+
+private String setSystemDDLProperties(String ddl) {
+return String.format(ddl,
+  props.getInt(DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB, 
QueryServicesOptions.DEFAULT_SYSTEM_MAX_VERSIONS),
+  props.getBoolean(DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB, 
QueryServicesOptions.DEFAULT_SYSTEM_KEEP_D

[1/2] phoenix git commit: PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY column (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.2 6fb7d45a7 -> bc14ad5dc


PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY 
column (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-HBase-1.2
Commit: c0e6e2a70adff20b0fc84481845e05ef9f1d5415
Parents: 6fb7d45
Author: James Taylor 
Authored: Fri Apr 13 14:19:15 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 14:36:23 2018 -0700

--
 .../phoenix/query/ConnectionQueryServicesImpl.java   | 11 ---
 1 file changed, 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c0e6e2a7/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 8b328d3..4c1902d 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -1130,10 +1130,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 
PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES
 {
 newDesc.setValue(HTableDescriptor.SPLIT_POLICY, 
IndexRegionSplitPolicy.class.getName());
 }
-// Remove the splitPolicy attribute to prevent HBASE-12570
-if (isMetaTable) {
-newDesc.remove(HTableDescriptor.SPLIT_POLICY);
-}
 try {
 if (splits == null) {
 admin.createTable(newDesc);
@@ -1150,13 +1146,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 }
 if (isMetaTable && !isUpgradeRequired()) {
 
checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES,
 this.getProps()).getName());
-/*
- * Now we modify the table to add the split policy, since 
we know that the client and
- * server and compatible. This works around HBASE-12570 
which causes the cluster to be
- * brought down.
- */
-newDesc.setValue(HTableDescriptor.SPLIT_POLICY, 
MetaDataSplitPolicy.class.getName());
-modifyTable(physicalTableName, newDesc, true);
 }
 return null;
 } else {



[1/2] phoenix git commit: PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY column (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-cdh5.11 2a3c9c94e -> dd8f1dab0


PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY 
column (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-cdh5.11
Commit: 83b0901479909e51f0f01977dba7322473dc180e
Parents: 2a3c9c9
Author: James Taylor 
Authored: Fri Apr 13 14:19:15 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 14:39:04 2018 -0700

--
 .../phoenix/query/ConnectionQueryServicesImpl.java   | 11 ---
 1 file changed, 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/83b09014/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index a94c083..7ea6b42 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -1112,10 +1112,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 
PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES
 {
 newDesc.setValue(HTableDescriptor.SPLIT_POLICY, 
IndexRegionSplitPolicy.class.getName());
 }
-// Remove the splitPolicy attribute to prevent HBASE-12570
-if (isMetaTable) {
-newDesc.remove(HTableDescriptor.SPLIT_POLICY);
-}
 try {
 if (splits == null) {
 admin.createTable(newDesc);
@@ -1132,13 +1128,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 }
 if (isMetaTable && !isUpgradeRequired()) {
 
checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES,
 this.getProps()).getName());
-/*
- * Now we modify the table to add the split policy, since 
we know that the client and
- * server and compatible. This works around HBASE-12570 
which causes the cluster to be
- * brought down.
- */
-newDesc.setValue(HTableDescriptor.SPLIT_POLICY, 
MetaDataSplitPolicy.class.getName());
-modifyTable(physicalTableName, newDesc, true);
 }
 return null;
 } else {



[2/2] phoenix git commit: PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be property driven (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be 
property driven (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-cdh5.11
Commit: dd8f1dab09aaec2b8c2ddead97c3d2f758c2b0a5
Parents: 83b0901
Author: James Taylor 
Authored: Fri Apr 13 14:30:00 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 14:46:45 2018 -0700

--
 .../phoenix/coprocessor/MetaDataProtocol.java   |  4 +--
 .../query/ConnectionQueryServicesImpl.java  | 32 ++--
 .../query/ConnectionlessQueryServicesImpl.java  | 29 --
 .../apache/phoenix/query/QueryConstants.java| 13 +++-
 .../org/apache/phoenix/query/QueryServices.java |  5 ++-
 .../phoenix/query/QueryServicesOptions.java |  3 +-
 .../org/apache/phoenix/util/UpgradeUtil.java|  4 +--
 7 files changed, 64 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/dd8f1dab/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index d6b2266..8a47d12 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -72,10 +72,10 @@ public abstract class MetaDataProtocol extends 
MetaDataService {
 public static final long MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP = 0;
 public static final String MIGRATION_IN_PROGRESS = "MigrationInProgress";
 
-public static final int DEFAULT_MAX_META_DATA_VERSIONS = 1000;
-public static final boolean DEFAULT_META_DATA_KEEP_DELETED_CELLS = true;
 public static final int DEFAULT_MAX_STAT_DATA_VERSIONS = 1;
 public static final boolean DEFAULT_STATS_KEEP_DELETED_CELLS = false;
+public static final int DEFAULT_LOG_VERSIONS = 10;
+public static final int DEFAULT_LOG_TTL = 7 * 24 * 60 * 60; // 7 days 
 
 // Min system table timestamps for every release.
 public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0 = 
MIN_TABLE_TIMESTAMP + 3;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/dd8f1dab/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 7ea6b42..b853842 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -702,10 +702,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 private HColumnDescriptor 
generateColumnFamilyDescriptor(Pair> family, 
PTableType tableType) throws SQLException {
 HColumnDescriptor columnDesc = new 
HColumnDescriptor(family.getFirst());
 if (tableType != PTableType.VIEW) {
-if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != 
null){
-columnDesc.setKeepDeletedCells(props.getBoolean(
-QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, 
QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
-}
 
columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
 columnDesc.setBloomFilterType(BloomType.NONE);
 for (Entry entry : family.getSecond().entrySet()) {
@@ -2442,8 +2438,24 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 }
 
 // Available for testing
-protected String getSystemCatalogDML() {
-return QueryConstants.CREATE_TABLE_METADATA;
+protected String getSystemCatalogTableDDL() {
+return setSystemDDLProperties(QueryConstants.CREATE_TABLE_METADATA);
+}
+
+// Available for testing
+protected String getFunctionTableDDL() {
+return setSystemDDLProperties(QueryConstants.CREATE_FUNCTION_METADATA);
+}
+
+private String setSystemDDLProperties(String ddl) {
+return String.format(ddl,
+  props.getInt(DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB, 
QueryServicesOptions.DEFAULT_SYSTEM_MAX_VERSIONS),
+  props.getBoolean(DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB, 
QueryServicesOptions.DEFAULT_SYSTEM_KEEP_DELETED_CEL

[2/2] phoenix git commit: PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be property driven (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be 
property driven (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: a816061c67a2ae812ef8fb39b15f1634e9f2ed79
Parents: 27c24ab
Author: James Taylor 
Authored: Fri Apr 13 14:30:00 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 15:01:59 2018 -0700

--
 .../phoenix/coprocessor/MetaDataProtocol.java   |  4 --
 .../query/ConnectionQueryServicesImpl.java  | 41 ++--
 .../query/ConnectionlessQueryServicesImpl.java  | 35 +++--
 .../apache/phoenix/query/QueryConstants.java| 14 +++
 .../org/apache/phoenix/query/QueryServices.java |  5 ++-
 .../phoenix/query/QueryServicesOptions.java |  3 +-
 .../org/apache/phoenix/util/UpgradeUtil.java|  4 +-
 7 files changed, 74 insertions(+), 32 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a816061c/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 26f8198..36d6f0d 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -72,10 +72,6 @@ public abstract class MetaDataProtocol extends 
MetaDataService {
 public static final long MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP = 0;
 public static final String MIGRATION_IN_PROGRESS = "MigrationInProgress";
 
-public static final int DEFAULT_MAX_META_DATA_VERSIONS = 1000;
-public static final boolean DEFAULT_META_DATA_KEEP_DELETED_CELLS = true;
-public static final int DEFAULT_MAX_STAT_DATA_VERSIONS = 1;
-public static final boolean DEFAULT_STATS_KEEP_DELETED_CELLS = false;
 public static final int DEFAULT_LOG_VERSIONS = 10;
 public static final int DEFAULT_LOG_TTL = 7 * 24 * 60 * 60; // 7 days 
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a816061c/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index c4a4d26..b61d8ce 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -719,10 +719,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 private HColumnDescriptor 
generateColumnFamilyDescriptor(Pair> family, 
PTableType tableType) throws SQLException {
 HColumnDescriptor columnDesc = new 
HColumnDescriptor(family.getFirst());
 if (tableType != PTableType.VIEW) {
-if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != 
null){
-columnDesc.setKeepDeletedCells(props.getBoolean(
-QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, 
QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
-}
 
columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
 columnDesc.setBloomFilterType(BloomType.NONE);
 for (Entry entry : family.getSecond().entrySet()) {
@@ -2456,8 +2452,29 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 }
 
 // Available for testing
-protected String getSystemCatalogDML() {
-return QueryConstants.CREATE_TABLE_METADATA;
+protected String getSystemCatalogTableDDL() {
+return setSystemDDLProperties(QueryConstants.CREATE_TABLE_METADATA);
+}
+
+// Available for testing
+protected String getFunctionTableDDL() {
+return setSystemDDLProperties(QueryConstants.CREATE_FUNCTION_METADATA);
+}
+
+// Available for testing
+protected String getLogTableDDL() {
+return setSystemLogDDLProperties(QueryConstants.CREATE_LOG_METADATA);
+}
+
+private String setSystemDDLProperties(String ddl) {
+return String.format(ddl,
+  props.getInt(DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB, 
QueryServicesOptions.DEFAULT_SYSTEM_MAX_VERSIONS),
+  props.getBoolean(DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB, 
QueryServicesOptions.DEFAULT_SYSTEM_KEEP_D

[1/2] phoenix git commit: PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY column (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 b28d11998 -> a816061c6


PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY 
column (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 27c24abe14e4c0b9b6416dd4472f38b484aaf17d
Parents: b28d119
Author: James Taylor 
Authored: Fri Apr 13 14:19:15 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 14:47:41 2018 -0700

--
 .../phoenix/query/ConnectionQueryServicesImpl.java   | 11 ---
 1 file changed, 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/27c24abe/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index c260845..c4a4d26 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -1128,10 +1128,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 
PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES
 {
 newDesc.setValue(HTableDescriptor.SPLIT_POLICY, 
IndexRegionSplitPolicy.class.getName());
 }
-// Remove the splitPolicy attribute to prevent HBASE-12570
-if (isMetaTable) {
-newDesc.remove(HTableDescriptor.SPLIT_POLICY);
-}
 try {
 if (splits == null) {
 admin.createTable(newDesc);
@@ -1148,13 +1144,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 }
 if (isMetaTable && !isUpgradeRequired()) {
 
checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES,
 this.getProps()).getName());
-/*
- * Now we modify the table to add the split policy, since 
we know that the client and
- * server and compatible. This works around HBASE-12570 
which causes the cluster to be
- * brought down.
- */
-newDesc.setValue(HTableDescriptor.SPLIT_POLICY, 
MetaDataSplitPolicy.class.getName());
-modifyTable(physicalTableName, newDesc, true);
 }
 return null;
 } else {



[1/2] phoenix git commit: PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY column (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 3e8049c91 -> fce9af534


PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY 
column (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 0a28d6aa8ac3197417353e32b8395738ac664ce7
Parents: 3e8049c
Author: James Taylor 
Authored: Fri Apr 13 14:19:15 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 15:04:26 2018 -0700

--
 .../phoenix/query/ConnectionQueryServicesImpl.java   | 11 ---
 1 file changed, 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0a28d6aa/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 6c316de..7a1317d 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -1112,10 +1112,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 
PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES
 {
 newDesc.setValue(HTableDescriptor.SPLIT_POLICY, 
IndexRegionSplitPolicy.class.getName());
 }
-// Remove the splitPolicy attribute to prevent HBASE-12570
-if (isMetaTable) {
-newDesc.remove(HTableDescriptor.SPLIT_POLICY);
-}
 try {
 if (splits == null) {
 admin.createTable(newDesc);
@@ -1132,13 +1128,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 }
 if (isMetaTable && !isUpgradeRequired()) {
 
checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES,
 this.getProps()).getName());
-/*
- * Now we modify the table to add the split policy, since 
we know that the client and
- * server and compatible. This works around HBASE-12570 
which causes the cluster to be
- * brought down.
- */
-newDesc.setValue(HTableDescriptor.SPLIT_POLICY, 
MetaDataSplitPolicy.class.getName());
-modifyTable(physicalTableName, newDesc, true);
 }
 return null;
 } else {



[2/2] phoenix git commit: PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be property driven (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be 
property driven (Chinmay Kulkarni)


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

Branch: refs/heads/4.x-HBase-0.98
Commit: fce9af53423d1804f5dc8daf8f28f9f7b38bbff5
Parents: 0a28d6a
Author: James Taylor 
Authored: Fri Apr 13 14:30:00 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 15:04:44 2018 -0700

--
 .../phoenix/coprocessor/MetaDataProtocol.java   |  4 +--
 .../query/ConnectionQueryServicesImpl.java  | 32 ++--
 .../query/ConnectionlessQueryServicesImpl.java  | 29 --
 .../apache/phoenix/query/QueryConstants.java| 13 +++-
 .../org/apache/phoenix/query/QueryServices.java |  5 ++-
 .../phoenix/query/QueryServicesOptions.java |  3 +-
 .../org/apache/phoenix/util/UpgradeUtil.java|  4 +--
 7 files changed, 64 insertions(+), 26 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/fce9af53/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index d6b2266..8a47d12 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -72,10 +72,10 @@ public abstract class MetaDataProtocol extends 
MetaDataService {
 public static final long MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP = 0;
 public static final String MIGRATION_IN_PROGRESS = "MigrationInProgress";
 
-public static final int DEFAULT_MAX_META_DATA_VERSIONS = 1000;
-public static final boolean DEFAULT_META_DATA_KEEP_DELETED_CELLS = true;
 public static final int DEFAULT_MAX_STAT_DATA_VERSIONS = 1;
 public static final boolean DEFAULT_STATS_KEEP_DELETED_CELLS = false;
+public static final int DEFAULT_LOG_VERSIONS = 10;
+public static final int DEFAULT_LOG_TTL = 7 * 24 * 60 * 60; // 7 days 
 
 // Min system table timestamps for every release.
 public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0 = 
MIN_TABLE_TIMESTAMP + 3;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/fce9af53/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 7a1317d..c5d5e0d 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -703,10 +703,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 private HColumnDescriptor 
generateColumnFamilyDescriptor(Pair> family, 
PTableType tableType) throws SQLException {
 HColumnDescriptor columnDesc = new 
HColumnDescriptor(family.getFirst());
 if (tableType != PTableType.VIEW) {
-if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != 
null){
-columnDesc.setKeepDeletedCells(props.getBoolean(
-QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, 
QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
-}
 
columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
 columnDesc.setBloomFilterType(BloomType.NONE);
 for (Entry entry : family.getSecond().entrySet()) {
@@ -2436,8 +2432,24 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 }
 
 // Available for testing
-protected String getSystemCatalogDML() {
-return QueryConstants.CREATE_TABLE_METADATA;
+protected String getSystemCatalogTableDDL() {
+return setSystemDDLProperties(QueryConstants.CREATE_TABLE_METADATA);
+}
+
+// Available for testing
+protected String getFunctionTableDDL() {
+return setSystemDDLProperties(QueryConstants.CREATE_FUNCTION_METADATA);
+}
+
+private String setSystemDDLProperties(String ddl) {
+return String.format(ddl,
+  props.getInt(DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB, 
QueryServicesOptions.DEFAULT_SYSTEM_MAX_VERSIONS),
+  props.getBoolean(DEFAULT_SYSTEM_KEEP_DELETED_CELLS_ATTRIB, 
QueryServicesOptions.DEFAULT_SYSTEM_KEEP_DELETED_

[2/2] phoenix git commit: PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be property driven (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
PHOENIX-4575 Phoenix metadata KEEP_DELETED_CELLS and VERSIONS should be 
property driven (Chinmay Kulkarni)


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

Branch: refs/heads/5.x-HBase-2.0
Commit: 2a8f0c0fe5b0afd2f154bbf0ad0932abfebdeda8
Parents: bbf454b
Author: James Taylor 
Authored: Fri Apr 13 14:30:00 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 15:17:18 2018 -0700

--
 .../phoenix/coprocessor/MetaDataProtocol.java   |  4 --
 .../query/ConnectionQueryServicesImpl.java  | 42 ++--
 .../query/ConnectionlessQueryServicesImpl.java  | 35 ++--
 .../apache/phoenix/query/QueryConstants.java| 14 +++
 .../org/apache/phoenix/query/QueryServices.java |  5 ++-
 .../phoenix/query/QueryServicesOptions.java |  3 +-
 .../org/apache/phoenix/util/UpgradeUtil.java|  4 +-
 7 files changed, 74 insertions(+), 33 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8f0c0f/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 4af85bf..8f33d41 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -72,10 +72,6 @@ public abstract class MetaDataProtocol extends 
MetaDataService {
 public static final long MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP = 0;
 public static final String MIGRATION_IN_PROGRESS = "MigrationInProgress";
 
-public static final int DEFAULT_MAX_META_DATA_VERSIONS = 1000;
-public static final boolean DEFAULT_META_DATA_KEEP_DELETED_CELLS = true;
-public static final int DEFAULT_MAX_STAT_DATA_VERSIONS = 1;
-public static final boolean DEFAULT_STATS_KEEP_DELETED_CELLS = false;
 public static final int DEFAULT_LOG_VERSIONS = 10;
 public static final int DEFAULT_LOG_TTL = 7 * 24 * 60 * 60; // 7 days 
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8f0c0f/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index bb94313..ffce2ea 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -726,11 +726,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 private ColumnFamilyDescriptor 
generateColumnFamilyDescriptor(Pair> family, 
PTableType tableType) throws SQLException {
 ColumnFamilyDescriptorBuilder columnDescBuilder = 
ColumnFamilyDescriptorBuilder.newBuilder(family.getFirst());
 if (tableType != PTableType.VIEW) {
-if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != 
null){
-
columnDescBuilder.setKeepDeletedCells(props.getBoolean(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB,
-QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS) ? 
KeepDeletedCells.TRUE
-: KeepDeletedCells.FALSE);
-}
 
columnDescBuilder.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
 columnDescBuilder.setBloomFilterType(BloomType.NONE);
 for (Entry entry : family.getSecond().entrySet()) {
@@ -2494,8 +2489,29 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 }
 
 // Available for testing
-protected String getSystemCatalogDML() {
-return QueryConstants.CREATE_TABLE_METADATA;
+protected String getSystemCatalogTableDDL() {
+return setSystemDDLProperties(QueryConstants.CREATE_TABLE_METADATA);
+}
+
+// Available for testing
+protected String getFunctionTableDDL() {
+return setSystemDDLProperties(QueryConstants.CREATE_FUNCTION_METADATA);
+}
+
+// Available for testing
+protected String getLogTableDDL() {
+return setSystemLogDDLProperties(QueryConstants.CREATE_LOG_METADATA);
+}
+
+private String setSystemDDLProperties(String ddl) {
+return String.format(ddl,
+  props.getInt(DEFAULT_SYSTEM_MAX_VERSIONS_ATTRIB, 
QueryServicesOption

[1/2] phoenix git commit: PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY column (Chinmay Kulkarni)

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/5.x-HBase-2.0 d30d77186 -> 2a8f0c0fe


PHOENIX-4668 Remove unnecessary table descriptor modification for SPLIT_POLICY 
column (Chinmay Kulkarni)


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

Branch: refs/heads/5.x-HBase-2.0
Commit: bbf454b3eb9d3fb38622ca757577fdb78bcdbb73
Parents: d30d771
Author: James Taylor 
Authored: Fri Apr 13 14:19:15 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 15:07:17 2018 -0700

--
 .../phoenix/query/ConnectionQueryServicesImpl.java   | 11 ---
 1 file changed, 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bbf454b3/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 446bbcf..bb94313 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -1140,10 +1140,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 
PBoolean.INSTANCE.toObject(newDesc.build().getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES
 {
 
newDesc.setRegionSplitPolicyClassName(IndexRegionSplitPolicy.class.getName());
 }
-// Remove the splitPolicy attribute to prevent HBASE-12570
-if (isMetaTable) {
-
newDesc.removeValue(Bytes.toBytes(TableDescriptorBuilder.SPLIT_POLICY));
-}
 try {
 if (splits == null) {
 admin.createTable(newDesc.build());
@@ -1160,13 +1156,6 @@ public class ConnectionQueryServicesImpl extends 
DelegateQueryServices implement
 }
 if (isMetaTable && !isUpgradeRequired()) {
 
checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES,
 this.getProps()).getName());
-/*
- * Now we modify the table to add the split policy, since 
we know that the client and
- * server and compatible. This works around HBASE-12570 
which causes the cluster to be
- * brought down.
- */
-
newDesc.setRegionSplitPolicyClassName(MetaDataSplitPolicy.class.getName());
-modifyTable(physicalTableName, newDesc.build(), true);
 }
 return null;
 } else {



[1/2] phoenix git commit: Revert "PHOENIX-4579 Remove SYSTEM.LOG from list of Phoenix system tables since that JIRA hasn't been backported"

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 a816061c6 -> 83ad23241


Revert "PHOENIX-4579 Remove SYSTEM.LOG from list of Phoenix system tables since 
that JIRA hasn't been backported"

This reverts commit b28d1199879dee00686467e24e8683488818ff8e.


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

Branch: refs/heads/4.x-HBase-1.1
Commit: adb98a163d5fd4b0fb1abca3e8cc9d648e951ddb
Parents: a816061
Author: James Taylor 
Authored: Fri Apr 13 23:19:01 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 23:19:01 2018 -0700

--
 .../SystemCatalogCreationOnConnectionIT.java| 34 +++-
 1 file changed, 11 insertions(+), 23 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/adb98a16/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
index a2bc272..689eb20 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
@@ -17,22 +17,6 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.TimeoutException;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -44,11 +28,7 @@ import org.apache.phoenix.exception.UpgradeRequiredException;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
 import org.apache.phoenix.jdbc.PhoenixTestDriver;
-import org.apache.phoenix.query.ConnectionQueryServices;
-import org.apache.phoenix.query.ConnectionQueryServicesImpl;
-import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.query.QueryServicesTestImpl;
+import org.apache.phoenix.query.*;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.UpgradeUtil;
 import org.junit.After;
@@ -56,6 +36,14 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.*;
+import java.util.concurrent.TimeoutException;
+
+import static org.junit.Assert.*;
+
 @Category(NeedsOwnMiniClusterTest.class)
 public class SystemCatalogCreationOnConnectionIT {
 private HBaseTestingUtility testUtil = null;
@@ -72,11 +60,11 @@ public class SystemCatalogCreationOnConnectionIT {
 
 private static final Set PHOENIX_SYSTEM_TABLES = new 
HashSet<>(Arrays.asList(
   "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION",
-  "SYSTEM.MUTEX"));
+  "SYSTEM.MUTEX", "SYSTEM.LOG"));
 
 private static final Set PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = 
new HashSet<>(
   Arrays.asList("SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", 
"SYSTEM:FUNCTION",
-"SYSTEM:MUTEX"));
+"SYSTEM:MUTEX", "SYSTEM:LOG"));
 
 private static class PhoenixSysCatCreationServices extends 
ConnectionQueryServicesImpl {
 



[2/2] phoenix git commit: more

2018-04-13 Thread jamestaylor
more


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 83ad2324121d52d752678acff56ec96ece880c64
Parents: adb98a1
Author: James Taylor 
Authored: Fri Apr 13 23:23:09 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 23:23:09 2018 -0700

--
 .../SystemCatalogCreationOnConnectionIT.java| 32 ++--
 1 file changed, 22 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/83ad2324/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
index 689eb20..fbc8cd7 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
@@ -17,6 +17,22 @@
  */
 package org.apache.phoenix.end2end;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -28,7 +44,11 @@ import org.apache.phoenix.exception.UpgradeRequiredException;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
 import org.apache.phoenix.jdbc.PhoenixTestDriver;
-import org.apache.phoenix.query.*;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.ConnectionQueryServicesImpl;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesTestImpl;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.UpgradeUtil;
 import org.junit.After;
@@ -36,14 +56,6 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.util.*;
-import java.util.concurrent.TimeoutException;
-
-import static org.junit.Assert.*;
-
 @Category(NeedsOwnMiniClusterTest.class)
 public class SystemCatalogCreationOnConnectionIT {
 private HBaseTestingUtility testUtil = null;
@@ -623,4 +635,4 @@ public class SystemCatalogCreationOnConnectionIT {
 assertEquals(0, countUpgradeAttempts);
 return driver;
 }
-}
\ No newline at end of file
+}



phoenix git commit: PHOENIX-4579 Fix import order [Forced Update!]

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 83ad23241 -> ea709974d (forced update)


PHOENIX-4579 Fix import order


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

Branch: refs/heads/4.x-HBase-1.1
Commit: ea709974d69ff9becc83e09d98a280d542475a7e
Parents: adb98a1
Author: James Taylor 
Authored: Fri Apr 13 23:23:09 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 23:27:03 2018 -0700

--
 .../SystemCatalogCreationOnConnectionIT.java| 32 ++--
 1 file changed, 22 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ea709974/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
index 689eb20..fbc8cd7 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
@@ -17,6 +17,22 @@
  */
 package org.apache.phoenix.end2end;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HConstants;
@@ -28,7 +44,11 @@ import org.apache.phoenix.exception.UpgradeRequiredException;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver;
 import org.apache.phoenix.jdbc.PhoenixTestDriver;
-import org.apache.phoenix.query.*;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.ConnectionQueryServicesImpl;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesTestImpl;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.UpgradeUtil;
 import org.junit.After;
@@ -36,14 +56,6 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.util.*;
-import java.util.concurrent.TimeoutException;
-
-import static org.junit.Assert.*;
-
 @Category(NeedsOwnMiniClusterTest.class)
 public class SystemCatalogCreationOnConnectionIT {
 private HBaseTestingUtility testUtil = null;
@@ -623,4 +635,4 @@ public class SystemCatalogCreationOnConnectionIT {
 assertEquals(0, countUpgradeAttempts);
 return driver;
 }
-}
\ No newline at end of file
+}



phoenix git commit: Revert PHOENIX-4579 Remove SYSTEM.LOG from list of Phoenix system tables since that JIRA hasn't been backported

2018-04-13 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 28d74053f -> 07125f25e


Revert PHOENIX-4579 Remove SYSTEM.LOG from list of Phoenix system tables since 
that JIRA hasn't been backported


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 07125f25e2f1ea7d2a69cda105c68a75fac02ec8
Parents: 28d7405
Author: James Taylor 
Authored: Fri Apr 13 23:32:28 2018 -0700
Committer: James Taylor 
Committed: Fri Apr 13 23:32:28 2018 -0700

--
 .../phoenix/end2end/SystemCatalogCreationOnConnectionIT.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/07125f25/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
index a2bc272..beb31c2 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogCreationOnConnectionIT.java
@@ -72,11 +72,11 @@ public class SystemCatalogCreationOnConnectionIT {
 
 private static final Set PHOENIX_SYSTEM_TABLES = new 
HashSet<>(Arrays.asList(
   "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION",
-  "SYSTEM.MUTEX"));
+  "SYSTEM.MUTEX", "SYSTEM.LOG"));
 
 private static final Set PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = 
new HashSet<>(
   Arrays.asList("SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", 
"SYSTEM:FUNCTION",
-"SYSTEM:MUTEX"));
+"SYSTEM:MUTEX", "SYSTEM.LOG"));
 
 private static class PhoenixSysCatCreationServices extends 
ConnectionQueryServicesImpl {
 
@@ -635,4 +635,4 @@ public class SystemCatalogCreationOnConnectionIT {
 assertEquals(0, countUpgradeAttempts);
 return driver;
 }
-}
\ No newline at end of file
+}



[4/6] phoenix git commit: PHOENIX-4698 Tolerate orphaned views (Maddineni Sukumar)

2018-04-26 Thread jamestaylor
PHOENIX-4698 Tolerate orphaned views (Maddineni Sukumar)


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

Branch: refs/heads/4.x-HBase-1.3
Commit: 7096a682c3bd1f6754b8294c706ebb38c00e0af1
Parents: f9369f8
Author: James Taylor 
Authored: Thu Apr 19 14:42:24 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:51:01 2018 -0700

--
 .../coprocessor/MetaDataEndpointImpl.java   | 50 +---
 1 file changed, 33 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7096a682/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 c28ad3c..ddd3ffe 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
@@ -592,7 +592,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso
 done.run(builder.build());
 return;
 } catch (Throwable t) {
-   logger.error("getTable failed", t);
+logger.error("getTable failed", t);
 ProtobufUtil.setControllerException(controller,
 
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
 }
@@ -755,9 +755,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso

// compatibility.
 Cell sortOrderKv = colKeyValues[SORT_ORDER_INDEX];
 SortOrder sortOrder =
-   sortOrderKv == null ? SortOrder.getDefault() : 
SortOrder.fromSystemValue(PInteger.INSTANCE
+sortOrderKv == null ? SortOrder.getDefault() : 
SortOrder.fromSystemValue(PInteger.INSTANCE
 .getCodec().decodeInt(sortOrderKv.getValueArray(),
-   sortOrderKv.getValueOffset(), 
SortOrder.getDefault()));
+sortOrderKv.getValueOffset(), 
SortOrder.getDefault()));
 
 Cell arraySizeKv = colKeyValues[ARRAY_SIZE_INDEX];
 Integer arraySize = arraySizeKv == null ? null :
@@ -1313,9 +1313,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 return table.getName() == null;
 }
 
-   private static boolean isSchemaDeleted(PSchema schema) {
-   return schema.getSchemaName() == null;
-   }
+private static boolean isSchemaDeleted(PSchema schema) {
+return schema.getSchemaName() == null;
+}
 
 private static boolean isFunctionDeleted(PFunction function) {
 return function.getFunctionName() == null;
@@ -1827,6 +1827,13 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 byte[] tableKey = SchemaUtil.getTableKey(viewtenantId, viewSchema, 
viewTable);
 ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
 PTable view = loadTable(env, tableKey, cacheKey, clientTimeStamp, 
clientTimeStamp, clientVersion);
+if (view == null) {
+logger.warn("Found orphan tenant view row in SYSTEM.CATALOG 
with tenantId:"
++ Bytes.toString(tenantId) + ", schema:"
++ Bytes.toString(viewSchema) + ", table:"
++ Bytes.toString(viewTable));
+continue;
+}
 findAllChildViews(region, viewtenantId, view, result, 
clientTimeStamp, clientVersion);
 }
 }
@@ -1969,8 +1976,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 if (systemCatalog.getTimeStamp() < MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0) {
 return findChildViews_deprecated(region, tenantId, table, 
PHYSICAL_TABLE_BYTES, stopAfterFirst);
 } else {
-return findChildViews_4_11(region, tenantId, 
table.getSchemaName().getBytes(),
-table.getTableName().getBytes(), stopAfterFirst);
+return findChildViews_4_11(region, tenantId, 
+table.getSchemaName() == null ? ByteUtil.EMPTY_BYTE_ARRAY 
: table.getSchemaName().getBytes(),
+table.getTableName().getBytes(), stopAfterFirst);
 }
 }
 
@@ -2548,6 +2556,14 @@ public class MetaData

[6/6] phoenix git commit: PHOENIX-4700 Fix split policy on system tables other than SYSTEM.CATALOG

2018-04-26 Thread jamestaylor
PHOENIX-4700 Fix split policy on system tables other than SYSTEM.CATALOG


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

Branch: refs/heads/4.x-HBase-1.3
Commit: 4b7a14cd0dc04c368366c8871531cdb41c7283d5
Parents: 7096a68
Author: James Taylor 
Authored: Mon Apr 23 10:14:36 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:51:01 2018 -0700

--
 .../apache/phoenix/query/QueryConstants.java| 117 +--
 .../SplitOnLeadingVarCharColumnsPolicy.java |  43 +++
 .../schema/SystemFunctionSplitPolicy.java   |  27 +
 .../phoenix/schema/SystemStatsSplitPolicy.java  |  27 +
 .../phoenix/schema/SystemSplitPolicyTest.java   |  97 +++
 5 files changed, 200 insertions(+), 111 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4b7a14cd/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java 
b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index 22fa2f4..4285334 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -18,112 +18,7 @@
 package org.apache.phoenix.query;
 
 
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARG_POSITION;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BASE_COLUMN_COUNT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BUFFER_LENGTH;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CACHE_SIZE;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CHAR_OCTET_LENGTH;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_DEF;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_FAMILY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CURRENT_VALUE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CYCLE_FLAG;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TABLE_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TYPE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DECIMAL_DIGITS;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_VALUE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DISABLE_WAL;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ENCODING_SCHEME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FUNCTION_NAME;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POST_KEY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INCREMENT_BY;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_TYPE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ARRAY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_AUTOINCREMENT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_CONSTANT;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NULLABLE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ROW_TIMESTAMP;
-import static 
org.apac

[3/6] phoenix git commit: PHOENIX-4694 Prevent locking of parent table when dropping view to reduce contention

2018-04-26 Thread jamestaylor
PHOENIX-4694 Prevent locking of parent table when dropping view to reduce 
contention


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

Branch: refs/heads/4.x-HBase-1.3
Commit: 2e1771261656002729b5943f2c4b2050a7bb8a1b
Parents: 4b7a14c
Author: James Taylor 
Authored: Thu Apr 19 15:09:43 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:51:01 2018 -0700

--
 .../java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/2e177126/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 ddd3ffe..29eee7e 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
@@ -2007,8 +2007,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 }
 List tableNamesToDelete = Lists.newArrayList();
 List sharedTablesToDelete = Lists.newArrayList();
+// No need to lock parent table for views
 byte[] parentTableName = 
MetaDataUtil.getParentTableName(tableMetadata);
-byte[] lockTableName = parentTableName == null ? tableName : 
parentTableName;
+byte[] lockTableName = parentTableName == null || 
tableType.equals(PTableType.VIEW.getSerializedValue()) ? tableName : 
parentTableName;
 byte[] lockKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, 
lockTableName);
 byte[] key =
 parentTableName == null ? lockKey : 
SchemaUtil.getTableKey(tenantIdBytes,



[1/6] phoenix git commit: PHOENIX-4699 Stop scan after finding first child of table during drop

2018-04-26 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.3 e578a869c -> 59ab3f15c


PHOENIX-4699 Stop scan after finding first child of table during drop


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

Branch: refs/heads/4.x-HBase-1.3
Commit: f9369f8f973456832bdcfcaf669f0e067bf73461
Parents: e578a86
Author: James Taylor 
Authored: Thu Apr 19 14:39:27 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:49:35 2018 -0700

--
 .../coprocessor/MetaDataEndpointImpl.java   | 44 +++-
 1 file changed, 25 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f9369f8f/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 4c72c2d..c28ad3c 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
@@ -122,8 +122,10 @@ import 
org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.PageFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.ipc.RpcServer.Call;
@@ -275,6 +277,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso
 public static final String ROW_KEY_ORDER_OPTIMIZABLE = 
"ROW_KEY_ORDER_OPTIMIZABLE";
 public static final byte[] ROW_KEY_ORDER_OPTIMIZABLE_BYTES = 
Bytes.toBytes(ROW_KEY_ORDER_OPTIMIZABLE);
 
+private static final byte[] CHILD_TABLE_BYTES = new byte[] 
{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
+private static final byte[] PHYSICAL_TABLE_BYTES =
+new byte[] { PTable.LinkType.PHYSICAL_TABLE.getSerializedValue() };
+
 // KeyValues for Table
 private static final KeyValue TABLE_TYPE_KV = 
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, 
TABLE_TYPE_BYTES);
 private static final KeyValue TABLE_SEQ_NUM_KV = 
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, 
TABLE_SEQ_NUM_BYTES);
@@ -1527,7 +1533,6 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 for (PTable index : parentTable.getIndexes()) {
 
indexes.add(TableName.valueOf(index.getPhysicalName().getBytes()));
 }
-
 } else {
 // Mapped View
 cParentPhysicalName = 
SchemaUtil.getTableNameAsBytes(schemaName, tableName);
@@ -1810,13 +1815,10 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 private boolean execeededIndexQuota(PTableType tableType, PTable 
parentTable) {
 return PTableType.INDEX == tableType && 
parentTable.getIndexes().size() >= maxIndexesPerTable;
 }
-
-private static final byte[] CHILD_TABLE_BYTES = new byte[] 
{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
-
 
 private void findAllChildViews(Region region, byte[] tenantId, PTable 
table,
 TableViewFinder result, long clientTimeStamp, int clientVersion) 
throws IOException, SQLException {
-TableViewFinder currResult = findChildViews(region, tenantId, table, 
clientVersion);
+TableViewFinder currResult = findChildViews(region, tenantId, table, 
clientVersion, false);
 result.addResult(currResult);
 for (ViewInfo viewInfo : currResult.getViewInfoList()) {
 byte[] viewtenantId = viewInfo.getTenantId();
@@ -1829,9 +1831,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 }
 }
 
-// TODO remove this in 4.13 release 
-@Deprecated
-private TableViewFinder findChildViews_deprecated(Region region, byte[] 
tenantId, PTable table, byte[] linkTypeBytes) throws IOException {
+// TODO use child link instead once splittable system catalog 
(PHOENIX-3534) is implemented
+// and we have a separate table for l

[2/6] phoenix git commit: PHOENIX-4708 Do not propagate GUIDE_POSTS_WIDTH to children

2018-04-26 Thread jamestaylor
PHOENIX-4708 Do not propagate GUIDE_POSTS_WIDTH to children


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

Branch: refs/heads/4.x-HBase-1.3
Commit: 59ab3f15cb11ade9d85d6175d3f8697f329bd0b9
Parents: b173aaf
Author: James Taylor 
Authored: Thu Apr 26 09:40:35 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:51:01 2018 -0700

--
 .../phoenix/end2end/AlterTableWithViewsIT.java  |  22 ++
 .../coprocessor/MetaDataEndpointImpl.java   | 115 +---
 .../apache/phoenix/schema/TableProperty.java| 270 ++-
 3 files changed, 248 insertions(+), 159 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/59ab3f15/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
index 237a8d2..e1b1372 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.sql.Connection;
+import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -39,12 +40,14 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.TephraTransactionalProcessor;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.util.StringUtil;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -166,6 +169,25 @@ public class AlterTableWithViewsIT extends 
ParallelStatsDisabledIT {
 assertTrue(viewTable2.isImmutableRows());
 // update cache frequency is not propagated to the view since it 
was altered on the view
 assertEquals(1, viewTable2.getUpdateCacheFrequency());
+
+long gpw = 100;
+conn.createStatement().execute("ALTER TABLE " + tableName + " SET 
GUIDE_POSTS_WIDTH=" + gpw);
+
+ResultSet rs;
+DatabaseMetaData md = conn.getMetaData();
+rs = md.getTables("", "", StringUtil.escapeLike(tableName), null);
+assertTrue(rs.next());
+assertEquals(gpw, 
rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH));
+
+rs = md.getTables(null, "", StringUtil.escapeLike(viewOfTable1), 
null);
+assertTrue(rs.next());
+rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH);
+assertTrue(rs.wasNull());
+
+rs = md.getTables(null, "", StringUtil.escapeLike(viewOfTable2), 
null);
+assertTrue(rs.next());
+rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH);
+assertTrue(rs.wasNull());
 } 
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/59ab3f15/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 29eee7e..b77f113 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
@@ -91,6 +91,7 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
@@ -3163,7 +3164,56 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 byte[] tableNameBytes = index.getTableName().getBytes();
 return ByteUtil.concat(tenantIdBytes, SEPARATOR_BYTE_ARRAY, 
schemaNameBytes, SEPARATOR_BYTE_ARRAY, tableNameBytes);
 }
+  
 
+/**
+ * Determines whether or not we hav

[5/6] phoenix git commit: PHOENIX-4686 Phoenix stats does not account for server side limit push downs (Abhishek Chouhan)

2018-04-26 Thread jamestaylor
PHOENIX-4686 Phoenix stats does not account for server side limit push downs 
(Abhishek Chouhan)


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

Branch: refs/heads/4.x-HBase-1.3
Commit: b173aaf79b282c8be1564d2be9981e7c11d40686
Parents: 2e17712
Author: James Taylor 
Authored: Thu Apr 26 09:14:52 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:51:01 2018 -0700

--
 .../end2end/ExplainPlanWithStatsEnabledIT.java  | 49 ++-
 .../org/apache/phoenix/execute/ScanPlan.java|  4 +-
 .../phoenix/iterate/BaseResultIterators.java| 50 +---
 .../phoenix/iterate/ParallelIterators.java  |  8 
 .../apache/phoenix/iterate/SerialIterators.java | 13 ++---
 .../phoenix/schema/stats/StatisticsUtil.java|  6 +++
 6 files changed, 111 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b173aaf7/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index 2099f4c..abaa2f6 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -40,6 +40,7 @@ import 
org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.schema.stats.StatisticsUtil;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EnvironmentEdge;
@@ -123,6 +124,50 @@ public class ExplainPlanWithStatsEnabledIT extends 
ParallelStatsEnabledIT {
 }
 
 @Test
+public void 
testBytesRowsForPointSelectWithLimitGreaterThanPointLookupSize() throws 
Exception {
+String sql = "SELECT * FROM " + tableA + " where k in (? ,?) limit 4";
+List binds = Lists.newArrayList();
+binds.add(103); binds.add(104);
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+Estimate info = getByteRowEstimates(conn, sql, binds);
+assertEquals((Long) 200L, info.estimatedBytes);
+assertEquals((Long) 2L, info.estimatedRows);
+assertEquals((Long) StatisticsUtil.NOT_STATS_BASED_TS, 
info.estimateInfoTs);
+}
+}
+
+@Test
+public void testBytesRowsForSelectWithLimit() throws Exception {
+String sql = "SELECT * FROM " + tableA + " where c1.a in (?,?) limit 
3";
+String noIndexSQL = "SELECT /*+ NO_INDEX */ * FROM " + tableA + " 
where c1.a in (?,?) limit 3";
+List binds = Lists.newArrayList();
+binds.add(1); binds.add(2);
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+Estimate info = getByteRowEstimates(conn, sql, binds);
+assertEquals((Long) 264L, info.estimatedBytes);
+assertEquals((Long) 3L, info.estimatedRows);
+assertEquals((Long) StatisticsUtil.NOT_STATS_BASED_TS, 
info.estimateInfoTs);
+
+info = getByteRowEstimates(conn, noIndexSQL, binds);
+assertEquals((Long) 634L, info.estimatedBytes);
+assertEquals((Long) 10L, info.estimatedRows);
+assertTrue(info.estimateInfoTs > 0);
+}
+}
+
+@Test
+public void testBytesRowsForSelectWithLimitIgnored() throws Exception {
+String sql = "SELECT * FROM " + tableA + " where (c1.a > c2.b) limit 
1";
+List binds = Lists.newArrayList();
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+Estimate info = getByteRowEstimates(conn, sql, binds);
+assertEquals((Long) 691L, info.estimatedBytes);
+assertEquals((Long) 10L, info.estimatedRows);
+assertTrue(info.estimateInfoTs > 0);
+}
+}
+
+@Test
 public void testBytesRowsForSelectWhenKeyInRange() throws Exception {
 String sql = "SELECT * FROM " + tableB + " where k >= ?";
 List binds = Lists.newArrayList();
@@ -278,7 +323,7 @@ public class ExplainPlanWithStatsEnabledIT extends 
ParallelStatsEnabledIT {
 Estimate info = getByteRowEstimates(conn, sql, binds);
 assertEquals((Long) 200L, info.estimatedBytes);
 assertEquals((Long) 2L, info.estimatedRows);
-assertTru

[3/6] phoenix git commit: PHOENIX-4694 Prevent locking of parent table when dropping view to reduce contention

2018-04-26 Thread jamestaylor
PHOENIX-4694 Prevent locking of parent table when dropping view to reduce 
contention


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

Branch: refs/heads/master
Commit: 9168f6698d6afdf12a825bd6a4e05ccb2a85b9d0
Parents: 5c637e6
Author: James Taylor 
Authored: Thu Apr 19 15:09:43 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:55:28 2018 -0700

--
 .../java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/9168f669/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 ddd3ffe..29eee7e 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
@@ -2007,8 +2007,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 }
 List tableNamesToDelete = Lists.newArrayList();
 List sharedTablesToDelete = Lists.newArrayList();
+// No need to lock parent table for views
 byte[] parentTableName = 
MetaDataUtil.getParentTableName(tableMetadata);
-byte[] lockTableName = parentTableName == null ? tableName : 
parentTableName;
+byte[] lockTableName = parentTableName == null || 
tableType.equals(PTableType.VIEW.getSerializedValue()) ? tableName : 
parentTableName;
 byte[] lockKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, 
lockTableName);
 byte[] key =
 parentTableName == null ? lockKey : 
SchemaUtil.getTableKey(tenantIdBytes,



[4/6] phoenix git commit: PHOENIX-4698 Tolerate orphaned views (Maddineni Sukumar)

2018-04-26 Thread jamestaylor
PHOENIX-4698 Tolerate orphaned views (Maddineni Sukumar)


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

Branch: refs/heads/master
Commit: 8dd637dc41ae9fa1cb9c3111481e216059f95c34
Parents: 041e7b5
Author: James Taylor 
Authored: Thu Apr 19 14:42:24 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:55:28 2018 -0700

--
 .../coprocessor/MetaDataEndpointImpl.java   | 50 +---
 1 file changed, 33 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8dd637dc/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 c28ad3c..ddd3ffe 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
@@ -592,7 +592,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso
 done.run(builder.build());
 return;
 } catch (Throwable t) {
-   logger.error("getTable failed", t);
+logger.error("getTable failed", t);
 ProtobufUtil.setControllerException(controller,
 
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
 }
@@ -755,9 +755,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso

// compatibility.
 Cell sortOrderKv = colKeyValues[SORT_ORDER_INDEX];
 SortOrder sortOrder =
-   sortOrderKv == null ? SortOrder.getDefault() : 
SortOrder.fromSystemValue(PInteger.INSTANCE
+sortOrderKv == null ? SortOrder.getDefault() : 
SortOrder.fromSystemValue(PInteger.INSTANCE
 .getCodec().decodeInt(sortOrderKv.getValueArray(),
-   sortOrderKv.getValueOffset(), 
SortOrder.getDefault()));
+sortOrderKv.getValueOffset(), 
SortOrder.getDefault()));
 
 Cell arraySizeKv = colKeyValues[ARRAY_SIZE_INDEX];
 Integer arraySize = arraySizeKv == null ? null :
@@ -1313,9 +1313,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 return table.getName() == null;
 }
 
-   private static boolean isSchemaDeleted(PSchema schema) {
-   return schema.getSchemaName() == null;
-   }
+private static boolean isSchemaDeleted(PSchema schema) {
+return schema.getSchemaName() == null;
+}
 
 private static boolean isFunctionDeleted(PFunction function) {
 return function.getFunctionName() == null;
@@ -1827,6 +1827,13 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 byte[] tableKey = SchemaUtil.getTableKey(viewtenantId, viewSchema, 
viewTable);
 ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
 PTable view = loadTable(env, tableKey, cacheKey, clientTimeStamp, 
clientTimeStamp, clientVersion);
+if (view == null) {
+logger.warn("Found orphan tenant view row in SYSTEM.CATALOG 
with tenantId:"
++ Bytes.toString(tenantId) + ", schema:"
++ Bytes.toString(viewSchema) + ", table:"
++ Bytes.toString(viewTable));
+continue;
+}
 findAllChildViews(region, viewtenantId, view, result, 
clientTimeStamp, clientVersion);
 }
 }
@@ -1969,8 +1976,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 if (systemCatalog.getTimeStamp() < MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0) {
 return findChildViews_deprecated(region, tenantId, table, 
PHYSICAL_TABLE_BYTES, stopAfterFirst);
 } else {
-return findChildViews_4_11(region, tenantId, 
table.getSchemaName().getBytes(),
-table.getTableName().getBytes(), stopAfterFirst);
+return findChildViews_4_11(region, tenantId, 
+table.getSchemaName() == null ? ByteUtil.EMPTY_BYTE_ARRAY 
: table.getSchemaName().getBytes(),
+table.getTableName().getBytes(), stopAfterFirst);
 }
 }
 
@@ -2548,6 +2556,14 @@ public class MetaDataEndpoin

[6/6] phoenix git commit: PHOENIX-4700 Fix split policy on system tables other than SYSTEM.CATALOG

2018-04-26 Thread jamestaylor
PHOENIX-4700 Fix split policy on system tables other than SYSTEM.CATALOG


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

Branch: refs/heads/master
Commit: 5c637e6dd4aabb9d8afe0e07a81533a96e111bfb
Parents: 8dd637d
Author: James Taylor 
Authored: Mon Apr 23 10:14:36 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:55:28 2018 -0700

--
 .../apache/phoenix/query/QueryConstants.java| 117 +--
 .../SplitOnLeadingVarCharColumnsPolicy.java |  43 +++
 .../schema/SystemFunctionSplitPolicy.java   |  27 +
 .../phoenix/schema/SystemStatsSplitPolicy.java  |  27 +
 .../phoenix/schema/SystemSplitPolicyTest.java   |  97 +++
 5 files changed, 200 insertions(+), 111 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5c637e6d/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java 
b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index 22fa2f4..4285334 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -18,112 +18,7 @@
 package org.apache.phoenix.query;
 
 
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARG_POSITION;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BASE_COLUMN_COUNT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BUFFER_LENGTH;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CACHE_SIZE;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CHAR_OCTET_LENGTH;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_DEF;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_FAMILY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CURRENT_VALUE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CYCLE_FLAG;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TABLE_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TYPE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DECIMAL_DIGITS;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_VALUE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DISABLE_WAL;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ENCODING_SCHEME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FUNCTION_NAME;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POST_KEY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INCREMENT_BY;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_TYPE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ARRAY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_AUTOINCREMENT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_CONSTANT;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NULLABLE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ROW_TIMESTAMP;
-import static 
org.apache.phoe

[1/6] phoenix git commit: PHOENIX-4699 Stop scan after finding first child of table during drop

2018-04-26 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/master 8f1cef824 -> ff3273480


PHOENIX-4699 Stop scan after finding first child of table during drop


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

Branch: refs/heads/master
Commit: 041e7b5a05213710fc2af0644145d8247cb02191
Parents: 8f1cef8
Author: James Taylor 
Authored: Thu Apr 19 14:39:27 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:55:27 2018 -0700

--
 .../coprocessor/MetaDataEndpointImpl.java   | 44 +++-
 1 file changed, 25 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/041e7b5a/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 4c72c2d..c28ad3c 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
@@ -122,8 +122,10 @@ import 
org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.PageFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.ipc.RpcServer.Call;
@@ -275,6 +277,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso
 public static final String ROW_KEY_ORDER_OPTIMIZABLE = 
"ROW_KEY_ORDER_OPTIMIZABLE";
 public static final byte[] ROW_KEY_ORDER_OPTIMIZABLE_BYTES = 
Bytes.toBytes(ROW_KEY_ORDER_OPTIMIZABLE);
 
+private static final byte[] CHILD_TABLE_BYTES = new byte[] 
{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
+private static final byte[] PHYSICAL_TABLE_BYTES =
+new byte[] { PTable.LinkType.PHYSICAL_TABLE.getSerializedValue() };
+
 // KeyValues for Table
 private static final KeyValue TABLE_TYPE_KV = 
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, 
TABLE_TYPE_BYTES);
 private static final KeyValue TABLE_SEQ_NUM_KV = 
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, 
TABLE_SEQ_NUM_BYTES);
@@ -1527,7 +1533,6 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 for (PTable index : parentTable.getIndexes()) {
 
indexes.add(TableName.valueOf(index.getPhysicalName().getBytes()));
 }
-
 } else {
 // Mapped View
 cParentPhysicalName = 
SchemaUtil.getTableNameAsBytes(schemaName, tableName);
@@ -1810,13 +1815,10 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 private boolean execeededIndexQuota(PTableType tableType, PTable 
parentTable) {
 return PTableType.INDEX == tableType && 
parentTable.getIndexes().size() >= maxIndexesPerTable;
 }
-
-private static final byte[] CHILD_TABLE_BYTES = new byte[] 
{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
-
 
 private void findAllChildViews(Region region, byte[] tenantId, PTable 
table,
 TableViewFinder result, long clientTimeStamp, int clientVersion) 
throws IOException, SQLException {
-TableViewFinder currResult = findChildViews(region, tenantId, table, 
clientVersion);
+TableViewFinder currResult = findChildViews(region, tenantId, table, 
clientVersion, false);
 result.addResult(currResult);
 for (ViewInfo viewInfo : currResult.getViewInfoList()) {
 byte[] viewtenantId = viewInfo.getTenantId();
@@ -1829,9 +1831,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 }
 }
 
-// TODO remove this in 4.13 release 
-@Deprecated
-private TableViewFinder findChildViews_deprecated(Region region, byte[] 
tenantId, PTable table, byte[] linkTypeBytes) throws IOException {
+// TODO use child link instead once splittable system catalog 
(PHOENIX-3534) is implemented
+// and we have a separate table for links.
+pri

[5/6] phoenix git commit: PHOENIX-4686 Phoenix stats does not account for server side limit push downs (Abhishek Chouhan)

2018-04-26 Thread jamestaylor
PHOENIX-4686 Phoenix stats does not account for server side limit push downs 
(Abhishek Chouhan)


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

Branch: refs/heads/master
Commit: bebe66c4680a7cc3b09a703f8608f966ad4905f1
Parents: 9168f66
Author: James Taylor 
Authored: Thu Apr 26 09:14:52 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:55:28 2018 -0700

--
 .../end2end/ExplainPlanWithStatsEnabledIT.java  | 49 ++-
 .../org/apache/phoenix/execute/ScanPlan.java|  4 +-
 .../phoenix/iterate/BaseResultIterators.java| 50 +---
 .../phoenix/iterate/ParallelIterators.java  |  8 
 .../apache/phoenix/iterate/SerialIterators.java | 13 ++---
 .../phoenix/schema/stats/StatisticsUtil.java|  6 +++
 6 files changed, 111 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bebe66c4/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index 2099f4c..abaa2f6 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -40,6 +40,7 @@ import 
org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.schema.stats.StatisticsUtil;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EnvironmentEdge;
@@ -123,6 +124,50 @@ public class ExplainPlanWithStatsEnabledIT extends 
ParallelStatsEnabledIT {
 }
 
 @Test
+public void 
testBytesRowsForPointSelectWithLimitGreaterThanPointLookupSize() throws 
Exception {
+String sql = "SELECT * FROM " + tableA + " where k in (? ,?) limit 4";
+List binds = Lists.newArrayList();
+binds.add(103); binds.add(104);
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+Estimate info = getByteRowEstimates(conn, sql, binds);
+assertEquals((Long) 200L, info.estimatedBytes);
+assertEquals((Long) 2L, info.estimatedRows);
+assertEquals((Long) StatisticsUtil.NOT_STATS_BASED_TS, 
info.estimateInfoTs);
+}
+}
+
+@Test
+public void testBytesRowsForSelectWithLimit() throws Exception {
+String sql = "SELECT * FROM " + tableA + " where c1.a in (?,?) limit 
3";
+String noIndexSQL = "SELECT /*+ NO_INDEX */ * FROM " + tableA + " 
where c1.a in (?,?) limit 3";
+List binds = Lists.newArrayList();
+binds.add(1); binds.add(2);
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+Estimate info = getByteRowEstimates(conn, sql, binds);
+assertEquals((Long) 264L, info.estimatedBytes);
+assertEquals((Long) 3L, info.estimatedRows);
+assertEquals((Long) StatisticsUtil.NOT_STATS_BASED_TS, 
info.estimateInfoTs);
+
+info = getByteRowEstimates(conn, noIndexSQL, binds);
+assertEquals((Long) 634L, info.estimatedBytes);
+assertEquals((Long) 10L, info.estimatedRows);
+assertTrue(info.estimateInfoTs > 0);
+}
+}
+
+@Test
+public void testBytesRowsForSelectWithLimitIgnored() throws Exception {
+String sql = "SELECT * FROM " + tableA + " where (c1.a > c2.b) limit 
1";
+List binds = Lists.newArrayList();
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+Estimate info = getByteRowEstimates(conn, sql, binds);
+assertEquals((Long) 691L, info.estimatedBytes);
+assertEquals((Long) 10L, info.estimatedRows);
+assertTrue(info.estimateInfoTs > 0);
+}
+}
+
+@Test
 public void testBytesRowsForSelectWhenKeyInRange() throws Exception {
 String sql = "SELECT * FROM " + tableB + " where k >= ?";
 List binds = Lists.newArrayList();
@@ -278,7 +323,7 @@ public class ExplainPlanWithStatsEnabledIT extends 
ParallelStatsEnabledIT {
 Estimate info = getByteRowEstimates(conn, sql, binds);
 assertEquals((Long) 200L, info.estimatedBytes);
 assertEquals((Long) 2L, info.estimatedRows);
-assertTrue(info.

[2/6] phoenix git commit: PHOENIX-4708 Do not propagate GUIDE_POSTS_WIDTH to children

2018-04-26 Thread jamestaylor
PHOENIX-4708 Do not propagate GUIDE_POSTS_WIDTH to children


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

Branch: refs/heads/master
Commit: ff3273480165c2832cf1dc1bd114a74b93cf9e72
Parents: bebe66c
Author: James Taylor 
Authored: Thu Apr 26 09:40:35 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:55:28 2018 -0700

--
 .../phoenix/end2end/AlterTableWithViewsIT.java  |  22 ++
 .../coprocessor/MetaDataEndpointImpl.java   | 115 +---
 .../apache/phoenix/schema/TableProperty.java| 270 ++-
 3 files changed, 248 insertions(+), 159 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ff327348/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
index 237a8d2..e1b1372 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.sql.Connection;
+import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -39,12 +40,14 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.TephraTransactionalProcessor;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.util.StringUtil;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -166,6 +169,25 @@ public class AlterTableWithViewsIT extends 
ParallelStatsDisabledIT {
 assertTrue(viewTable2.isImmutableRows());
 // update cache frequency is not propagated to the view since it 
was altered on the view
 assertEquals(1, viewTable2.getUpdateCacheFrequency());
+
+long gpw = 100;
+conn.createStatement().execute("ALTER TABLE " + tableName + " SET 
GUIDE_POSTS_WIDTH=" + gpw);
+
+ResultSet rs;
+DatabaseMetaData md = conn.getMetaData();
+rs = md.getTables("", "", StringUtil.escapeLike(tableName), null);
+assertTrue(rs.next());
+assertEquals(gpw, 
rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH));
+
+rs = md.getTables(null, "", StringUtil.escapeLike(viewOfTable1), 
null);
+assertTrue(rs.next());
+rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH);
+assertTrue(rs.wasNull());
+
+rs = md.getTables(null, "", StringUtil.escapeLike(viewOfTable2), 
null);
+assertTrue(rs.next());
+rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH);
+assertTrue(rs.wasNull());
 } 
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ff327348/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 29eee7e..b77f113 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
@@ -91,6 +91,7 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
@@ -3163,7 +3164,56 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 byte[] tableNameBytes = index.getTableName().getBytes();
 return ByteUtil.concat(tenantIdBytes, SEPARATOR_BYTE_ARRAY, 
schemaNameBytes, SEPARATOR_BYTE_ARRAY, tableNameBytes);
 }
+  
 
+/**
+ * Determines whether or not we have a cha

[5/6] phoenix git commit: PHOENIX-4686 Phoenix stats does not account for server side limit push downs (Abhishek Chouhan)

2018-04-26 Thread jamestaylor
PHOENIX-4686 Phoenix stats does not account for server side limit push downs 
(Abhishek Chouhan)


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

Branch: refs/heads/4.x-HBase-1.2
Commit: 56636228fc96d5c1dbefbc7a9944998f04b39c4a
Parents: 1edb286
Author: James Taylor 
Authored: Thu Apr 26 09:14:52 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:57:27 2018 -0700

--
 .../end2end/ExplainPlanWithStatsEnabledIT.java  | 49 ++-
 .../org/apache/phoenix/execute/ScanPlan.java|  4 +-
 .../phoenix/iterate/BaseResultIterators.java| 50 +---
 .../phoenix/iterate/ParallelIterators.java  |  8 
 .../apache/phoenix/iterate/SerialIterators.java | 13 ++---
 .../phoenix/schema/stats/StatisticsUtil.java|  6 +++
 6 files changed, 111 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/56636228/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index 2099f4c..abaa2f6 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -40,6 +40,7 @@ import 
org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.schema.stats.StatisticsUtil;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EnvironmentEdge;
@@ -123,6 +124,50 @@ public class ExplainPlanWithStatsEnabledIT extends 
ParallelStatsEnabledIT {
 }
 
 @Test
+public void 
testBytesRowsForPointSelectWithLimitGreaterThanPointLookupSize() throws 
Exception {
+String sql = "SELECT * FROM " + tableA + " where k in (? ,?) limit 4";
+List binds = Lists.newArrayList();
+binds.add(103); binds.add(104);
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+Estimate info = getByteRowEstimates(conn, sql, binds);
+assertEquals((Long) 200L, info.estimatedBytes);
+assertEquals((Long) 2L, info.estimatedRows);
+assertEquals((Long) StatisticsUtil.NOT_STATS_BASED_TS, 
info.estimateInfoTs);
+}
+}
+
+@Test
+public void testBytesRowsForSelectWithLimit() throws Exception {
+String sql = "SELECT * FROM " + tableA + " where c1.a in (?,?) limit 
3";
+String noIndexSQL = "SELECT /*+ NO_INDEX */ * FROM " + tableA + " 
where c1.a in (?,?) limit 3";
+List binds = Lists.newArrayList();
+binds.add(1); binds.add(2);
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+Estimate info = getByteRowEstimates(conn, sql, binds);
+assertEquals((Long) 264L, info.estimatedBytes);
+assertEquals((Long) 3L, info.estimatedRows);
+assertEquals((Long) StatisticsUtil.NOT_STATS_BASED_TS, 
info.estimateInfoTs);
+
+info = getByteRowEstimates(conn, noIndexSQL, binds);
+assertEquals((Long) 634L, info.estimatedBytes);
+assertEquals((Long) 10L, info.estimatedRows);
+assertTrue(info.estimateInfoTs > 0);
+}
+}
+
+@Test
+public void testBytesRowsForSelectWithLimitIgnored() throws Exception {
+String sql = "SELECT * FROM " + tableA + " where (c1.a > c2.b) limit 
1";
+List binds = Lists.newArrayList();
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+Estimate info = getByteRowEstimates(conn, sql, binds);
+assertEquals((Long) 691L, info.estimatedBytes);
+assertEquals((Long) 10L, info.estimatedRows);
+assertTrue(info.estimateInfoTs > 0);
+}
+}
+
+@Test
 public void testBytesRowsForSelectWhenKeyInRange() throws Exception {
 String sql = "SELECT * FROM " + tableB + " where k >= ?";
 List binds = Lists.newArrayList();
@@ -278,7 +323,7 @@ public class ExplainPlanWithStatsEnabledIT extends 
ParallelStatsEnabledIT {
 Estimate info = getByteRowEstimates(conn, sql, binds);
 assertEquals((Long) 200L, info.estimatedBytes);
 assertEquals((Long) 2L, info.estimatedRows);
-assertTru

[1/6] phoenix git commit: PHOENIX-4708 Do not propagate GUIDE_POSTS_WIDTH to children

2018-04-26 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.2 8c0a76124 -> 3b9aaa98d


PHOENIX-4708 Do not propagate GUIDE_POSTS_WIDTH to children


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

Branch: refs/heads/4.x-HBase-1.2
Commit: 3b9aaa98d617262d5ba0c7ec2a5b2969ad62008c
Parents: 5663622
Author: James Taylor 
Authored: Thu Apr 26 09:40:35 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:57:27 2018 -0700

--
 .../phoenix/end2end/AlterTableWithViewsIT.java  |  22 ++
 .../coprocessor/MetaDataEndpointImpl.java   | 115 +---
 .../apache/phoenix/schema/TableProperty.java| 270 ++-
 3 files changed, 248 insertions(+), 159 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3b9aaa98/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
index 237a8d2..e1b1372 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.sql.Connection;
+import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -39,12 +40,14 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.TephraTransactionalProcessor;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.util.StringUtil;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -166,6 +169,25 @@ public class AlterTableWithViewsIT extends 
ParallelStatsDisabledIT {
 assertTrue(viewTable2.isImmutableRows());
 // update cache frequency is not propagated to the view since it 
was altered on the view
 assertEquals(1, viewTable2.getUpdateCacheFrequency());
+
+long gpw = 100;
+conn.createStatement().execute("ALTER TABLE " + tableName + " SET 
GUIDE_POSTS_WIDTH=" + gpw);
+
+ResultSet rs;
+DatabaseMetaData md = conn.getMetaData();
+rs = md.getTables("", "", StringUtil.escapeLike(tableName), null);
+assertTrue(rs.next());
+assertEquals(gpw, 
rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH));
+
+rs = md.getTables(null, "", StringUtil.escapeLike(viewOfTable1), 
null);
+assertTrue(rs.next());
+rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH);
+assertTrue(rs.wasNull());
+
+rs = md.getTables(null, "", StringUtil.escapeLike(viewOfTable2), 
null);
+assertTrue(rs.next());
+rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH);
+assertTrue(rs.wasNull());
 } 
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3b9aaa98/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 29eee7e..b77f113 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
@@ -91,6 +91,7 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
@@ -3163,7 +3164,56 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 byte[] tableNameBytes = index.getTableName().getBytes();
 return ByteUtil.concat(tenantIdBytes, SEPARATOR_BYTE_ARRAY, 
schemaNameBytes, SEPARATOR_BYTE_

[3/6] phoenix git commit: PHOENIX-4699 Stop scan after finding first child of table during drop

2018-04-26 Thread jamestaylor
PHOENIX-4699 Stop scan after finding first child of table during drop


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

Branch: refs/heads/4.x-HBase-1.2
Commit: ec9497f7f927ea0f24d3fc87ca6c99cdbffb2b39
Parents: 8c0a761
Author: James Taylor 
Authored: Thu Apr 19 14:39:27 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:57:27 2018 -0700

--
 .../coprocessor/MetaDataEndpointImpl.java   | 44 +++-
 1 file changed, 25 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ec9497f7/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 4c72c2d..c28ad3c 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
@@ -122,8 +122,10 @@ import 
org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.PageFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.ipc.RpcServer.Call;
@@ -275,6 +277,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso
 public static final String ROW_KEY_ORDER_OPTIMIZABLE = 
"ROW_KEY_ORDER_OPTIMIZABLE";
 public static final byte[] ROW_KEY_ORDER_OPTIMIZABLE_BYTES = 
Bytes.toBytes(ROW_KEY_ORDER_OPTIMIZABLE);
 
+private static final byte[] CHILD_TABLE_BYTES = new byte[] 
{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
+private static final byte[] PHYSICAL_TABLE_BYTES =
+new byte[] { PTable.LinkType.PHYSICAL_TABLE.getSerializedValue() };
+
 // KeyValues for Table
 private static final KeyValue TABLE_TYPE_KV = 
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, 
TABLE_TYPE_BYTES);
 private static final KeyValue TABLE_SEQ_NUM_KV = 
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, 
TABLE_SEQ_NUM_BYTES);
@@ -1527,7 +1533,6 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 for (PTable index : parentTable.getIndexes()) {
 
indexes.add(TableName.valueOf(index.getPhysicalName().getBytes()));
 }
-
 } else {
 // Mapped View
 cParentPhysicalName = 
SchemaUtil.getTableNameAsBytes(schemaName, tableName);
@@ -1810,13 +1815,10 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 private boolean execeededIndexQuota(PTableType tableType, PTable 
parentTable) {
 return PTableType.INDEX == tableType && 
parentTable.getIndexes().size() >= maxIndexesPerTable;
 }
-
-private static final byte[] CHILD_TABLE_BYTES = new byte[] 
{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
-
 
 private void findAllChildViews(Region region, byte[] tenantId, PTable 
table,
 TableViewFinder result, long clientTimeStamp, int clientVersion) 
throws IOException, SQLException {
-TableViewFinder currResult = findChildViews(region, tenantId, table, 
clientVersion);
+TableViewFinder currResult = findChildViews(region, tenantId, table, 
clientVersion, false);
 result.addResult(currResult);
 for (ViewInfo viewInfo : currResult.getViewInfoList()) {
 byte[] viewtenantId = viewInfo.getTenantId();
@@ -1829,9 +1831,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 }
 }
 
-// TODO remove this in 4.13 release 
-@Deprecated
-private TableViewFinder findChildViews_deprecated(Region region, byte[] 
tenantId, PTable table, byte[] linkTypeBytes) throws IOException {
+// TODO use child link instead once splittable system catalog 
(PHOENIX-3534) is implemented
+// and we have a separate table for links.
+private TableViewFinder findChildViews_deprecated(Region region, byte[] 
tenant

[4/6] phoenix git commit: PHOENIX-4698 Tolerate orphaned views (Maddineni Sukumar)

2018-04-26 Thread jamestaylor
PHOENIX-4698 Tolerate orphaned views (Maddineni Sukumar)


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

Branch: refs/heads/4.x-HBase-1.2
Commit: 9a1c2e782f97a394549f9be43e73486f64a12b81
Parents: ec9497f
Author: James Taylor 
Authored: Thu Apr 19 14:42:24 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:57:27 2018 -0700

--
 .../coprocessor/MetaDataEndpointImpl.java   | 50 +---
 1 file changed, 33 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/9a1c2e78/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 c28ad3c..ddd3ffe 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
@@ -592,7 +592,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso
 done.run(builder.build());
 return;
 } catch (Throwable t) {
-   logger.error("getTable failed", t);
+logger.error("getTable failed", t);
 ProtobufUtil.setControllerException(controller,
 
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
 }
@@ -755,9 +755,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso

// compatibility.
 Cell sortOrderKv = colKeyValues[SORT_ORDER_INDEX];
 SortOrder sortOrder =
-   sortOrderKv == null ? SortOrder.getDefault() : 
SortOrder.fromSystemValue(PInteger.INSTANCE
+sortOrderKv == null ? SortOrder.getDefault() : 
SortOrder.fromSystemValue(PInteger.INSTANCE
 .getCodec().decodeInt(sortOrderKv.getValueArray(),
-   sortOrderKv.getValueOffset(), 
SortOrder.getDefault()));
+sortOrderKv.getValueOffset(), 
SortOrder.getDefault()));
 
 Cell arraySizeKv = colKeyValues[ARRAY_SIZE_INDEX];
 Integer arraySize = arraySizeKv == null ? null :
@@ -1313,9 +1313,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 return table.getName() == null;
 }
 
-   private static boolean isSchemaDeleted(PSchema schema) {
-   return schema.getSchemaName() == null;
-   }
+private static boolean isSchemaDeleted(PSchema schema) {
+return schema.getSchemaName() == null;
+}
 
 private static boolean isFunctionDeleted(PFunction function) {
 return function.getFunctionName() == null;
@@ -1827,6 +1827,13 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 byte[] tableKey = SchemaUtil.getTableKey(viewtenantId, viewSchema, 
viewTable);
 ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
 PTable view = loadTable(env, tableKey, cacheKey, clientTimeStamp, 
clientTimeStamp, clientVersion);
+if (view == null) {
+logger.warn("Found orphan tenant view row in SYSTEM.CATALOG 
with tenantId:"
++ Bytes.toString(tenantId) + ", schema:"
++ Bytes.toString(viewSchema) + ", table:"
++ Bytes.toString(viewTable));
+continue;
+}
 findAllChildViews(region, viewtenantId, view, result, 
clientTimeStamp, clientVersion);
 }
 }
@@ -1969,8 +1976,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 if (systemCatalog.getTimeStamp() < MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0) {
 return findChildViews_deprecated(region, tenantId, table, 
PHYSICAL_TABLE_BYTES, stopAfterFirst);
 } else {
-return findChildViews_4_11(region, tenantId, 
table.getSchemaName().getBytes(),
-table.getTableName().getBytes(), stopAfterFirst);
+return findChildViews_4_11(region, tenantId, 
+table.getSchemaName() == null ? ByteUtil.EMPTY_BYTE_ARRAY 
: table.getSchemaName().getBytes(),
+table.getTableName().getBytes(), stopAfterFirst);
 }
 }
 
@@ -2548,6 +2556,14 @@ public class MetaData

[6/6] phoenix git commit: PHOENIX-4700 Fix split policy on system tables other than SYSTEM.CATALOG

2018-04-26 Thread jamestaylor
PHOENIX-4700 Fix split policy on system tables other than SYSTEM.CATALOG


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

Branch: refs/heads/4.x-HBase-1.2
Commit: 619a353cca71198d589156e9cb2e79e50e7a6713
Parents: 9a1c2e7
Author: James Taylor 
Authored: Mon Apr 23 10:14:36 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:57:27 2018 -0700

--
 .../apache/phoenix/query/QueryConstants.java| 117 +--
 .../SplitOnLeadingVarCharColumnsPolicy.java |  43 +++
 .../schema/SystemFunctionSplitPolicy.java   |  27 +
 .../phoenix/schema/SystemStatsSplitPolicy.java  |  27 +
 .../phoenix/schema/SystemSplitPolicyTest.java   |  97 +++
 5 files changed, 200 insertions(+), 111 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/619a353c/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java 
b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index 22fa2f4..4285334 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -18,112 +18,7 @@
 package org.apache.phoenix.query;
 
 
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARG_POSITION;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BASE_COLUMN_COUNT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BUFFER_LENGTH;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CACHE_SIZE;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CHAR_OCTET_LENGTH;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_DEF;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_FAMILY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CURRENT_VALUE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CYCLE_FLAG;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TABLE_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TYPE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DECIMAL_DIGITS;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_VALUE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DISABLE_WAL;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ENCODING_SCHEME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FUNCTION_NAME;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POST_KEY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INCREMENT_BY;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_TYPE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ARRAY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_AUTOINCREMENT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_CONSTANT;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NULLABLE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ROW_TIMESTAMP;
-import static 
org.apac

[2/6] phoenix git commit: PHOENIX-4694 Prevent locking of parent table when dropping view to reduce contention

2018-04-26 Thread jamestaylor
PHOENIX-4694 Prevent locking of parent table when dropping view to reduce 
contention


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

Branch: refs/heads/4.x-HBase-1.2
Commit: 1edb28660d6bb2e792e0424a5532fd3d010a2001
Parents: 619a353
Author: James Taylor 
Authored: Thu Apr 19 15:09:43 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 09:57:27 2018 -0700

--
 .../java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1edb2866/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 ddd3ffe..29eee7e 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
@@ -2007,8 +2007,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 }
 List tableNamesToDelete = Lists.newArrayList();
 List sharedTablesToDelete = Lists.newArrayList();
+// No need to lock parent table for views
 byte[] parentTableName = 
MetaDataUtil.getParentTableName(tableMetadata);
-byte[] lockTableName = parentTableName == null ? tableName : 
parentTableName;
+byte[] lockTableName = parentTableName == null || 
tableType.equals(PTableType.VIEW.getSerializedValue()) ? tableName : 
parentTableName;
 byte[] lockKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, 
lockTableName);
 byte[] key =
 parentTableName == null ? lockKey : 
SchemaUtil.getTableKey(tenantIdBytes,



[4/6] phoenix git commit: PHOENIX-4698 Tolerate orphaned views (Maddineni Sukumar)

2018-04-26 Thread jamestaylor
PHOENIX-4698 Tolerate orphaned views (Maddineni Sukumar)


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

Branch: refs/heads/4.x-cdh5.11
Commit: 67093243cbe95daef07e8b47fbe8f06f6c035d3d
Parents: 1725728
Author: James Taylor 
Authored: Thu Apr 19 14:42:24 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 10:00:16 2018 -0700

--
 .../coprocessor/MetaDataEndpointImpl.java   | 50 +---
 1 file changed, 33 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/67093243/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 c28ad3c..ddd3ffe 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
@@ -592,7 +592,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso
 done.run(builder.build());
 return;
 } catch (Throwable t) {
-   logger.error("getTable failed", t);
+logger.error("getTable failed", t);
 ProtobufUtil.setControllerException(controller,
 
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
 }
@@ -755,9 +755,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso

// compatibility.
 Cell sortOrderKv = colKeyValues[SORT_ORDER_INDEX];
 SortOrder sortOrder =
-   sortOrderKv == null ? SortOrder.getDefault() : 
SortOrder.fromSystemValue(PInteger.INSTANCE
+sortOrderKv == null ? SortOrder.getDefault() : 
SortOrder.fromSystemValue(PInteger.INSTANCE
 .getCodec().decodeInt(sortOrderKv.getValueArray(),
-   sortOrderKv.getValueOffset(), 
SortOrder.getDefault()));
+sortOrderKv.getValueOffset(), 
SortOrder.getDefault()));
 
 Cell arraySizeKv = colKeyValues[ARRAY_SIZE_INDEX];
 Integer arraySize = arraySizeKv == null ? null :
@@ -1313,9 +1313,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 return table.getName() == null;
 }
 
-   private static boolean isSchemaDeleted(PSchema schema) {
-   return schema.getSchemaName() == null;
-   }
+private static boolean isSchemaDeleted(PSchema schema) {
+return schema.getSchemaName() == null;
+}
 
 private static boolean isFunctionDeleted(PFunction function) {
 return function.getFunctionName() == null;
@@ -1827,6 +1827,13 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 byte[] tableKey = SchemaUtil.getTableKey(viewtenantId, viewSchema, 
viewTable);
 ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
 PTable view = loadTable(env, tableKey, cacheKey, clientTimeStamp, 
clientTimeStamp, clientVersion);
+if (view == null) {
+logger.warn("Found orphan tenant view row in SYSTEM.CATALOG 
with tenantId:"
++ Bytes.toString(tenantId) + ", schema:"
++ Bytes.toString(viewSchema) + ", table:"
++ Bytes.toString(viewTable));
+continue;
+}
 findAllChildViews(region, viewtenantId, view, result, 
clientTimeStamp, clientVersion);
 }
 }
@@ -1969,8 +1976,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 if (systemCatalog.getTimeStamp() < MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0) {
 return findChildViews_deprecated(region, tenantId, table, 
PHYSICAL_TABLE_BYTES, stopAfterFirst);
 } else {
-return findChildViews_4_11(region, tenantId, 
table.getSchemaName().getBytes(),
-table.getTableName().getBytes(), stopAfterFirst);
+return findChildViews_4_11(region, tenantId, 
+table.getSchemaName() == null ? ByteUtil.EMPTY_BYTE_ARRAY 
: table.getSchemaName().getBytes(),
+table.getTableName().getBytes(), stopAfterFirst);
 }
 }
 
@@ -2548,6 +2556,14 @@ public class MetaDataEn

[2/6] phoenix git commit: PHOENIX-4694 Prevent locking of parent table when dropping view to reduce contention

2018-04-26 Thread jamestaylor
PHOENIX-4694 Prevent locking of parent table when dropping view to reduce 
contention


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

Branch: refs/heads/4.x-cdh5.11
Commit: 526dc06ca147b3a2bcc15d7c4163d90641385231
Parents: 7381869
Author: James Taylor 
Authored: Thu Apr 19 15:09:43 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 10:00:16 2018 -0700

--
 .../java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/526dc06c/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 ddd3ffe..29eee7e 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
@@ -2007,8 +2007,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 }
 List tableNamesToDelete = Lists.newArrayList();
 List sharedTablesToDelete = Lists.newArrayList();
+// No need to lock parent table for views
 byte[] parentTableName = 
MetaDataUtil.getParentTableName(tableMetadata);
-byte[] lockTableName = parentTableName == null ? tableName : 
parentTableName;
+byte[] lockTableName = parentTableName == null || 
tableType.equals(PTableType.VIEW.getSerializedValue()) ? tableName : 
parentTableName;
 byte[] lockKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, 
lockTableName);
 byte[] key =
 parentTableName == null ? lockKey : 
SchemaUtil.getTableKey(tenantIdBytes,



[5/6] phoenix git commit: PHOENIX-4700 Fix split policy on system tables other than SYSTEM.CATALOG

2018-04-26 Thread jamestaylor
PHOENIX-4700 Fix split policy on system tables other than SYSTEM.CATALOG


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

Branch: refs/heads/4.x-cdh5.11
Commit: 738186946cf835591f78920c865a1d404cab9b96
Parents: 6709324
Author: James Taylor 
Authored: Mon Apr 23 10:14:36 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 10:00:16 2018 -0700

--
 .../apache/phoenix/query/QueryConstants.java| 117 +--
 .../SplitOnLeadingVarCharColumnsPolicy.java |  43 +++
 .../schema/SystemFunctionSplitPolicy.java   |  27 +
 .../phoenix/schema/SystemStatsSplitPolicy.java  |  27 +
 .../phoenix/schema/SystemSplitPolicyTest.java   |  97 +++
 5 files changed, 200 insertions(+), 111 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/73818694/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java 
b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index 22fa2f4..4285334 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -18,112 +18,7 @@
 package org.apache.phoenix.query;
 
 
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARG_POSITION;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BASE_COLUMN_COUNT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BUFFER_LENGTH;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CACHE_SIZE;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CHAR_OCTET_LENGTH;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_DEF;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_FAMILY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CURRENT_VALUE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CYCLE_FLAG;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TABLE_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TYPE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DECIMAL_DIGITS;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_VALUE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DISABLE_WAL;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ENCODING_SCHEME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FUNCTION_NAME;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POST_KEY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INCREMENT_BY;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_TYPE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ARRAY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_AUTOINCREMENT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_CONSTANT;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NULLABLE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ROW_TIMESTAMP;
-import static 
org.apache

[1/6] phoenix git commit: PHOENIX-4686 Phoenix stats does not account for server side limit push downs (Abhishek Chouhan)

2018-04-26 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-cdh5.11 8bba23976 -> 33ed01d54


PHOENIX-4686 Phoenix stats does not account for server side limit push downs 
(Abhishek Chouhan)


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

Branch: refs/heads/4.x-cdh5.11
Commit: d776ce9a65157fd485df8ec7ce42d506d8e3743a
Parents: 526dc06
Author: James Taylor 
Authored: Thu Apr 26 09:14:52 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 10:00:16 2018 -0700

--
 .../end2end/ExplainPlanWithStatsEnabledIT.java  | 49 ++-
 .../org/apache/phoenix/execute/ScanPlan.java|  4 +-
 .../phoenix/iterate/BaseResultIterators.java| 50 +---
 .../phoenix/iterate/ParallelIterators.java  |  8 
 .../apache/phoenix/iterate/SerialIterators.java | 13 ++---
 .../phoenix/schema/stats/StatisticsUtil.java|  6 +++
 6 files changed, 111 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d776ce9a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index 2099f4c..abaa2f6 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -40,6 +40,7 @@ import 
org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.schema.stats.StatisticsUtil;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EnvironmentEdge;
@@ -123,6 +124,50 @@ public class ExplainPlanWithStatsEnabledIT extends 
ParallelStatsEnabledIT {
 }
 
 @Test
+public void 
testBytesRowsForPointSelectWithLimitGreaterThanPointLookupSize() throws 
Exception {
+String sql = "SELECT * FROM " + tableA + " where k in (? ,?) limit 4";
+List binds = Lists.newArrayList();
+binds.add(103); binds.add(104);
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+Estimate info = getByteRowEstimates(conn, sql, binds);
+assertEquals((Long) 200L, info.estimatedBytes);
+assertEquals((Long) 2L, info.estimatedRows);
+assertEquals((Long) StatisticsUtil.NOT_STATS_BASED_TS, 
info.estimateInfoTs);
+}
+}
+
+@Test
+public void testBytesRowsForSelectWithLimit() throws Exception {
+String sql = "SELECT * FROM " + tableA + " where c1.a in (?,?) limit 
3";
+String noIndexSQL = "SELECT /*+ NO_INDEX */ * FROM " + tableA + " 
where c1.a in (?,?) limit 3";
+List binds = Lists.newArrayList();
+binds.add(1); binds.add(2);
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+Estimate info = getByteRowEstimates(conn, sql, binds);
+assertEquals((Long) 264L, info.estimatedBytes);
+assertEquals((Long) 3L, info.estimatedRows);
+assertEquals((Long) StatisticsUtil.NOT_STATS_BASED_TS, 
info.estimateInfoTs);
+
+info = getByteRowEstimates(conn, noIndexSQL, binds);
+assertEquals((Long) 634L, info.estimatedBytes);
+assertEquals((Long) 10L, info.estimatedRows);
+assertTrue(info.estimateInfoTs > 0);
+}
+}
+
+@Test
+public void testBytesRowsForSelectWithLimitIgnored() throws Exception {
+String sql = "SELECT * FROM " + tableA + " where (c1.a > c2.b) limit 
1";
+List binds = Lists.newArrayList();
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+Estimate info = getByteRowEstimates(conn, sql, binds);
+assertEquals((Long) 691L, info.estimatedBytes);
+assertEquals((Long) 10L, info.estimatedRows);
+assertTrue(info.estimateInfoTs > 0);
+}
+}
+
+@Test
 public void testBytesRowsForSelectWhenKeyInRange() throws Exception {
 String sql = "SELECT * FROM " + tableB + " where k >= ?";
 List binds = Lists.newArrayList();
@@ -278,7 +323,7 @@ public class ExplainPlanWithStatsEnabledIT extends 
ParallelStatsEnabledIT {
 Estimate info = getByteRowEstimates(conn, sql, binds);
 assertEquals((Long) 200L, info.estimatedBy

[6/6] phoenix git commit: PHOENIX-4708 Do not propagate GUIDE_POSTS_WIDTH to children

2018-04-26 Thread jamestaylor
PHOENIX-4708 Do not propagate GUIDE_POSTS_WIDTH to children


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

Branch: refs/heads/4.x-cdh5.11
Commit: 33ed01d54da92de4c7eadaa65bd6beebcc4d1b01
Parents: d776ce9
Author: James Taylor 
Authored: Thu Apr 26 09:40:35 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 10:00:17 2018 -0700

--
 .../phoenix/end2end/AlterTableWithViewsIT.java  |  22 ++
 .../coprocessor/MetaDataEndpointImpl.java   | 115 +---
 .../apache/phoenix/schema/TableProperty.java| 270 ++-
 3 files changed, 248 insertions(+), 159 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ed01d5/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
index 237a8d2..e1b1372 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.sql.Connection;
+import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -39,12 +40,14 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.TephraTransactionalProcessor;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.util.StringUtil;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -166,6 +169,25 @@ public class AlterTableWithViewsIT extends 
ParallelStatsDisabledIT {
 assertTrue(viewTable2.isImmutableRows());
 // update cache frequency is not propagated to the view since it 
was altered on the view
 assertEquals(1, viewTable2.getUpdateCacheFrequency());
+
+long gpw = 100;
+conn.createStatement().execute("ALTER TABLE " + tableName + " SET 
GUIDE_POSTS_WIDTH=" + gpw);
+
+ResultSet rs;
+DatabaseMetaData md = conn.getMetaData();
+rs = md.getTables("", "", StringUtil.escapeLike(tableName), null);
+assertTrue(rs.next());
+assertEquals(gpw, 
rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH));
+
+rs = md.getTables(null, "", StringUtil.escapeLike(viewOfTable1), 
null);
+assertTrue(rs.next());
+rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH);
+assertTrue(rs.wasNull());
+
+rs = md.getTables(null, "", StringUtil.escapeLike(viewOfTable2), 
null);
+assertTrue(rs.next());
+rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH);
+assertTrue(rs.wasNull());
 } 
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ed01d5/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 29eee7e..b77f113 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
@@ -91,6 +91,7 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
@@ -3163,7 +3164,56 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 byte[] tableNameBytes = index.getTableName().getBytes();
 return ByteUtil.concat(tenantIdBytes, SEPARATOR_BYTE_ARRAY, 
schemaNameBytes, SEPARATOR_BYTE_ARRAY, tableNameBytes);
 }
+  
 
+/**
+ * Determines whether or not we have 

[3/6] phoenix git commit: PHOENIX-4699 Stop scan after finding first child of table during drop

2018-04-26 Thread jamestaylor
PHOENIX-4699 Stop scan after finding first child of table during drop


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

Branch: refs/heads/4.x-cdh5.11
Commit: 172572803cfc3fa145ad04b1257bd3d62db3d022
Parents: 8bba239
Author: James Taylor 
Authored: Thu Apr 19 14:39:27 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 10:00:16 2018 -0700

--
 .../coprocessor/MetaDataEndpointImpl.java   | 44 +++-
 1 file changed, 25 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/17257280/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 4c72c2d..c28ad3c 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
@@ -122,8 +122,10 @@ import 
org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.PageFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.ipc.RpcServer.Call;
@@ -275,6 +277,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso
 public static final String ROW_KEY_ORDER_OPTIMIZABLE = 
"ROW_KEY_ORDER_OPTIMIZABLE";
 public static final byte[] ROW_KEY_ORDER_OPTIMIZABLE_BYTES = 
Bytes.toBytes(ROW_KEY_ORDER_OPTIMIZABLE);
 
+private static final byte[] CHILD_TABLE_BYTES = new byte[] 
{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
+private static final byte[] PHYSICAL_TABLE_BYTES =
+new byte[] { PTable.LinkType.PHYSICAL_TABLE.getSerializedValue() };
+
 // KeyValues for Table
 private static final KeyValue TABLE_TYPE_KV = 
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, 
TABLE_TYPE_BYTES);
 private static final KeyValue TABLE_SEQ_NUM_KV = 
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, 
TABLE_SEQ_NUM_BYTES);
@@ -1527,7 +1533,6 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 for (PTable index : parentTable.getIndexes()) {
 
indexes.add(TableName.valueOf(index.getPhysicalName().getBytes()));
 }
-
 } else {
 // Mapped View
 cParentPhysicalName = 
SchemaUtil.getTableNameAsBytes(schemaName, tableName);
@@ -1810,13 +1815,10 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 private boolean execeededIndexQuota(PTableType tableType, PTable 
parentTable) {
 return PTableType.INDEX == tableType && 
parentTable.getIndexes().size() >= maxIndexesPerTable;
 }
-
-private static final byte[] CHILD_TABLE_BYTES = new byte[] 
{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
-
 
 private void findAllChildViews(Region region, byte[] tenantId, PTable 
table,
 TableViewFinder result, long clientTimeStamp, int clientVersion) 
throws IOException, SQLException {
-TableViewFinder currResult = findChildViews(region, tenantId, table, 
clientVersion);
+TableViewFinder currResult = findChildViews(region, tenantId, table, 
clientVersion, false);
 result.addResult(currResult);
 for (ViewInfo viewInfo : currResult.getViewInfoList()) {
 byte[] viewtenantId = viewInfo.getTenantId();
@@ -1829,9 +1831,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 }
 }
 
-// TODO remove this in 4.13 release 
-@Deprecated
-private TableViewFinder findChildViews_deprecated(Region region, byte[] 
tenantId, PTable table, byte[] linkTypeBytes) throws IOException {
+// TODO use child link instead once splittable system catalog 
(PHOENIX-3534) is implemented
+// and we have a separate table for links.
+private TableViewFinder findChildViews_deprecated(Region region, byte[] 
tenantId

[3/6] phoenix git commit: PHOENIX-4694 Prevent locking of parent table when dropping view to reduce contention

2018-04-26 Thread jamestaylor
PHOENIX-4694 Prevent locking of parent table when dropping view to reduce 
contention


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

Branch: refs/heads/4.x-HBase-1.1
Commit: c6d4037802fc2083b2d8991dad38d7130f5ca8de
Parents: 36e1176
Author: James Taylor 
Authored: Thu Apr 19 15:09:43 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 10:14:02 2018 -0700

--
 .../java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c6d40378/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 ddd0be1..ed4e04b 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
@@ -1891,8 +1891,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 }
 List tableNamesToDelete = Lists.newArrayList();
 List sharedTablesToDelete = Lists.newArrayList();
+// No need to lock parent table for views
 byte[] parentTableName = 
MetaDataUtil.getParentTableName(tableMetadata);
-byte[] lockTableName = parentTableName == null ? tableName : 
parentTableName;
+byte[] lockTableName = parentTableName == null || 
tableType.equals(PTableType.VIEW.getSerializedValue()) ? tableName : 
parentTableName;
 byte[] lockKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, 
lockTableName);
 byte[] key =
 parentTableName == null ? lockKey : 
SchemaUtil.getTableKey(tenantIdBytes,



[1/6] phoenix git commit: PHOENIX-4699 Stop scan after finding first child of table during drop

2018-04-26 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 16a353b35 -> d9869fdd7


PHOENIX-4699 Stop scan after finding first child of table during drop


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

Branch: refs/heads/4.x-HBase-1.1
Commit: f8322721494e80586ff66ddbf8deef498a89fa02
Parents: 16a353b
Author: James Taylor 
Authored: Thu Apr 19 14:39:27 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 10:13:58 2018 -0700

--
 .../coprocessor/MetaDataEndpointImpl.java   | 43 
 1 file changed, 25 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f8322721/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 d7abe08..41bcf5a 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
@@ -118,8 +118,10 @@ import 
org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.PageFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.Region;
@@ -269,6 +271,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso
 public static final String ROW_KEY_ORDER_OPTIMIZABLE = 
"ROW_KEY_ORDER_OPTIMIZABLE";
 public static final byte[] ROW_KEY_ORDER_OPTIMIZABLE_BYTES = 
Bytes.toBytes(ROW_KEY_ORDER_OPTIMIZABLE);
 
+private static final byte[] CHILD_TABLE_BYTES = new byte[] 
{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
+private static final byte[] PHYSICAL_TABLE_BYTES =
+new byte[] { PTable.LinkType.PHYSICAL_TABLE.getSerializedValue() };
+
 // KeyValues for Table
 private static final KeyValue TABLE_TYPE_KV = 
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, 
TABLE_TYPE_BYTES);
 private static final KeyValue TABLE_SEQ_NUM_KV = 
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, 
TABLE_SEQ_NUM_BYTES);
@@ -1693,13 +1699,10 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 locks.add(rowLock);
 return rowLock;
 }
-
-private static final byte[] CHILD_TABLE_BYTES = new byte[] 
{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
-
 
 private void findAllChildViews(Region region, byte[] tenantId, PTable 
table,
 TableViewFinder result, long clientTimeStamp, int clientVersion) 
throws IOException, SQLException {
-TableViewFinder currResult = findChildViews(region, tenantId, table, 
clientVersion);
+TableViewFinder currResult = findChildViews(region, tenantId, table, 
clientVersion, false);
 result.addResult(currResult);
 for (ViewInfo viewInfo : currResult.getViewInfoList()) {
 byte[] viewtenantId = viewInfo.getTenantId();
@@ -1712,9 +1715,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 }
 }
 
-// TODO remove this in 4.13 release 
-@Deprecated
-private TableViewFinder findChildViews_deprecated(Region region, byte[] 
tenantId, PTable table, byte[] linkTypeBytes) throws IOException {
+// TODO use child link instead once splittable system catalog 
(PHOENIX-3534) is implemented
+// and we have a separate table for links.
+private TableViewFinder findChildViews_deprecated(Region region, byte[] 
tenantId, PTable table, byte[] linkTypeBytes, boolean stopAfterFirst) throws 
IOException {
 byte[] schemaName = table.getSchemaName().getBytes();
 byte[] tableName = table.getTableName().getBytes();
 boolean isMultiTenant = table.isMultiTenant();
@@ -1737,7 +1740,11 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 .getPhysicalHBaseTableName(schemaName, tableName, 
table.isNamespaceMapped())
 .getBytes());
  

[4/6] phoenix git commit: PHOENIX-4700 Fix split policy on system tables other than SYSTEM.CATALOG

2018-04-26 Thread jamestaylor
PHOENIX-4700 Fix split policy on system tables other than SYSTEM.CATALOG


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 36e11768c9c0c96fcd397b9b6e08f6f1830aeebe
Parents: 8c8f632
Author: James Taylor 
Authored: Mon Apr 23 10:14:36 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 10:14:02 2018 -0700

--
 .../apache/phoenix/query/QueryConstants.java| 117 +--
 .../SplitOnLeadingVarCharColumnsPolicy.java |  43 +++
 .../schema/SystemFunctionSplitPolicy.java   |  27 +
 .../phoenix/schema/SystemStatsSplitPolicy.java  |  27 +
 .../phoenix/schema/SystemSplitPolicyTest.java   |  97 +++
 5 files changed, 200 insertions(+), 111 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/36e11768/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
--
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java 
b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index 22fa2f4..4285334 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -18,112 +18,7 @@
 package org.apache.phoenix.query;
 
 
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARG_POSITION;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BASE_COLUMN_COUNT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BUFFER_LENGTH;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CACHE_SIZE;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CHAR_OCTET_LENGTH;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_DEF;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_FAMILY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CURRENT_VALUE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CYCLE_FLAG;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TABLE_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TYPE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DECIMAL_DIGITS;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_VALUE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DISABLE_WAL;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ENCODING_SCHEME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FUNCTION_NAME;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POST_KEY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INCREMENT_BY;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_TYPE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ARRAY;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_AUTOINCREMENT;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_CONSTANT;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NULLABLE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_ROW_TIMESTAMP;
-import static 
org.apac

[6/6] phoenix git commit: PHOENIX-4708 Do not propagate GUIDE_POSTS_WIDTH to children

2018-04-26 Thread jamestaylor
PHOENIX-4708 Do not propagate GUIDE_POSTS_WIDTH to children


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

Branch: refs/heads/4.x-HBase-1.1
Commit: d9869fdd790c5d595b4507c2e5bdd2408eba8e9b
Parents: 9416335
Author: James Taylor 
Authored: Thu Apr 26 09:40:35 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 10:14:48 2018 -0700

--
 .../phoenix/end2end/AlterTableWithViewsIT.java  |  22 ++
 .../coprocessor/MetaDataEndpointImpl.java   | 115 +---
 .../apache/phoenix/schema/TableProperty.java| 270 ++-
 3 files changed, 248 insertions(+), 159 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d9869fdd/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
index 237a8d2..e1b1372 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
@@ -25,6 +25,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.sql.Connection;
+import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
@@ -39,12 +40,14 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.TephraTransactionalProcessor;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.util.StringUtil;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -166,6 +169,25 @@ public class AlterTableWithViewsIT extends 
ParallelStatsDisabledIT {
 assertTrue(viewTable2.isImmutableRows());
 // update cache frequency is not propagated to the view since it 
was altered on the view
 assertEquals(1, viewTable2.getUpdateCacheFrequency());
+
+long gpw = 100;
+conn.createStatement().execute("ALTER TABLE " + tableName + " SET 
GUIDE_POSTS_WIDTH=" + gpw);
+
+ResultSet rs;
+DatabaseMetaData md = conn.getMetaData();
+rs = md.getTables("", "", StringUtil.escapeLike(tableName), null);
+assertTrue(rs.next());
+assertEquals(gpw, 
rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH));
+
+rs = md.getTables(null, "", StringUtil.escapeLike(viewOfTable1), 
null);
+assertTrue(rs.next());
+rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH);
+assertTrue(rs.wasNull());
+
+rs = md.getTables(null, "", StringUtil.escapeLike(viewOfTable2), 
null);
+assertTrue(rs.next());
+rs.getLong(PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH);
+assertTrue(rs.wasNull());
 } 
 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d9869fdd/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 ed4e04b..937c9f5 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
@@ -90,6 +90,7 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Iterator;
@@ -3017,7 +3018,56 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 byte[] tableNameBytes = index.getTableName().getBytes();
 return ByteUtil.concat(tenantIdBytes, SEPARATOR_BYTE_ARRAY, 
schemaNameBytes, SEPARATOR_BYTE_ARRAY, tableNameBytes);
 }
+  
 
+/**
+ * Determines whether or not we ha

[5/6] phoenix git commit: PHOENIX-4686 Phoenix stats does not account for server side limit push downs (Abhishek Chouhan)

2018-04-26 Thread jamestaylor
PHOENIX-4686 Phoenix stats does not account for server side limit push downs 
(Abhishek Chouhan)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 94163357039316d6f7859dd37d1d9e8aad1c61a5
Parents: c6d4037
Author: James Taylor 
Authored: Thu Apr 26 09:14:52 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 10:14:46 2018 -0700

--
 .../end2end/ExplainPlanWithStatsEnabledIT.java  | 53 ++--
 .../org/apache/phoenix/execute/ScanPlan.java|  4 +-
 .../phoenix/iterate/BaseResultIterators.java| 50 +++---
 .../phoenix/iterate/ParallelIterators.java  |  8 +++
 .../apache/phoenix/iterate/SerialIterators.java | 13 ++---
 .../phoenix/schema/stats/StatisticsUtil.java|  6 +++
 6 files changed, 113 insertions(+), 21 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/94163357/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
--
diff --git 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index 39bdb67..abaa2f6 100644
--- 
a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ 
b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -40,6 +40,7 @@ import 
org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.schema.stats.StatisticsUtil;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EnvironmentEdge;
@@ -123,6 +124,50 @@ public class ExplainPlanWithStatsEnabledIT extends 
ParallelStatsEnabledIT {
 }
 
 @Test
+public void 
testBytesRowsForPointSelectWithLimitGreaterThanPointLookupSize() throws 
Exception {
+String sql = "SELECT * FROM " + tableA + " where k in (? ,?) limit 4";
+List binds = Lists.newArrayList();
+binds.add(103); binds.add(104);
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+Estimate info = getByteRowEstimates(conn, sql, binds);
+assertEquals((Long) 200L, info.estimatedBytes);
+assertEquals((Long) 2L, info.estimatedRows);
+assertEquals((Long) StatisticsUtil.NOT_STATS_BASED_TS, 
info.estimateInfoTs);
+}
+}
+
+@Test
+public void testBytesRowsForSelectWithLimit() throws Exception {
+String sql = "SELECT * FROM " + tableA + " where c1.a in (?,?) limit 
3";
+String noIndexSQL = "SELECT /*+ NO_INDEX */ * FROM " + tableA + " 
where c1.a in (?,?) limit 3";
+List binds = Lists.newArrayList();
+binds.add(1); binds.add(2);
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+Estimate info = getByteRowEstimates(conn, sql, binds);
+assertEquals((Long) 264L, info.estimatedBytes);
+assertEquals((Long) 3L, info.estimatedRows);
+assertEquals((Long) StatisticsUtil.NOT_STATS_BASED_TS, 
info.estimateInfoTs);
+
+info = getByteRowEstimates(conn, noIndexSQL, binds);
+assertEquals((Long) 634L, info.estimatedBytes);
+assertEquals((Long) 10L, info.estimatedRows);
+assertTrue(info.estimateInfoTs > 0);
+}
+}
+
+@Test
+public void testBytesRowsForSelectWithLimitIgnored() throws Exception {
+String sql = "SELECT * FROM " + tableA + " where (c1.a > c2.b) limit 
1";
+List binds = Lists.newArrayList();
+try (Connection conn = DriverManager.getConnection(getUrl())) {
+Estimate info = getByteRowEstimates(conn, sql, binds);
+assertEquals((Long) 691L, info.estimatedBytes);
+assertEquals((Long) 10L, info.estimatedRows);
+assertTrue(info.estimateInfoTs > 0);
+}
+}
+
+@Test
 public void testBytesRowsForSelectWhenKeyInRange() throws Exception {
 String sql = "SELECT * FROM " + tableB + " where k >= ?";
 List binds = Lists.newArrayList();
@@ -278,7 +323,7 @@ public class ExplainPlanWithStatsEnabledIT extends 
ParallelStatsEnabledIT {
 Estimate info = getByteRowEstimates(conn, sql, binds);
 assertEquals((Long) 200L, info.estimatedBytes);
 assertEquals((Long) 2L, info.estimatedRows);
-assertTrue(

[2/6] phoenix git commit: PHOENIX-4698 Tolerate orphaned views (Maddineni Sukumar)

2018-04-26 Thread jamestaylor
PHOENIX-4698 Tolerate orphaned views (Maddineni Sukumar)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: 8c8f6324e227face36302a45bdc62e77e822f6c0
Parents: f832272
Author: James Taylor 
Authored: Thu Apr 19 14:42:24 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 10:14:01 2018 -0700

--
 .../coprocessor/MetaDataEndpointImpl.java   | 50 +---
 1 file changed, 33 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8c8f6324/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 41bcf5a..ddd0be1 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
@@ -567,7 +567,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso
 done.run(builder.build());
 return;
 } catch (Throwable t) {
-   logger.error("getTable failed", t);
+logger.error("getTable failed", t);
 ProtobufUtil.setControllerException(controller,
 
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
 }
@@ -728,9 +728,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso

// compatibility.
 Cell sortOrderKv = colKeyValues[SORT_ORDER_INDEX];
 SortOrder sortOrder =
-   sortOrderKv == null ? SortOrder.getDefault() : 
SortOrder.fromSystemValue(PInteger.INSTANCE
+sortOrderKv == null ? SortOrder.getDefault() : 
SortOrder.fromSystemValue(PInteger.INSTANCE
 .getCodec().decodeInt(sortOrderKv.getValueArray(),
-   sortOrderKv.getValueOffset(), 
SortOrder.getDefault()));
+sortOrderKv.getValueOffset(), 
SortOrder.getDefault()));
 
 Cell arraySizeKv = colKeyValues[ARRAY_SIZE_INDEX];
 Integer arraySize = arraySizeKv == null ? null :
@@ -1286,9 +1286,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 return table.getName() == null;
 }
 
-   private static boolean isSchemaDeleted(PSchema schema) {
-   return schema.getSchemaName() == null;
-   }
+private static boolean isSchemaDeleted(PSchema schema) {
+return schema.getSchemaName() == null;
+}
 
 private static boolean isFunctionDeleted(PFunction function) {
 return function.getFunctionName() == null;
@@ -1711,6 +1711,13 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 byte[] tableKey = SchemaUtil.getTableKey(viewtenantId, viewSchema, 
viewTable);
 ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
 PTable view = loadTable(env, tableKey, cacheKey, clientTimeStamp, 
clientTimeStamp, clientVersion);
+if (view == null) {
+logger.warn("Found orphan tenant view row in SYSTEM.CATALOG 
with tenantId:"
++ Bytes.toString(tenantId) + ", schema:"
++ Bytes.toString(viewSchema) + ", table:"
++ Bytes.toString(viewTable));
+continue;
+}
 findAllChildViews(region, viewtenantId, view, result, 
clientTimeStamp, clientVersion);
 }
 }
@@ -1853,8 +1860,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 if (systemCatalog.getTimeStamp() < MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0) {
 return findChildViews_deprecated(region, tenantId, table, 
PHYSICAL_TABLE_BYTES, stopAfterFirst);
 } else {
-return findChildViews_4_11(region, tenantId, 
table.getSchemaName().getBytes(),
-table.getTableName().getBytes(), stopAfterFirst);
+return findChildViews_4_11(region, tenantId, 
+table.getSchemaName() == null ? ByteUtil.EMPTY_BYTE_ARRAY 
: table.getSchemaName().getBytes(),
+table.getTableName().getBytes(), stopAfterFirst);
 }
 }
 
@@ -2402,6 +2410,14 @@ public class MetaData

[4/6] phoenix git commit: PHOENIX-4698 Tolerate orphaned views (Maddineni Sukumar)

2018-04-26 Thread jamestaylor
PHOENIX-4698 Tolerate orphaned views (Maddineni Sukumar)


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

Branch: refs/heads/4.x-HBase-0.98
Commit: 532447a7e45a622fe2f78f7e53356448aad5fe79
Parents: cca6b44
Author: James Taylor 
Authored: Thu Apr 19 14:42:24 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 10:35:06 2018 -0700

--
 .../coprocessor/MetaDataEndpointImpl.java   | 48 +---
 1 file changed, 32 insertions(+), 16 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/532447a7/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 a965680..277a8e1 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
@@ -578,7 +578,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso
 done.run(builder.build());
 return;
 } catch (Throwable t) {
-   logger.error("getTable failed", t);
+logger.error("getTable failed", t);
 ProtobufUtil.setControllerException(controller,
 
ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), 
t));
 }
@@ -737,9 +737,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso

// compatibility.
 Cell sortOrderKv = colKeyValues[SORT_ORDER_INDEX];
 SortOrder sortOrder =
-   sortOrderKv == null ? SortOrder.getDefault() : 
SortOrder.fromSystemValue(PInteger.INSTANCE
+sortOrderKv == null ? SortOrder.getDefault() : 
SortOrder.fromSystemValue(PInteger.INSTANCE
 .getCodec().decodeInt(sortOrderKv.getValueArray(),
-   sortOrderKv.getValueOffset(), 
SortOrder.getDefault()));
+sortOrderKv.getValueOffset(), 
SortOrder.getDefault()));
 
 Cell arraySizeKv = colKeyValues[ARRAY_SIZE_INDEX];
 Integer arraySize = arraySizeKv == null ? null :
@@ -1296,9 +1296,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 return table.getName() == null;
 }
 
-   private static boolean isSchemaDeleted(PSchema schema) {
-   return schema.getSchemaName() == null;
-   }
+private static boolean isSchemaDeleted(PSchema schema) {
+return schema.getSchemaName() == null;
+}
 
 private static boolean isFunctionDeleted(PFunction function) {
 return function.getFunctionName() == null;
@@ -1717,6 +1717,13 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 byte[] tableKey = SchemaUtil.getTableKey(viewtenantId, viewSchema, 
viewTable);
 ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
 PTable view = loadTable(env, tableKey, cacheKey, clientTimeStamp, 
clientTimeStamp, clientVersion);
+if (view == null) {
+logger.warn("Found orphan tenant view row in SYSTEM.CATALOG 
with tenantId:"
++ Bytes.toString(tenantId) + ", schema:"
++ Bytes.toString(viewSchema) + ", table:"
++ Bytes.toString(viewTable));
+continue;
+}
 findAllChildViews(region, viewtenantId, view, result, 
clientTimeStamp, clientVersion);
 }
 }
@@ -1859,8 +1866,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 if (systemCatalog.getTimeStamp() < MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0) {
 return findChildViews_deprecated(region, tenantId, table, 
PHYSICAL_TABLE_BYTES, stopAfterFirst);
 } else {
-return findChildViews_4_11(region, tenantId, 
table.getSchemaName().getBytes(),
-table.getTableName().getBytes(), stopAfterFirst);
+return findChildViews_4_11(region, tenantId, 
+table.getSchemaName() == null ? ByteUtil.EMPTY_BYTE_ARRAY 
: table.getSchemaName().getBytes(),
+table.getTableName().getBytes(), stopAfterFirst);
 }
 }
 
@@ -2407,6 +2415,14 @@ public class MetaDat

[1/6] phoenix git commit: PHOENIX-4699 Stop scan after finding first child of table during drop

2018-04-26 Thread jamestaylor
Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-0.98 4c31ac3f5 -> 761a60e23


PHOENIX-4699 Stop scan after finding first child of table during drop


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

Branch: refs/heads/4.x-HBase-0.98
Commit: cca6b44a8885ceaa3b4600dd3024efcff726fd32
Parents: 4c31ac3
Author: James Taylor 
Authored: Thu Apr 19 14:39:27 2018 -0700
Committer: James Taylor 
Committed: Thu Apr 26 10:35:02 2018 -0700

--
 .../coprocessor/MetaDataEndpointImpl.java   | 42 
 1 file changed, 25 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/phoenix/blob/cca6b44a/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 62c608a..a965680 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
@@ -118,8 +118,10 @@ import 
org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
+import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
+import org.apache.hadoop.hbase.filter.PageFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.HRegion;
@@ -268,6 +270,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol 
implements Coprocesso
 public static final String ROW_KEY_ORDER_OPTIMIZABLE = 
"ROW_KEY_ORDER_OPTIMIZABLE";
 public static final byte[] ROW_KEY_ORDER_OPTIMIZABLE_BYTES = 
Bytes.toBytes(ROW_KEY_ORDER_OPTIMIZABLE);
 
+private static final byte[] CHILD_TABLE_BYTES = new byte[] 
{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
+private static final byte[] PHYSICAL_TABLE_BYTES =
+new byte[] { PTable.LinkType.PHYSICAL_TABLE.getSerializedValue() };
+
 // KeyValues for Table
 private static final KeyValue TABLE_TYPE_KV = 
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, 
TABLE_TYPE_BYTES);
 private static final KeyValue TABLE_SEQ_NUM_KV = 
createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, 
TABLE_SEQ_NUM_BYTES);
@@ -1682,8 +1688,6 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 return rowLock;
 }
 
-private static final byte[] CHILD_TABLE_BYTES = new byte[] 
{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
-
 private PSchema loadSchema(RegionCoprocessorEnvironment env, byte[] key, 
ImmutableBytesPtr cacheKey,
 long clientTimeStamp, long asOfTimeStamp) throws IOException, 
SQLException {
 HRegion region = env.getRegion();
@@ -1704,7 +1708,7 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 
 private void findAllChildViews(HRegion region, byte[] tenantId, PTable 
table,
 TableViewFinder result, long clientTimeStamp, int clientVersion) 
throws IOException, SQLException {
-TableViewFinder currResult = findChildViews(region, tenantId, table, 
clientVersion);
+TableViewFinder currResult = findChildViews(region, tenantId, table, 
clientVersion, false);
 result.addResult(currResult);
 for (ViewInfo viewInfo : currResult.getViewInfoList()) {
 byte[] viewtenantId = viewInfo.getTenantId();
@@ -1717,9 +1721,9 @@ public class MetaDataEndpointImpl extends 
MetaDataProtocol implements Coprocesso
 }
 }
 
-// TODO remove this in 4.13 release 
-@Deprecated
-private TableViewFinder findChildViews_deprecated(HRegion region, byte[] 
tenantId, PTable table, byte[] linkTypeBytes) throws IOException {
+// TODO use child link instead once splittable system catalog 
(PHOENIX-3534) is implemented
+// and we have a separate table for links.
+private TableViewFinder findChildViews_deprecated(HRegion region, byte[] 
tenantId, PTable table, byte[] linkTypeBytes, boolean stopAfterFirst) throws 
IOException {
 byte[] schemaName = table.getSchemaName().getBytes();
 byte[] tableName = ta

  1   2   3   4   5   6   7   8   9   10   >