diff --git a/phoenix-core-client/src/main/java/org/apache/hadoop/hbase/ipc/controller/InvalidateMetadataCacheController.java b/phoenix-core-client/src/main/java/org/apache/hadoop/hbase/ipc/controller/InvalidateMetadataCacheController.java new file mode 100644 index 00000000000..cdfa9da168f --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/hadoop/hbase/ipc/controller/InvalidateMetadataCacheController.java @@ -0,0 +1,51 @@ +/** + * 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.hadoop.hbase.ipc.controller; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.ipc.DelegatingHBaseRpcController; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.phoenix.util.IndexUtil; + +/** + * Controller used to invalidate server side metadata cache RPCs. + */ +public class InvalidateMetadataCacheController extends DelegatingHBaseRpcController { + private int priority; + + public InvalidateMetadataCacheController(HBaseRpcController delegate, Configuration conf) { + super(delegate); + this.priority = IndexUtil.getInvalidateMetadataCachePriority(conf); + } + + @Override + public void setPriority(int priority) { + this.priority = priority; + } + + @Override + public void setPriority(TableName tn) { + // Nothing + } + + @Override + public int getPriority() { + return this.priority; + } +} diff --git a/phoenix-core-client/src/main/java/org/apache/hadoop/hbase/ipc/controller/InvalidateMetadataCacheControllerFactory.java b/phoenix-core-client/src/main/java/org/apache/hadoop/hbase/ipc/controller/InvalidateMetadataCacheControllerFactory.java new file mode 100644 index 00000000000..ee6b3b24ffa --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/hadoop/hbase/ipc/controller/InvalidateMetadataCacheControllerFactory.java @@ -0,0 +1,41 @@ +/** + * 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.hadoop.hbase.ipc.controller; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.ipc.RpcControllerFactory; + +/** + * Factory to instantiate InvalidateMetadataCacheControllers + */ +public class InvalidateMetadataCacheControllerFactory extends RpcControllerFactory { + public InvalidateMetadataCacheControllerFactory(Configuration conf) { + super(conf); + } + + @Override + public HBaseRpcController newController() { + HBaseRpcController delegate = super.newController(); + return getController(delegate); + } + + private HBaseRpcController getController(HBaseRpcController delegate) { + return new InvalidateMetadataCacheController(delegate, conf); + } +} diff --git a/phoenix-core-client/src/main/java/org/apache/hadoop/hbase/ipc/controller/ServerSideRPCControllerFactory.java b/phoenix-core-client/src/main/java/org/apache/hadoop/hbase/ipc/controller/ServerSideRPCControllerFactory.java index ba7fb6339d8..a1a97cf6ce7 100644 --- a/phoenix-core-client/src/main/java/org/apache/hadoop/hbase/ipc/controller/ServerSideRPCControllerFactory.java +++ b/phoenix-core-client/src/main/java/org/apache/hadoop/hbase/ipc/controller/ServerSideRPCControllerFactory.java @@ -19,8 +19,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** * {@link RpcControllerFactory} that should only be used when @@ -28,7 +26,6 @@ */ public class ServerSideRPCControllerFactory { - private static final Logger LOG = LoggerFactory.getLogger(ServerSideRPCControllerFactory.class); protected final Configuration conf; public ServerSideRPCControllerFactory(Configuration conf) { diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/cache/ServerMetadataCache.java b/phoenix-core-client/src/main/java/org/apache/phoenix/cache/ServerMetadataCache.java new file mode 100644 index 00000000000..f251dc7123c --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/cache/ServerMetadataCache.java @@ -0,0 +1,28 @@ +/* + * 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.cache; + +import java.sql.SQLException; + +/** + * Interface for server side metadata cache hosted on each region server. + */ +public interface ServerMetadataCache { + long getLastDDLTimestampForTable(byte[] tenantID, byte[] schemaName, byte[] tableName) + throws SQLException; + void invalidate(byte[] tenantID, byte[] schemaName, byte[] tableName); +} diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/cache/ServerMetadataCacheImpl.java b/phoenix-core-client/src/main/java/org/apache/phoenix/cache/ServerMetadataCacheImpl.java new file mode 100644 index 00000000000..5f9aa104556 --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/cache/ServerMetadataCacheImpl.java @@ -0,0 +1,156 @@ +/* + * 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.cache; + +import java.sql.Connection; +import java.sql.SQLException; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.phoenix.coprocessorclient.metrics.MetricsMetadataCachingSource; +import org.apache.phoenix.coprocessorclient.metrics.MetricsPhoenixCoprocessorSourceFactory; +import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr; +import org.apache.phoenix.jdbc.PhoenixConnection; +import org.apache.phoenix.schema.PTable; +import org.apache.phoenix.thirdparty.com.google.common.cache.Cache; +import org.apache.phoenix.thirdparty.com.google.common.cache.CacheBuilder; +import org.apache.phoenix.thirdparty.com.google.common.cache.RemovalListener; +import org.apache.phoenix.util.PhoenixRuntime; +import org.apache.phoenix.util.QueryUtil; +import org.apache.phoenix.util.SchemaUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB; +/** + * This manages the cache for all the objects(data table, views, indexes) on each region server. + * Currently, it only stores LAST_DDL_TIMESTAMP in the cache. + */ +public class ServerMetadataCacheImpl implements ServerMetadataCache { + + protected Configuration conf; + // key is the combination of , value is the lastDDLTimestamp + protected final Cache lastDDLTimestampMap; + private static final Logger LOGGER = LoggerFactory.getLogger(ServerMetadataCacheImpl.class); + private static final String PHOENIX_COPROC_REGIONSERVER_CACHE_SIZE + = "phoenix.coprocessor.regionserver.cache.size"; + private static final long DEFAULT_PHOENIX_COPROC_REGIONSERVER_CACHE_SIZE = 10000L; + private static volatile ServerMetadataCacheImpl cacheInstance; + private MetricsMetadataCachingSource metricsSource; + + /** + * Creates/gets an instance of ServerMetadataCache. + * + * @param conf configuration + * @return cache + */ + public static ServerMetadataCacheImpl getInstance(Configuration conf) { + ServerMetadataCacheImpl result = cacheInstance; + if (result == null) { + synchronized (ServerMetadataCacheImpl.class) { + result = cacheInstance; + if (result == null) { + cacheInstance = result = new ServerMetadataCacheImpl(conf); + } + } + } + return result; + } + + public ServerMetadataCacheImpl(Configuration conf) { + this.conf = HBaseConfiguration.create(conf); + this.metricsSource = MetricsPhoenixCoprocessorSourceFactory + .getInstance().getMetadataCachingSource(); + long maxSize = conf.getLong(PHOENIX_COPROC_REGIONSERVER_CACHE_SIZE, + DEFAULT_PHOENIX_COPROC_REGIONSERVER_CACHE_SIZE); + lastDDLTimestampMap = CacheBuilder.newBuilder() + .removalListener((RemovalListener) notification -> { + String key = notification.getKey().toString(); + LOGGER.debug("Expiring " + key + " because of " + + notification.getCause().name()); + }) + // maximum number of entries this cache can handle. + .maximumSize(maxSize) + .build(); + } + + /** + * Returns the last DDL timestamp from the table. + * If not found in cache, then query SYSCAT regionserver. + * @param tenantID tenant id + * @param schemaName schema name + * @param tableName table name + * @return last DDL timestamp + * @throws Exception + */ + public long getLastDDLTimestampForTable(byte[] tenantID, byte[] schemaName, byte[] tableName) + throws SQLException { + byte[] tableKey = SchemaUtil.getTableKey(tenantID, schemaName, tableName); + ImmutableBytesPtr tableKeyPtr = new ImmutableBytesPtr(tableKey); + // Lookup in cache if present. + Long lastDDLTimestamp = lastDDLTimestampMap.getIfPresent(tableKeyPtr); + if (lastDDLTimestamp != null) { + metricsSource.incrementRegionServerMetadataCacheHitCount(); + LOGGER.trace("Retrieving last ddl timestamp value from cache for " + "schema: {}, " + + "table: {}", Bytes.toString(schemaName), Bytes.toString(tableName)); + return lastDDLTimestamp; + } + metricsSource.incrementRegionServerMetadataCacheMissCount(); + PTable table; + String tenantIDStr = Bytes.toString(tenantID); + if (tenantIDStr == null || tenantIDStr.isEmpty()) { + tenantIDStr = null; + } + Properties properties = new Properties(); + if (tenantIDStr != null) { + properties.setProperty(TENANT_ID_ATTRIB, tenantIDStr); + } + try (Connection connection = getConnection(properties)) { + // Using PhoenixConnection#getTableFromServerNoCache to completely bypass CQSI cache. + table = connection.unwrap(PhoenixConnection.class) + .getTableFromServerNoCache(schemaName, tableName); + // TODO PhoenixConnection#getTableFromServerNoCache can throw TableNotFoundException. + // In that case, do we want to throw non retryable exception back to the client? + // Update cache with the latest DDL timestamp from SYSCAT server. + lastDDLTimestampMap.put(tableKeyPtr, table.getLastDDLTimestamp()); + } + return table.getLastDDLTimestamp(); + } + + /** + * Invalidate cache for the given tenantID, schema name and table name. + * Guava cache is thread safe so we don't have to synchronize it explicitly. + * @param tenantID tenantID + * @param schemaName schemaName + * @param tableName tableName + */ + public void invalidate(byte[] tenantID, byte[] schemaName, byte[] tableName) { + LOGGER.info("Invalidating server metadata cache for tenantID: {}, schema: {}, table: {}", + Bytes.toString(tenantID), Bytes.toString(schemaName), Bytes.toString(tableName)); + byte[] tableKey = SchemaUtil.getTableKey(tenantID, schemaName, tableName); + ImmutableBytesPtr tableKeyPtr = new ImmutableBytesPtr(tableKey); + lastDDLTimestampMap.invalidate(tableKeyPtr); + } + + protected Connection getConnection(Properties properties) throws SQLException { + return QueryUtil.getConnectionOnServer(properties, this.conf); + } +} diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/compile/CreateIndexCompiler.java b/phoenix-core-client/src/main/java/org/apache/phoenix/compile/CreateIndexCompiler.java index 787067266bc..5e8c1e835a7 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/compile/CreateIndexCompiler.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/compile/CreateIndexCompiler.java @@ -227,7 +227,9 @@ private void verifyIndexWhere(ParseNode indexWhere, StatementContext context, } public MutationPlan compile(final CreateIndexStatement create) throws SQLException { final PhoenixConnection connection = statement.getConnection(); - final ColumnResolver resolver = FromCompiler.getResolver(create, connection, create.getUdfParseNodes()); + final ColumnResolver resolver + = FromCompiler.getResolverForCreateIndex( + create, connection, create.getUdfParseNodes()); Scan scan = new Scan(); final StatementContext context = new StatementContext(statement, resolver, scan, new SequenceManager(statement)); verifyIndexWhere(create.getWhere(), context, create.getTable().getName()); diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/compile/FromCompiler.java b/phoenix-core-client/src/main/java/org/apache/phoenix/compile/FromCompiler.java index ec8b3a71892..9d3a6b8952f 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/compile/FromCompiler.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/compile/FromCompiler.java @@ -180,7 +180,8 @@ public static ColumnResolver getResolverForCreation(final CreateTableStatement s NamedTableNode tableNode = NamedTableNode.create(null, baseTable, Collections.emptyList()); // Always use non-tenant-specific connection here try { - SingleTableColumnResolver visitor = new SingleTableColumnResolver(connection, tableNode, true); + SingleTableColumnResolver visitor + = new SingleTableColumnResolver(connection, tableNode, true, true); return visitor; } catch (TableNotFoundException e) { // Used for mapped VIEW, since we won't be able to resolve that. @@ -280,7 +281,8 @@ public static ColumnResolver getResolver(NamedTableNode tableNode, PhoenixConnec public static ColumnResolver getResolver(SingleTableStatement statement, PhoenixConnection connection) throws SQLException { - SingleTableColumnResolver visitor = new SingleTableColumnResolver(connection, statement.getTable(), true); + SingleTableColumnResolver visitor + = new SingleTableColumnResolver(connection, statement.getTable(), true, true); return visitor; } @@ -293,9 +295,14 @@ public static ColumnResolver getIndexResolver(SingleTableStatement statement, } } - public static ColumnResolver getResolver(SingleTableStatement statement, PhoenixConnection connection, Map udfParseNodes) + public static ColumnResolver getResolverForCreateIndex(SingleTableStatement statement, + PhoenixConnection connection, Map udfParseNodes) throws SQLException { - SingleTableColumnResolver visitor = new SingleTableColumnResolver(connection, statement.getTable(), true, 0, udfParseNodes); + // use alwaysHitServer=true to ensure client's cache is up-to-date even when client is + // validating last_ddl_timestamps and UCF = never. + SingleTableColumnResolver visitor + = new SingleTableColumnResolver(connection, statement.getTable(), true, 0, + udfParseNodes, true, null); return visitor; } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/compile/StatementContext.java b/phoenix-core-client/src/main/java/org/apache/phoenix/compile/StatementContext.java index 8c198486b04..96d4e36c45b 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/compile/StatementContext.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/compile/StatementContext.java @@ -289,7 +289,7 @@ public PhoenixStatement getStatement() { } public long getCurrentTime() throws SQLException { - long ts = this.getCurrentTable().getCurrentTime(); + long ts = this.getCurrentTable().getTimeStamp(); // if the table is transactional then it is only resolved once per query, so we can't use the table timestamp if (this.getCurrentTable().getTable().getType() != PTableType.SUBQUERY && this.getCurrentTable().getTable().getType() != PTableType.PROJECTED diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core-client/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java index 890b99a90c5..c996516628a 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java @@ -636,7 +636,7 @@ public MutationPlan compile(UpsertStatement upsert) throws SQLException { // as max TS, so that the query can safely restarted and still work of a snapshot // (so it won't see its own data in case of concurrent splits) // see PHOENIX-4849 - long serverTime = selectResolver.getTables().get(0).getCurrentTime(); + long serverTime = selectResolver.getTables().get(0).getTimeStamp(); if (serverTime == QueryConstants.UNSET_TIMESTAMP) { // if this is the first time this table is resolved the ref's current time might not be defined, yet // in that case force an RPC to get the server time diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/InvalidateServerMetadataCacheRequest.java b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/InvalidateServerMetadataCacheRequest.java new file mode 100644 index 00000000000..7ac94c8968d --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/InvalidateServerMetadataCacheRequest.java @@ -0,0 +1,53 @@ +/* + * 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.coprocessorclient; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.phoenix.util.SchemaUtil; + +public class InvalidateServerMetadataCacheRequest { + private final byte[] tenantId; + private final byte[] schemaName; + private final byte[] tableName; + + public InvalidateServerMetadataCacheRequest(byte[] tenantId, byte[] schemaName, + byte[] tableName) { + this.tenantId = tenantId; + this.schemaName = schemaName; + this.tableName = tableName; + } + + public byte[] getTenantId() { + return tenantId; + } + + public byte[] getSchemaName() { + return schemaName; + } + + public byte[] getTableName() { + return tableName; + } + + @Override + public String toString() { + String fullTableName = SchemaUtil.getTableName(schemaName, tableName); + return "tenantId = " + Bytes.toString(tenantId) + + ", table name = " + fullTableName; + } +} diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/metrics/MetricsMetadataCachingSource.java b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/metrics/MetricsMetadataCachingSource.java new file mode 100644 index 00000000000..2d810d5fa48 --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/metrics/MetricsMetadataCachingSource.java @@ -0,0 +1,230 @@ +/* + * 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.coprocessorclient.metrics; + +import org.apache.hadoop.hbase.metrics.BaseSource; +import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; + +/** + * Interface for metrics about Distributed Metadata Caching + */ +public interface MetricsMetadataCachingSource extends BaseSource { + // Metrics2 and JMX constants + String METRICS_NAME = "MetadataCaching"; + String METRICS_CONTEXT = "phoenix"; + String METRICS_DESCRIPTION = "Metrics about Distributed Metadata Caching"; + String METRICS_JMX_CONTEXT = "RegionServer,sub=" + METRICS_NAME; + + String REGIONSERVER_METADATA_CACHE_HITS = "numRegionServerMetadataCacheHits"; + String REGIONSERVER_METADATA_CACHE_HITS_DESC + = "Number of cache hits in PhoenixRegionServerEndpoint " + + "when serving validate ddl timestamp requests."; + + String REGIONSERVER_METADATA_CACHE_MISSES = "numRegionServerMetadataCacheMisses"; + String REGIONSERVER_METADATA_CACHE_MISSES_DESC + = "Number of cache misses in PhoenixRegionServerEndpoint " + + "when serving validate ddl timestamp requests."; + + String VALIDATE_LAST_DDL_TIMESTAMP_REQUESTS = "numValidateLastDDLTimestampRequests"; + String VALIDATE_LAST_DDL_TIMESTAMP_REQUEST_DESC + = "Number of validate last ddl timestamp requests."; + + String METADATA_CACHE_INVALIDATION_OPERATIONS = "numMetadataCacheInvalidationOps"; + String METADATA_CACHE_INVALIDATION_OPERATIONS_DESC = "Number of times we invoke " + + "cache invalidation within a DDL operation"; + + String METADATA_CACHE_INVALIDATION_SUCCESS = "numMetadataCacheInvalidationOpsSuccess"; + String METADATA_CACHE_INVALIDATION_SUCCESS_DESC + = "Number of times cache invalidation was successful."; + + String METADATA_CACHE_INVALIDATION_FAILURE = "numMetadataCacheInvalidationOpsFailure"; + String METADATA_CACHE_INVALIDATION_FAILURE_DESC = "Number of times cache invalidation failed."; + + String METADATA_CACHE_INVALIDATION_RPC_TIME = "metadataCacheInvalidationRpcTimeMs"; + String METADATA_CACHE_INVALIDATION_RPC_TIME_DESC = "Histogram for the time in milliseconds for" + + " cache invalidation RPC"; + String METADATA_CACHE_INVALIDATION_TOTAL_TIME = "metadataCacheInvalidationTotalTimeMs"; + String METADATA_CACHE_INVALIDATION_TOTAL_TIME_DESC + = "Histogram for the total time in milliseconds " + + "for cache invalidation on all regionservers"; + + /** + * Report the number of cache hits when validating last ddl timestamps. + */ + void incrementRegionServerMetadataCacheHitCount(); + + /** + * Report the number of cache misses when validating last ddl timestamps. + */ + void incrementRegionServerMetadataCacheMissCount(); + + /** + * Report the number of requests for validating last ddl timestamps. + */ + void incrementValidateTimestampRequestCount(); + + /** + * Report number of cache invalidations performed. + */ + void incrementMetadataCacheInvalidationOperationsCount(); + + /** + * Report number of cache invalidations which were successful. + */ + void incrementMetadataCacheInvalidationSuccessCount(); + + /** + * Report number of cache invalidations which failed. + */ + void incrementMetadataCacheInvalidationFailureCount(); + + /** + * Add to the cache invalidation rpc time histogram. + */ + void addMetadataCacheInvalidationRpcTime(long t); + + /** + * Add to the cache invalidation total time histogram. + * @param t + */ + void addMetadataCacheInvalidationTotalTime(long t); + + /** + * Return current values of all metrics. + * @return {@link MetadataCachingMetricValues} object + */ + @VisibleForTesting + MetadataCachingMetricValues getCurrentMetricValues(); + + /** + * Class to represent values of all metrics related to server metadata caching. + */ + @VisibleForTesting + class MetadataCachingMetricValues { + private long cacheHitCount; + private long cacheMissCount; + private long validateDDLTimestampRequestsCount; + private long cacheInvalidationOpsCount; + private long cacheInvalidationSuccessCount; + private long cacheInvalidationFailureCount; + private long cacheInvalidationRpcTimeCount; + private long cacheInvalidationTotalTimeCount; + + MetadataCachingMetricValues(Builder builder) { + this.cacheHitCount = builder.cacheHitCount; + this.cacheMissCount = builder.cacheMissCount; + this.validateDDLTimestampRequestsCount = builder.validateDDLTimestampRequestsCount; + this.cacheInvalidationOpsCount = builder.cacheInvalidationOpsCount; + this.cacheInvalidationSuccessCount = builder.cacheInvalidationSuccessCount; + this.cacheInvalidationFailureCount = builder.cacheInvalidationFailureCount; + this.cacheInvalidationRpcTimeCount = builder.cacheInvalidationRpcTimeCount; + this.cacheInvalidationTotalTimeCount = builder.cacheInvalidationTotalTimeCount; + } + + public long getCacheHitCount() { + return cacheHitCount; + } + + public long getCacheMissCount() { + return cacheMissCount; + } + + public long getValidateDDLTimestampRequestsCount() { + return validateDDLTimestampRequestsCount; + } + + public long getCacheInvalidationOpsCount() { + return cacheInvalidationOpsCount; + } + + public long getCacheInvalidationSuccessCount() { + return cacheInvalidationSuccessCount; + } + + public long getCacheInvalidationFailureCount() { + return cacheInvalidationFailureCount; + } + + public long getCacheInvalidationRpcTimeCount() { + return cacheInvalidationRpcTimeCount; + } + + public long getCacheInvalidationTotalTimeCount() { + return cacheInvalidationTotalTimeCount; + } + + /** + * Builder for {@link MetadataCachingMetricValues} + */ + public static class Builder { + private long cacheHitCount; + private long cacheMissCount; + private long validateDDLTimestampRequestsCount; + private long cacheInvalidationOpsCount; + private long cacheInvalidationSuccessCount; + private long cacheInvalidationFailureCount; + private long cacheInvalidationRpcTimeCount; + private long cacheInvalidationTotalTimeCount; + + public MetadataCachingMetricValues build() { + return new MetadataCachingMetricValues(this); + } + + public Builder setCacheHitCount(long c) { + this.cacheHitCount = c; + return this; + } + public Builder setCacheMissCount(long cacheMissCount) { + this.cacheMissCount = cacheMissCount; + return this; + } + + public Builder setValidateDDLTimestampRequestsCount( + long validateDDLTimestampRequestsCount) { + this.validateDDLTimestampRequestsCount = validateDDLTimestampRequestsCount; + return this; + } + + public Builder setCacheInvalidationOpsCount(long cacheInvalidationOpsCount) { + this.cacheInvalidationOpsCount = cacheInvalidationOpsCount; + return this; + } + + public Builder setCacheInvalidationSuccessCount(long cacheInvalidationSuccessCount) { + this.cacheInvalidationSuccessCount = cacheInvalidationSuccessCount; + return this; + } + + public Builder setCacheInvalidationFailureCount(long cacheInvalidationFailureCount) { + this.cacheInvalidationFailureCount = cacheInvalidationFailureCount; + return this; + } + + public Builder setCacheInvalidationRpcTimeCount(long cacheInvalidationRpcTimeCount) { + this.cacheInvalidationRpcTimeCount = cacheInvalidationRpcTimeCount; + return this; + } + + public Builder setCacheInvalidationTotalTimeCount( + long cacheInvalidationTotalTimeCount) { + this.cacheInvalidationTotalTimeCount = cacheInvalidationTotalTimeCount; + return this; + } + } + } +} diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/metrics/MetricsMetadataCachingSourceImpl.java b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/metrics/MetricsMetadataCachingSourceImpl.java new file mode 100644 index 00000000000..1fc249bcc2c --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/metrics/MetricsMetadataCachingSourceImpl.java @@ -0,0 +1,122 @@ +/* + * 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.coprocessorclient.metrics; + +import org.apache.hadoop.hbase.metrics.BaseSourceImpl; +import org.apache.hadoop.metrics2.MetricHistogram; +import org.apache.hadoop.metrics2.lib.MutableFastCounter; + +/** + * Implementation for tracking Distributed Metadata Caching metrics. + */ +public class MetricsMetadataCachingSourceImpl + extends BaseSourceImpl + implements MetricsMetadataCachingSource { + + private final MutableFastCounter cacheHitCounter; + private final MutableFastCounter cacheMissCounter; + private final MutableFastCounter validateDDLTimestampRequestCounter; + private final MutableFastCounter cacheInvalidationOpsCounter; + private final MutableFastCounter cacheInvalidationSuccessCounter; + private final MutableFastCounter cacheInvalidationFailureCounter; + private final MetricHistogram cacheInvalidationRpcTimeHistogram; + private final MetricHistogram cacheInvalidationTotalTimeHistogram; + + public MetricsMetadataCachingSourceImpl() { + this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT); + } + + public MetricsMetadataCachingSourceImpl(String metricsName, + String metricsDescription, + String metricsContext, + String metricsJmxContext) { + super(metricsName, metricsDescription, metricsContext, metricsJmxContext); + cacheHitCounter = getMetricsRegistry().newCounter( + REGIONSERVER_METADATA_CACHE_HITS, REGIONSERVER_METADATA_CACHE_HITS_DESC, 0L); + cacheMissCounter = getMetricsRegistry().newCounter( + REGIONSERVER_METADATA_CACHE_MISSES, REGIONSERVER_METADATA_CACHE_MISSES_DESC, 0L); + validateDDLTimestampRequestCounter = getMetricsRegistry().newCounter( + VALIDATE_LAST_DDL_TIMESTAMP_REQUESTS, VALIDATE_LAST_DDL_TIMESTAMP_REQUEST_DESC, 0L); + cacheInvalidationOpsCounter = getMetricsRegistry().newCounter( + METADATA_CACHE_INVALIDATION_OPERATIONS, + METADATA_CACHE_INVALIDATION_OPERATIONS_DESC, 0L); + cacheInvalidationSuccessCounter = getMetricsRegistry().newCounter( + METADATA_CACHE_INVALIDATION_SUCCESS, METADATA_CACHE_INVALIDATION_SUCCESS_DESC, 0L); + cacheInvalidationFailureCounter = getMetricsRegistry().newCounter( + METADATA_CACHE_INVALIDATION_FAILURE, METADATA_CACHE_INVALIDATION_FAILURE_DESC, 0L); + cacheInvalidationRpcTimeHistogram = getMetricsRegistry().newHistogram( + METADATA_CACHE_INVALIDATION_RPC_TIME, METADATA_CACHE_INVALIDATION_RPC_TIME_DESC); + cacheInvalidationTotalTimeHistogram = getMetricsRegistry().newHistogram( + METADATA_CACHE_INVALIDATION_TOTAL_TIME, METADATA_CACHE_INVALIDATION_TOTAL_TIME_DESC); + } + + @Override + public void incrementRegionServerMetadataCacheHitCount() { + cacheHitCounter.incr(); + } + + @Override + public void incrementRegionServerMetadataCacheMissCount() { + cacheMissCounter.incr(); + } + + @Override + public void incrementValidateTimestampRequestCount() { + validateDDLTimestampRequestCounter.incr(); + } + + @Override + public void addMetadataCacheInvalidationRpcTime(long t) { + cacheInvalidationRpcTimeHistogram.add(t); + } + + @Override + public void addMetadataCacheInvalidationTotalTime(long t) { + cacheInvalidationTotalTimeHistogram.add(t); + } + + @Override + public void incrementMetadataCacheInvalidationOperationsCount() { + cacheInvalidationOpsCounter.incr(); + } + + @Override + public void incrementMetadataCacheInvalidationSuccessCount() { + cacheInvalidationSuccessCounter.incr(); + } + + @Override + public void incrementMetadataCacheInvalidationFailureCount() { + cacheInvalidationFailureCounter.incr(); + } + + @Override + public MetadataCachingMetricValues getCurrentMetricValues() { + return new MetadataCachingMetricValues + .Builder() + .setCacheHitCount(cacheHitCounter.value()) + .setCacheMissCount(cacheMissCounter.value()) + .setValidateDDLTimestampRequestsCount(validateDDLTimestampRequestCounter.value()) + .setCacheInvalidationRpcTimeCount(cacheInvalidationRpcTimeHistogram.getCount()) + .setCacheInvalidationTotalTimeCount(cacheInvalidationTotalTimeHistogram.getCount()) + .setCacheInvalidationOpsCount(cacheInvalidationOpsCounter.value()) + .setCacheInvalidationSuccessCount(cacheInvalidationSuccessCounter.value()) + .setCacheInvalidationFailureCount(cacheInvalidationFailureCounter.value()) + .build(); + } +} diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/metrics/MetricsPhoenixCoprocessorSourceFactory.java b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/metrics/MetricsPhoenixCoprocessorSourceFactory.java index 0742f4ba6cc..280a116f686 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/metrics/MetricsPhoenixCoprocessorSourceFactory.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/metrics/MetricsPhoenixCoprocessorSourceFactory.java @@ -26,6 +26,7 @@ public class MetricsPhoenixCoprocessorSourceFactory { INSTANCE = new MetricsPhoenixCoprocessorSourceFactory(); // Holds the PHOENIX_TTL related metrics. private static volatile MetricsPhoenixTTLSource phoenixTTLSource; + private static volatile MetricsMetadataCachingSource metadataCachingSource; public static MetricsPhoenixCoprocessorSourceFactory getInstance() { return INSTANCE; @@ -42,4 +43,15 @@ public MetricsPhoenixTTLSource getPhoenixTTLSource() { } return INSTANCE.phoenixTTLSource; } + + public MetricsMetadataCachingSource getMetadataCachingSource() { + if (INSTANCE.metadataCachingSource == null) { + synchronized (MetricsMetadataCachingSource.class) { + if (INSTANCE.metadataCachingSource == null) { + INSTANCE.metadataCachingSource = new MetricsMetadataCachingSourceImpl(); + } + } + } + return INSTANCE.metadataCachingSource; + } } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core-client/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java index 7e0c4818ff2..048c80a0c87 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java @@ -613,6 +613,9 @@ public SQLException newException(SQLExceptionInfo info) { CANNOT_TRANSFORM_TRANSACTIONAL_TABLE(914, "43M25", "Cannot transform a transactional table."), + STALE_METADATA_CACHE_EXCEPTION(915, "43M26", "Stale metadata cache exception", + info -> new StaleMetadataCacheException(info.getMessage())), + //SQLCode for testing exceptions FAILED_KNOWINGLY_FOR_TEST(7777, "TEST", "Exception was thrown to test something"); diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/exception/StaleMetadataCacheException.java b/phoenix-core-client/src/main/java/org/apache/phoenix/exception/StaleMetadataCacheException.java new file mode 100644 index 00000000000..584e33e0b86 --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/exception/StaleMetadataCacheException.java @@ -0,0 +1,32 @@ +/* + * 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.exception; + +import java.sql.SQLException; + +/** + * Indicates metadata cache is stale. + */ +public class StaleMetadataCacheException extends SQLException { + private static final long serialVersionUID = 1L; + private static final SQLExceptionCode code = SQLExceptionCode.STALE_METADATA_CACHE_EXCEPTION; + + public StaleMetadataCacheException(String message) { + super(message, code.getSQLState(), code.getErrorCode()); + } +} \ No newline at end of file diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core-client/src/main/java/org/apache/phoenix/execute/MutationState.java index c0e39021299..1f2493aab24 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/execute/MutationState.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/execute/MutationState.java @@ -71,6 +71,7 @@ import org.apache.phoenix.coprocessorclient.MetaDataProtocol.MetaDataMutationResult; import org.apache.phoenix.exception.SQLExceptionCode; import org.apache.phoenix.exception.SQLExceptionInfo; +import org.apache.phoenix.exception.StaleMetadataCacheException; import org.apache.phoenix.hbase.index.AbstractValueGetter; import org.apache.phoenix.hbase.index.ValueGetter; import org.apache.phoenix.hbase.index.covered.update.ColumnReference; @@ -128,6 +129,7 @@ import org.apache.phoenix.util.SchemaUtil; import org.apache.phoenix.util.SizedUtil; import org.apache.phoenix.util.TransactionUtil; +import org.apache.phoenix.util.ValidateLastDDLTimestampUtil; import org.apache.phoenix.util.WALAnnotationUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -165,6 +167,7 @@ public class MutationState implements SQLCloseable { private long estimatedSize = 0; private int[] uncommittedStatementIndexes = EMPTY_STATEMENT_INDEX_ARRAY; private boolean isExternalTxContext = false; + private boolean validateLastDdlTimestamp; private Map> txMutations = Collections.emptyMap(); private PhoenixTransactionContext phoenixTransactionContext = PhoenixTransactionContext.NULL_CONTEXT; @@ -229,6 +232,8 @@ private MutationState(int maxSize, long maxSizeBytes, PhoenixConnection connecti boolean isMetricsEnabled = connection.isRequestLevelMetricsEnabled(); this.mutationMetricQueue = isMetricsEnabled ? new MutationMetricQueue() : NoOpMutationMetricsQueue.NO_OP_MUTATION_METRICS_QUEUE; + this.validateLastDdlTimestamp = ValidateLastDDLTimestampUtil + .getValidateLastDdlTimestampEnabled(this.connection); 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 @@ -1027,26 +1032,33 @@ private long validateAndGetServerTimestamp(TableRef tableRef, MultiRowMutationSt .setTableName(table.getTableName().getString()).build().buildException(); } } long timestamp = result.getMutationTime(); - if (timestamp != QueryConstants.UNSET_TIMESTAMP) { - serverTimeStamp = timestamp; - if (result.wasUpdated()) { - List columns = Lists.newArrayListWithExpectedSize(table.getColumns().size()); - for (Map.Entry rowEntry : rowKeyToColumnMap.entrySet()) { - RowMutationState valueEntry = rowEntry.getValue(); - if (valueEntry != null) { - Map colValues = valueEntry.getColumnValues(); - if (colValues != PRow.DELETE_MARKER) { - for (PColumn column : colValues.keySet()) { - if (!column.isDynamic()) columns.add(column); + serverTimeStamp = timestamp; + + /* when last_ddl_timestamp validation is enabled, + we don't know if this table's cache result was force updated + during the validation, so always validate columns */ + if ((timestamp != QueryConstants.UNSET_TIMESTAMP && result.wasUpdated()) + || this.validateLastDdlTimestamp) { + List columns + = Lists.newArrayListWithExpectedSize(table.getColumns().size()); + for (Map.Entry + rowEntry : rowKeyToColumnMap.entrySet()) { + RowMutationState valueEntry = rowEntry.getValue(); + if (valueEntry != null) { + Map colValues = valueEntry.getColumnValues(); + if (colValues != PRow.DELETE_MARKER) { + for (PColumn column : colValues.keySet()) { + if (!column.isDynamic()) { + columns.add(column); } } } } - for (PColumn column : columns) { - if (column != null) { - resolvedTable.getColumnFamily(column.getFamilyName().getString()).getPColumnForColumnName( - column.getName().getString()); - } + } + for (PColumn column : columns) { + if (column != null) { + resolvedTable.getColumnFamily(column.getFamilyName().getString()) + .getPColumnForColumnName(column.getName().getString()); } } } @@ -1282,6 +1294,29 @@ private void send(Iterator tableRefIterator) throws SQLException { commitBatches = createCommitBatches(tableRefIterator); } + //if enabled, validate last ddl timestamps for all tables in the mutationsMap + //for now, force update client cache for all tables if StaleMetadataCacheException is seen + //mutationsMap can be empty, for e.g. during a DDL operation + if (this.validateLastDdlTimestamp && !this.mutationsMap.isEmpty()) { + List tableRefs = new ArrayList<>(this.mutationsMap.keySet()); + try { + ValidateLastDDLTimestampUtil.validateLastDDLTimestamp( + connection, tableRefs, true); + } catch (StaleMetadataCacheException e) { + GlobalClientMetrics + .GLOBAL_CLIENT_STALE_METADATA_CACHE_EXCEPTION_COUNTER.increment(); + MetaDataClient mc = new MetaDataClient(connection); + PName tenantId = connection.getTenantId(); + LOGGER.debug("Force updating client metadata cache for {}", + ValidateLastDDLTimestampUtil.getInfoString(tenantId, tableRefs)); + for (TableRef tableRef : tableRefs) { + String schemaName = tableRef.getTable().getSchemaName().toString(); + String tableName = tableRef.getTable().getTableName().toString(); + mc.updateCache(tenantId, schemaName, tableName, true); + } + } + } + for (Map commitBatch : commitBatches) { long [] serverTimestamps = validateServerTimestamps ? validateAll(commitBatch) : null; sendBatch(commitBatch, serverTimestamps, sendAll); diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java index 597436f2b38..e2830f19a8a 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java @@ -129,6 +129,7 @@ import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableMap.Builder; import org.apache.phoenix.trace.util.Tracing; import org.apache.phoenix.transaction.PhoenixTransactionContext; +import org.apache.phoenix.util.ByteUtil; import org.apache.phoenix.util.DateUtil; import org.apache.phoenix.util.EnvironmentEdgeManager; import org.apache.phoenix.util.JDBCUtil; @@ -726,6 +727,23 @@ public PTable getTableNoCache(String name) throws SQLException { return getTableNoCache(getTenantId(), name); } + /** + * Returns the most recent PTable fetched from the server without updating the CQSI cache. + */ + public PTable getTableFromServerNoCache(byte[] schemaName, byte[] tableName) + throws SQLException { + if (schemaName == null) { + schemaName = ByteUtil.EMPTY_BYTE_ARRAY; + } + MetaDataProtocol.MetaDataMutationResult result = + getQueryServices().getTable(getTenantId(), schemaName, + tableName, HConstants.LATEST_TIMESTAMP, HConstants.LATEST_TIMESTAMP); + if (result.getMutationCode() != MetaDataProtocol.MutationCode.TABLE_ALREADY_EXISTS) { + throw new TableNotFoundException(new String(schemaName), new String(tableName)); + } + return result.getTable(); + } + /** * Returns the table if it is found in the client metadata cache. If the metadata of this * table has changed since it was put in the cache these changes will not necessarily be diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java index 3bee7a33a68..c7e51cd8b7f 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java @@ -66,7 +66,6 @@ import java.util.Properties; import java.util.Set; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -105,6 +104,7 @@ import org.apache.phoenix.coprocessorclient.MetaDataProtocol; import org.apache.phoenix.exception.SQLExceptionCode; import org.apache.phoenix.exception.SQLExceptionInfo; +import org.apache.phoenix.exception.StaleMetadataCacheException; import org.apache.phoenix.exception.UpgradeRequiredException; import org.apache.phoenix.execute.MutationState; import org.apache.phoenix.execute.visitor.QueryPlanVisitor; @@ -121,6 +121,7 @@ import org.apache.phoenix.log.QueryLogger; import org.apache.phoenix.log.QueryLoggerUtil; import org.apache.phoenix.log.QueryStatus; +import org.apache.phoenix.monitoring.GlobalClientMetrics; import org.apache.phoenix.monitoring.TableMetricsManager; import org.apache.phoenix.optimize.Cost; import org.apache.phoenix.parse.AddColumnStatement; @@ -198,6 +199,7 @@ import org.apache.phoenix.schema.PColumnImpl; import org.apache.phoenix.schema.PDatum; import org.apache.phoenix.schema.PIndexState; +import org.apache.phoenix.schema.PName; import org.apache.phoenix.schema.PNameFactory; import org.apache.phoenix.schema.PTable; import org.apache.phoenix.schema.PTable.IndexType; @@ -226,6 +228,7 @@ import org.apache.phoenix.util.QueryUtil; import org.apache.phoenix.util.SQLCloseable; import org.apache.phoenix.util.ParseNodeUtil.RewriteResult; +import org.apache.phoenix.util.ValidateLastDDLTimestampUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -296,10 +299,13 @@ public String toString() { private int queryTimeoutMillis; // Caching per Statement protected final Calendar localCalendar = Calendar.getInstance(); + private boolean validateLastDdlTimestamp; public PhoenixStatement(PhoenixConnection connection) { this.connection = connection; this.queryTimeoutMillis = getDefaultQueryTimeoutMillis(); + this.validateLastDdlTimestamp = ValidateLastDDLTimestampUtil + .getValidateLastDdlTimestampEnabled(this.connection); } /** @@ -330,16 +336,20 @@ protected QueryPlan optimizeQuery(CompilableStatement stmt) throws SQLException protected PhoenixResultSet executeQuery(final CompilableStatement stmt, final QueryLogger queryLogger) throws SQLException { - return executeQuery(stmt, true, queryLogger, false); + return executeQuery(stmt, true, queryLogger, false, this.validateLastDdlTimestamp); } protected PhoenixResultSet executeQuery(final CompilableStatement stmt, final QueryLogger queryLogger, boolean noCommit) throws SQLException { - return executeQuery(stmt, true, queryLogger, noCommit); + return executeQuery(stmt, true, queryLogger, noCommit, this.validateLastDdlTimestamp); } + private PhoenixResultSet executeQuery(final CompilableStatement stmt, - final boolean doRetryOnMetaNotFoundError, final QueryLogger queryLogger, final boolean noCommit) throws SQLException { + final boolean doRetryOnMetaNotFoundError, + final QueryLogger queryLogger, final boolean noCommit, + boolean shouldValidateLastDdlTimestamp) + throws SQLException { GLOBAL_SELECT_SQL_COUNTER.increment(); try { @@ -348,10 +358,12 @@ private PhoenixResultSet executeQuery(final CompilableStatement stmt, @Override public PhoenixResultSet call() throws SQLException { final long startTime = EnvironmentEdgeManager.currentTimeMillis(); boolean success = false; + boolean updateMetrics = true; boolean pointLookup = false; String tableName = null; clearResultSet(); PhoenixResultSet rs = null; + QueryPlan plan = null; try { PhoenixConnection conn = getConnection(); conn.checkOpen(); @@ -361,9 +373,7 @@ private PhoenixResultSet executeQuery(final CompilableStatement stmt, && stmt.getOperation() != Operation.UPGRADE) { throw new UpgradeRequiredException(); } - QueryPlan - plan = - stmt.compilePlan(PhoenixStatement.this, + plan = stmt.compilePlan(PhoenixStatement.this, Sequence.ValueOp.VALIDATE_SEQUENCE); // Send mutations to hbase, so they are visible to subsequent reads. // Use original plan for data table so that data and immutable indexes will be sent @@ -377,6 +387,14 @@ private PhoenixResultSet executeQuery(final CompilableStatement stmt, plan = connection.getQueryServices().getOptimizer() .optimize(PhoenixStatement.this, plan); + setLastQueryPlan(plan); + + //verify metadata for the table/view/index in the query plan + //plan.getTableRef can be null in some cases like EXPLAIN + if (shouldValidateLastDdlTimestamp && plan.getTableRef() != null) { + ValidateLastDDLTimestampUtil.validateLastDDLTimestamp( + connection, Arrays.asList(plan.getTableRef()), true); + } if (plan.getTableRef() != null && plan.getTableRef().getTable() != null && !Strings @@ -423,21 +441,55 @@ private PhoenixResultSet executeQuery(final CompilableStatement stmt, //Force update cache and retry if meta not found error occurs catch (MetaDataEntityNotFoundException e) { if (doRetryOnMetaNotFoundError && e.getTableName() != null) { + String sName = e.getSchemaName(); + String tName = e.getTableName(); + // when the query plan uses the local index PTable, + // the TNFE can still be for the base table + if (plan != null && plan.getTableRef() != null) { + PTable queryPlanTable = plan.getTableRef().getTable(); + if (queryPlanTable != null + && queryPlanTable.getIndexType() + == IndexType.LOCAL) { + sName = queryPlanTable.getSchemaName().getString(); + tName = queryPlanTable.getTableName().getString(); + } + } if (LOGGER.isDebugEnabled()) { LOGGER.debug("Reloading table {} data from server", - e.getTableName()); + tName); } if (new MetaDataClient(connection) .updateCache(connection.getTenantId(), - e.getSchemaName(), e.getTableName(), true) + sName, tName, true) .wasUpdated()) { + updateMetrics = false; //TODO we can log retry count and error for debugging in LOG table - return executeQuery(stmt, false, queryLogger, noCommit); + return executeQuery(stmt, false, queryLogger, noCommit, + shouldValidateLastDdlTimestamp); } } throw e; - } catch (RuntimeException e) { - + } catch (StaleMetadataCacheException e) { + GlobalClientMetrics + .GLOBAL_CLIENT_STALE_METADATA_CACHE_EXCEPTION_COUNTER + .increment(); + updateMetrics = false; + PTable pTable = lastQueryPlan.getTableRef().getTable(); + String schemaN = pTable.getSchemaName().toString(); + String tableN = pTable.getTableName().toString(); + PName tenantId = connection.getTenantId(); + LOGGER.debug("Force updating client metadata cache for {}", + ValidateLastDDLTimestampUtil.getInfoString(tenantId, + Arrays.asList(getLastQueryPlan().getTableRef()))); + // force update client metadata cache for the table/view + // this also updates the cache for all ancestors in case of a view + new MetaDataClient(connection) + .updateCache(tenantId, schemaN, tableN, true); + // skip last ddl timestamp validation in the retry + return executeQuery(stmt, doRetryOnMetaNotFoundError, queryLogger, + noCommit, false); + } + catch (RuntimeException e) { // FIXME: Expression.evaluate does not throw SQLException // so this will unwrap throws from that. if (e.getCause() instanceof SQLException) { @@ -445,41 +497,43 @@ private PhoenixResultSet executeQuery(final CompilableStatement stmt, } throw e; } finally { - // Regardless of whether the query was successfully handled or not, - // update the time spent so far. If needed, we can separate out the - // success times and failure times. - GLOBAL_QUERY_TIME.update(EnvironmentEdgeManager.currentTimeMillis() - - startTime); - long - executeQueryTimeSpent = - EnvironmentEdgeManager.currentTimeMillis() - startTime; - if (tableName != null) { - - TableMetricsManager - .updateMetricsMethod(tableName, SELECT_SQL_COUNTER, 1); - TableMetricsManager - .updateMetricsMethod(tableName, SELECT_SQL_QUERY_TIME, - executeQueryTimeSpent); - if (success) { - TableMetricsManager.updateMetricsMethod(tableName, - SELECT_SUCCESS_SQL_COUNTER, 1); - TableMetricsManager.updateMetricsMethod(tableName, - pointLookup ? - SELECT_POINTLOOKUP_SUCCESS_SQL_COUNTER : - SELECT_SCAN_SUCCESS_SQL_COUNTER, 1); - } else { - TableMetricsManager.updateMetricsMethod(tableName, - SELECT_FAILED_SQL_COUNTER, 1); - TableMetricsManager.updateMetricsMethod(tableName, - SELECT_AGGREGATE_FAILURE_SQL_COUNTER, 1); - TableMetricsManager.updateMetricsMethod(tableName, - pointLookup ? - SELECT_POINTLOOKUP_FAILED_SQL_COUNTER : - SELECT_SCAN_FAILED_SQL_COUNTER, 1); + if (updateMetrics) { + // Regardless of whether the query was successfully handled or not, + // update the time spent so far. If needed, we can separate out the + // success times and failure times. + GLOBAL_QUERY_TIME.update(EnvironmentEdgeManager.currentTimeMillis() + - startTime); + long + executeQueryTimeSpent = + EnvironmentEdgeManager.currentTimeMillis() - startTime; + if (tableName != null) { + + TableMetricsManager + .updateMetricsMethod(tableName, SELECT_SQL_COUNTER, 1); + TableMetricsManager + .updateMetricsMethod(tableName, SELECT_SQL_QUERY_TIME, + executeQueryTimeSpent); + if (success) { + TableMetricsManager.updateMetricsMethod(tableName, + SELECT_SUCCESS_SQL_COUNTER, 1); + TableMetricsManager.updateMetricsMethod(tableName, + pointLookup ? + SELECT_POINTLOOKUP_SUCCESS_SQL_COUNTER : + SELECT_SCAN_SUCCESS_SQL_COUNTER, 1); + } else { + TableMetricsManager.updateMetricsMethod(tableName, + SELECT_FAILED_SQL_COUNTER, 1); + TableMetricsManager.updateMetricsMethod(tableName, + SELECT_AGGREGATE_FAILURE_SQL_COUNTER, 1); + TableMetricsManager.updateMetricsMethod(tableName, + pointLookup ? + SELECT_POINTLOOKUP_FAILED_SQL_COUNTER : + SELECT_SCAN_FAILED_SQL_COUNTER, 1); + } + } + if (rs != null) { + rs.setQueryTime(executeQueryTimeSpent); } - } - if (rs != null) { - rs.setQueryTime(executeQueryTimeSpent); } } return rs; @@ -830,6 +884,19 @@ public int getBindCount() { public QueryPlan compilePlan(PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException { CompilableStatement compilableStmt = getStatement(); StatementPlan compilePlan = compilableStmt.compilePlan(stmt, Sequence.ValueOp.VALIDATE_SEQUENCE); + // if client is validating timestamps, ensure its metadata cache is up to date. + if (ValidateLastDDLTimestampUtil + .getValidateLastDdlTimestampEnabled(stmt.getConnection())) { + Set tableRefs = compilePlan.getSourceRefs(); + for (TableRef tableRef : tableRefs) { + new MetaDataClient(stmt.getConnection()).updateCache( + stmt.getConnection().getTenantId(), + tableRef.getTable().getSchemaName().getString(), + tableRef.getTable().getTableName().getString(), + true); + } + compilePlan = compilableStmt.compilePlan(stmt, Sequence.ValueOp.VALIDATE_SEQUENCE); + } // For a QueryPlan, we need to get its optimized plan; for a MutationPlan, its enclosed QueryPlan // has already been optimized during compilation. if (compilePlan instanceof QueryPlan) { diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java b/phoenix-core-client/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java index 70529dc36cf..8763a7c5ecf 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java @@ -39,6 +39,7 @@ import static org.apache.phoenix.monitoring.MetricType.SELECT_SQL_COUNTER; import static org.apache.phoenix.monitoring.MetricType.SPOOL_FILE_COUNTER; import static org.apache.phoenix.monitoring.MetricType.SPOOL_FILE_SIZE; +import static org.apache.phoenix.monitoring.MetricType.STALE_METADATA_CACHE_EXCEPTION_COUNTER; import static org.apache.phoenix.monitoring.MetricType.TASK_END_TO_END_TIME; import static org.apache.phoenix.monitoring.MetricType.TASK_EXECUTED_COUNTER; import static org.apache.phoenix.monitoring.MetricType.TASK_EXECUTION_TIME; @@ -159,7 +160,8 @@ public enum GlobalClientMetrics { GLOBAL_HA_PARALLEL_CONNECTION_CREATED_COUNTER(HA_PARALLEL_CONNECTION_CREATED_COUNTER), GLOBAL_CLIENT_METADATA_CACHE_MISS_COUNTER(CLIENT_METADATA_CACHE_MISS_COUNTER), - GLOBAL_CLIENT_METADATA_CACHE_HIT_COUNTER(CLIENT_METADATA_CACHE_HIT_COUNTER); + GLOBAL_CLIENT_METADATA_CACHE_HIT_COUNTER(CLIENT_METADATA_CACHE_HIT_COUNTER), + GLOBAL_CLIENT_STALE_METADATA_CACHE_EXCEPTION_COUNTER(STALE_METADATA_CACHE_EXCEPTION_COUNTER); private static final Logger LOGGER = LoggerFactory.getLogger(GlobalClientMetrics.class); private static final boolean isGlobalMetricsEnabled = QueryServicesOptions.withDefaults().isGlobalMetricsEnabled(); diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/monitoring/MetricType.java b/phoenix-core-client/src/main/java/org/apache/phoenix/monitoring/MetricType.java index acdba9e551b..b7b0c4b562e 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/monitoring/MetricType.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/monitoring/MetricType.java @@ -150,6 +150,9 @@ public enum MetricType { CLIENT_METADATA_CACHE_MISS_COUNTER("cmcm", "Number of cache misses for the CQSI cache.", LogLevel.DEBUG, PLong.INSTANCE), CLIENT_METADATA_CACHE_HIT_COUNTER("cmch", "Number of cache hits for the CQSI cache.", LogLevel.DEBUG, PLong.INSTANCE), PAGED_ROWS_COUNTER("prc", "Number of dummy rows returned to client due to paging.", LogLevel.DEBUG, PLong.INSTANCE), + STALE_METADATA_CACHE_EXCEPTION_COUNTER("smce", + "Number of StaleMetadataCacheException encountered.", + LogLevel.DEBUG, PLong.INSTANCE), // hbase metrics COUNT_RPC_CALLS("rp", "Number of RPC calls",LogLevel.DEBUG, PLong.INSTANCE), diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java index 95ea0bab424..528534e0cb3 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java @@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Mutation; @@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Pair; import org.apache.phoenix.compile.MutationPlan; +import org.apache.phoenix.coprocessorclient.InvalidateServerMetadataCacheRequest; import org.apache.phoenix.coprocessorclient.MetaDataProtocol.MetaDataMutationResult; import org.apache.phoenix.exception.SQLExceptionInfo; import org.apache.phoenix.execute.MutationState; @@ -196,6 +198,8 @@ public MetaDataMutationResult dropColumn(List tableMetadata, public int getLowestClusterHBaseVersion(); public Admin getAdmin() throws SQLException; + void refreshLiveRegionServers() throws SQLException; + List getLiveRegionServers(); void clearTableRegionCache(TableName name) throws SQLException; @@ -285,4 +289,7 @@ public default ConnectionLimiter getConnectionLimiter() { } int getConnectionCount(boolean isInternal); + + void invalidateServerMetadataCache(List requests) + throws Throwable; } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java index 557d1e917d8..b7e1e84f97e 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java @@ -21,6 +21,7 @@ import static org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder.MAX_VERSIONS; import static org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder.REPLICATION_SCOPE; import static org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder.TTL; +import static org.apache.hadoop.hbase.ipc.RpcControllerFactory.CUSTOM_CONTROLLER_CONF_KEY; import static org.apache.phoenix.coprocessorclient.MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP; import static org.apache.phoenix.coprocessorclient.MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_15_0; import static org.apache.phoenix.coprocessorclient.MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_16_0; @@ -74,6 +75,7 @@ import static org.apache.phoenix.monitoring.MetricType.TIME_SPENT_IN_SYSTEM_TABLE_RPC_CALLS; import static org.apache.phoenix.query.QueryConstants.DEFAULT_COLUMN_FAMILY; import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADATA; +import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_PHOENIX_METADATA_INVALIDATE_CACHE_ENABLED; import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_ENABLED; import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_THREAD_POOL_SIZE; import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_THRESHOLD_MILLISECONDS; @@ -110,11 +112,15 @@ import java.util.Random; import java.util.Set; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadFactory; @@ -128,6 +134,7 @@ import javax.annotation.concurrent.GuardedBy; import com.google.protobuf.RpcController; +import com.google.protobuf.ServiceException; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HConstants; @@ -135,6 +142,7 @@ import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.NamespaceNotFoundException; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableExistsException; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; @@ -157,7 +165,9 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils.BlockingRpcCallback; import org.apache.hadoop.hbase.ipc.HBaseRpcController; +import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.ipc.ServerRpcController; +import org.apache.hadoop.hbase.ipc.controller.InvalidateMetadataCacheControllerFactory; import org.apache.hadoop.hbase.ipc.controller.ServerToServerRpcController; import org.apache.hadoop.hbase.ipc.controller.ServerSideRPCControllerFactory; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto; @@ -193,6 +203,10 @@ import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse; import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService; import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest; +import org.apache.phoenix.coprocessor.generated.RegionServerEndpointProtos; +import org.apache.phoenix.coprocessorclient.InvalidateServerMetadataCacheRequest; +import org.apache.phoenix.coprocessorclient.metrics.MetricsMetadataCachingSource; +import org.apache.phoenix.coprocessorclient.metrics.MetricsPhoenixCoprocessorSourceFactory; import org.apache.phoenix.coprocessorclient.MetaDataProtocol; import org.apache.phoenix.coprocessorclient.MetaDataProtocol.MetaDataMutationResult; import org.apache.phoenix.coprocessorclient.MetaDataProtocol.MutationCode; @@ -262,6 +276,7 @@ import org.apache.phoenix.schema.types.PUnsignedTinyint; import org.apache.phoenix.schema.types.PVarbinary; import org.apache.phoenix.schema.types.PVarchar; +import org.apache.phoenix.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.phoenix.transaction.PhoenixTransactionClient; import org.apache.phoenix.transaction.PhoenixTransactionContext; import org.apache.phoenix.transaction.PhoenixTransactionProvider; @@ -307,6 +322,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement private static final int DEFAULT_OUT_OF_ORDER_MUTATIONS_WAIT_TIME_MS = 1000; private static final String ALTER_TABLE_SET_PROPS = "ALTER TABLE %s SET %s=%s"; + private final GuidePostsCacheProvider + GUIDE_POSTS_CACHE_PROVIDER = new GuidePostsCacheProvider(); protected final Configuration config; public ConnectionInfo getConnectionInfo() { @@ -320,7 +337,7 @@ public ConnectionInfo getConnectionInfo() { private final String userName; private final User user; private final ConcurrentHashMap childServices; - private GuidePostsCacheWrapper tableStatsCache; + private final GuidePostsCacheWrapper tableStatsCache; // Cache the latest meta data here for future connections // writes guarded by "latestMetaDataLock" @@ -381,6 +398,16 @@ public ConnectionInfo getConnectionInfo() { private final ConnectionLimiter connectionLimiter; + // writes guarded by "liveRegionServersLock" + private volatile List liveRegionServers; + private final Object liveRegionServersLock = new Object(); + // Writes guarded by invalidateMetadataCacheConnLock + private Connection invalidateMetadataCacheConnection = null; + private final Object invalidateMetadataCacheConnLock = new Object(); + private MetricsMetadataCachingSource metricsMetadataCachingSource; + public static final String INVALIDATE_SERVER_METADATA_CACHE_EX_MESSAGE = + "Cannot invalidate server metadata cache on a non-server connection"; + private static interface FeatureSupported { boolean isSupported(ConnectionQueryServices services); } @@ -469,6 +496,10 @@ public ConnectionQueryServicesImpl(QueryServices services, ConnectionInfo connec } connectionQueues = ImmutableList.copyOf(list); + // A little bit of a smell to leak `this` here, but should not be a problem + this.tableStatsCache = GUIDE_POSTS_CACHE_PROVIDER.getGuidePostsCache(props.get(GUIDE_POSTS_CACHE_FACTORY_CLASS, + QueryServicesOptions.DEFAULT_GUIDE_POSTS_CACHE_FACTORY_CLASS), this, config); + this.isAutoUpgradeEnabled = config.getBoolean(AUTO_UPGRADE_ENABLED, QueryServicesOptions.DEFAULT_AUTO_UPGRADE_ENABLED); this.maxConnectionsAllowed = config.getInt(QueryServices.CLIENT_CONNECTION_MAX_ALLOWED_CONNECTIONS, QueryServicesOptions.DEFAULT_CLIENT_CONNECTION_MAX_ALLOWED_CONNECTIONS); @@ -508,35 +539,61 @@ public ConnectionQueryServicesImpl(QueryServices services, ConnectionInfo connec e.printStackTrace(); } } - + nSequenceSaltBuckets = config.getInt(QueryServices.SEQUENCE_SALT_BUCKETS_ATTRIB, + QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS); + this.metricsMetadataCachingSource = MetricsPhoenixCoprocessorSourceFactory.getInstance() + .getMetadataCachingSource(); } - private void openConnection() throws SQLException { + private Connection openConnection(Configuration conf) throws SQLException { + Connection localConnection; try { - this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config); + localConnection = HBaseFactoryProvider.getHConnectionFactory().createConnection(conf); GLOBAL_HCONNECTIONS_COUNTER.increment(); LOGGER.info("HConnection established. Stacktrace for informational purposes: " - + connection + " " + LogUtil.getCallerStackTrace()); + + localConnection + " " + LogUtil.getCallerStackTrace()); } catch (IOException e) { throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION) .setRootCause(e).build().buildException(); } - if (this.connection.isClosed()) { // TODO: why the heck doesn't this throw above? + if (localConnection.isClosed()) { // TODO: why the heck doesn't this throw above? throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION).build().buildException(); } + return localConnection; + } + + /** + * We create a long-lived hbase connection to run invalidate cache RPCs. We override + * CUSTOM_CONTROLLER_CONF_KEY to instantiate InvalidateMetadataCacheController which has + * a special priority for invalidate metadata cache operations. + * @return hbase connection + * @throws SQLException SQLException + */ + public Connection getInvalidateMetadataCacheConnection() throws SQLException { + if (invalidateMetadataCacheConnection != null) { + return invalidateMetadataCacheConnection; + } + + synchronized (invalidateMetadataCacheConnLock) { + Configuration clonedConfiguration = PropertiesUtil.cloneConfig(this.config); + clonedConfiguration.setClass(CUSTOM_CONTROLLER_CONF_KEY, + InvalidateMetadataCacheControllerFactory.class, RpcControllerFactory.class); + invalidateMetadataCacheConnection = openConnection(clonedConfiguration); + } + return invalidateMetadataCacheConnection; } /** * Close the HBase connection and decrement the counter. * @throws IOException throws IOException */ - private void closeConnection() throws IOException { + private void closeConnection(Connection connection) throws IOException { if (connection != null) { connection.close(); LOGGER.info("{} HConnection closed. Stacktrace for informational" + " purposes: {}", connection, LogUtil.getCallerStackTrace()); + GLOBAL_HCONNECTIONS_COUNTER.decrement(); } - GLOBAL_HCONNECTIONS_COUNTER.decrement(); } @Override @@ -645,8 +702,9 @@ public void close() throws SQLException { latestMetaDataLock.notifyAll(); } try { - // close the HBase connection - closeConnection(); + // close HBase connections. + closeConnection(this.connection); + closeConnection(this.invalidateMetadataCacheConnection); } finally { if (renewLeaseExecutor != null) { renewLeaseExecutor.shutdownNow(); @@ -3621,13 +3679,15 @@ public Void call() throws Exception { try { GLOBAL_QUERY_SERVICES_COUNTER.increment(); LOGGER.info("An instance of ConnectionQueryServices was created."); - openConnection(); + connection = openConnection(config); hConnectionEstablished = true; - tableStatsCache = - (new GuidePostsCacheProvider()).getGuidePostsCache( - props.getProperty(GUIDE_POSTS_CACHE_FACTORY_CLASS, - QueryServicesOptions.DEFAULT_GUIDE_POSTS_CACHE_FACTORY_CLASS), - ConnectionQueryServicesImpl.this, config); + boolean lastDDLTimestampValidationEnabled + = getProps().getBoolean( + QueryServices.LAST_DDL_TIMESTAMP_VALIDATION_ENABLED, + QueryServicesOptions.DEFAULT_LAST_DDL_TIMESTAMP_VALIDATION_ENABLED); + if (lastDDLTimestampValidationEnabled) { + refreshLiveRegionServers(); + } String skipSystemExistenceCheck = props.getProperty(SKIP_SYSTEM_TABLES_EXISTENCE_CHECK); if (skipSystemExistenceCheck != null && @@ -3636,9 +3696,6 @@ public Void call() throws Exception { success = true; return null; } - nSequenceSaltBuckets = ConnectionQueryServicesImpl.this.props.getInt( - QueryServices.SEQUENCE_SALT_BUCKETS_ATTRIB, - QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS); boolean isDoNotUpgradePropSet = UpgradeUtil.isNoUpgradeSet(props); Properties scnProps = PropertiesUtil.deepCopy(props); scnProps.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, @@ -3729,7 +3786,8 @@ Collections.> singletonList( } try { if (!success && hConnectionEstablished) { - closeConnection(); + closeConnection(connection); + closeConnection(invalidateMetadataCacheConnection); } } catch (IOException e) { SQLException ex = new SQLException(e); @@ -4567,7 +4625,6 @@ public PhoenixConnection upgradeSystemSequence( } else { nSequenceSaltBuckets = getSaltBuckets(e); } - updateSystemSequenceWithCacheOnWriteProps(metaConnection); } return metaConnection; @@ -5359,6 +5416,23 @@ private void flushTable(byte[] tableName) throws SQLException { } } + @Override + public void refreshLiveRegionServers() throws SQLException { + synchronized (liveRegionServersLock) { + try (Admin admin = getAdmin()) { + this.liveRegionServers = new ArrayList<>(admin.getRegionServers(true)); + } catch (IOException e) { + throw ClientUtil.parseServerException(e); + } + } + LOGGER.info("Refreshed list of live region servers."); + } + + @Override + public List getLiveRegionServers() { + return this.liveRegionServers; + } + @Override public Admin getAdmin() throws SQLException { try { @@ -6323,4 +6397,176 @@ public synchronized PhoenixTransactionClient initTransactionClient(Provider prov public List>> getCachedConnections() { return connectionQueues; } + + /** + * Invalidate metadata cache from all region servers for the given list of + * InvalidateServerMetadataCacheRequest. + * @throws Throwable + */ + public void invalidateServerMetadataCache(List requests) + throws Throwable { + boolean invalidateCacheEnabled = + config.getBoolean(PHOENIX_METADATA_INVALIDATE_CACHE_ENABLED, + DEFAULT_PHOENIX_METADATA_INVALIDATE_CACHE_ENABLED); + if (!invalidateCacheEnabled) { + LOGGER.info("Skip invalidating server metadata cache since conf property" + + " phoenix.metadata.invalidate.cache.enabled is set to false"); + return; + } + if (!QueryUtil.isServerConnection(props)) { + LOGGER.warn(INVALIDATE_SERVER_METADATA_CACHE_EX_MESSAGE); + throw new Exception(INVALIDATE_SERVER_METADATA_CACHE_EX_MESSAGE); + } + + metricsMetadataCachingSource.incrementMetadataCacheInvalidationOperationsCount(); + Admin admin = getInvalidateMetadataCacheConnection().getAdmin(); + // This will incur an extra RPC to the master. This RPC is required since we want to + // get current list of regionservers. + Collection serverNames = admin.getRegionServers(true); + PhoenixStopWatch stopWatch = new PhoenixStopWatch().start(); + try { + invalidateServerMetadataCacheWithRetries(admin, serverNames, requests, false); + metricsMetadataCachingSource.incrementMetadataCacheInvalidationSuccessCount(); + } catch (Throwable t) { + metricsMetadataCachingSource.incrementMetadataCacheInvalidationFailureCount(); + throw t; + } finally { + metricsMetadataCachingSource + .addMetadataCacheInvalidationTotalTime(stopWatch.stop().elapsedMillis()); + } + } + + /** + * Invalidate metadata cache on all regionservers with retries for the given list of + * InvalidateServerMetadataCacheRequest. Each InvalidateServerMetadataCacheRequest contains + * tenantID, schema name and table name. + * We retry once before failing the operation. + * + * @param admin + * @param serverNames + * @param invalidateCacheRequests + * @param isRetry + * @throws Throwable + */ + private void invalidateServerMetadataCacheWithRetries(Admin admin, + Collection serverNames, + List invalidateCacheRequests, + boolean isRetry) throws Throwable { + RegionServerEndpointProtos.InvalidateServerMetadataCacheRequest protoRequest = + getRequest(invalidateCacheRequests); + List> futures = new ArrayList<>(); + Map map = new HashMap<>(); + int poolSize = config.getInt( + PHOENIX_METADATA_CACHE_INVALIDATION_THREAD_POOL_SIZE, + QueryServicesOptions.DEFAULT_PHOENIX_METADATA_CACHE_INVALIDATION_THREAD_POOL_SIZE); + ThreadFactoryBuilder builder = new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat("metadata-cache-invalidation-pool-%d"); + ExecutorService executor = Executors.newFixedThreadPool(poolSize, builder.build()); + for (ServerName serverName : serverNames) { + CompletableFuture future = CompletableFuture.runAsync(() -> { + try { + PhoenixStopWatch innerWatch = new PhoenixStopWatch().start(); + for (InvalidateServerMetadataCacheRequest invalidateCacheRequest + : invalidateCacheRequests) { + LOGGER.info("Sending invalidate metadata cache for {} to region server:" + + " {}", invalidateCacheRequest.toString(), serverName); + } + + RegionServerEndpointProtos.RegionServerEndpointService.BlockingInterface + service = RegionServerEndpointProtos.RegionServerEndpointService + .newBlockingStub(admin.coprocessorService(serverName)); + // The timeout for this particular request is managed by config parameter: + // hbase.rpc.timeout. Even if the future times out, this runnable can be in + // RUNNING state and will not be interrupted. + // We use the controller set in hbase connection. + service.invalidateServerMetadataCache(null, protoRequest); + long cacheInvalidationTime = innerWatch.stop().elapsedMillis(); + LOGGER.info("Invalidating metadata cache" + + " on region server: {} completed successfully and it took {} ms", + serverName, cacheInvalidationTime); + metricsMetadataCachingSource + .addMetadataCacheInvalidationRpcTime(cacheInvalidationTime); + } catch (ServiceException se) { + LOGGER.error("Invalidating metadata cache failed for regionserver {}", + serverName, se); + IOException ioe = ClientUtil.parseServiceException(se); + throw new CompletionException(ioe); + } + }, executor); + futures.add(future); + map.put(future, serverName); + } + // Here we create one master like future which tracks individual future + // for each region server. + CompletableFuture allFutures = CompletableFuture.allOf( + futures.toArray(new CompletableFuture[0])); + long metadataCacheInvalidationTimeoutMs = config.getLong( + PHOENIX_METADATA_CACHE_INVALIDATION_TIMEOUT_MS, + PHOENIX_METADATA_CACHE_INVALIDATION_TIMEOUT_MS_DEFAULT); + try { + allFutures.get(metadataCacheInvalidationTimeoutMs, TimeUnit.MILLISECONDS); + } catch (Throwable t) { + List failedServers = getFailedServers(futures, map); + LOGGER.error("Invalidating metadata cache for failed for region servers: {}", + failedServers, t); + if (isRetry) { + // If this is a retry attempt then just fail the operation. + if (allFutures.isCompletedExceptionally()) { + if (t instanceof ExecutionException) { + t = t.getCause(); + } + } + throw t; + } else { + // This is the first attempt, we can retry once. + // Indicate that this is a retry attempt. + invalidateServerMetadataCacheWithRetries(admin, failedServers, + invalidateCacheRequests, true); + } + } finally { + executor.shutdown(); + } + } + + /** + * Get the list of regionservers that failed the invalidateCache rpc. + * @param futures futtures + * @param map map of future to server names + * @return the list of servers that failed the invalidateCache RPC. + */ + private List getFailedServers(List> futures, + Map map) { + List failedServers = new ArrayList<>(); + for (CompletableFuture completedFuture : futures) { + if (!completedFuture.isDone()) { + // If this task is still running, cancel it and keep in retry list. + ServerName sn = map.get(completedFuture); + failedServers.add(sn); + // Even though we cancel this future but it doesn't interrupt the executing thread. + completedFuture.cancel(true); + } else if (completedFuture.isCompletedExceptionally() + || completedFuture.isCancelled()) { + // This means task is done but completed with exception + // or was canceled. Add it to retry list. + ServerName sn = map.get(completedFuture); + failedServers.add(sn); + } + } + return failedServers; + } + + private RegionServerEndpointProtos.InvalidateServerMetadataCacheRequest getRequest( + List requests) { + RegionServerEndpointProtos.InvalidateServerMetadataCacheRequest.Builder builder = + RegionServerEndpointProtos.InvalidateServerMetadataCacheRequest.newBuilder(); + for (InvalidateServerMetadataCacheRequest request: requests) { + RegionServerEndpointProtos.InvalidateServerMetadataCache.Builder innerBuilder + = RegionServerEndpointProtos.InvalidateServerMetadataCache.newBuilder(); + innerBuilder.setTenantId(ByteStringer.wrap(request.getTenantId())); + innerBuilder.setSchemaName(ByteStringer.wrap(request.getSchemaName())); + innerBuilder.setTableName(ByteStringer.wrap(request.getTableName())); + builder.addInvalidateServerMetadataCacheRequests(innerBuilder.build()); + } + return builder.build(); + } } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java index f0e5277090a..6b39eeb50b9 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java @@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.phoenix.compile.MutationPlan; +import org.apache.phoenix.coprocessorclient.InvalidateServerMetadataCacheRequest; import org.apache.phoenix.coprocessorclient.MetaDataProtocol; import org.apache.phoenix.coprocessorclient.MetaDataProtocol.MetaDataMutationResult; import org.apache.phoenix.coprocessorclient.MetaDataProtocol.MutationCode; @@ -110,6 +111,8 @@ */ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices implements ConnectionQueryServices { private static ServerName SERVER_NAME = ServerName.parseServerName(HConstants.LOCALHOST + Addressing.HOSTNAME_PORT_SEPARATOR + HConstants.DEFAULT_ZOOKEEPER_CLIENT_PORT); + private static final GuidePostsCacheProvider + GUIDE_POSTS_CACHE_PROVIDER = new GuidePostsCacheProvider(); private final ReadOnlyProps props; private PMetaData metaData; private final Map sequenceMap = Maps.newHashMap(); @@ -118,7 +121,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple private volatile boolean initialized; private volatile SQLException initializationException; private final Map> tableSplits = Maps.newHashMap(); - private GuidePostsCacheWrapper guidePostsCache; + private final GuidePostsCacheWrapper guidePostsCache; private final Configuration config; private User user; @@ -151,6 +154,9 @@ public ConnectionlessQueryServicesImpl(QueryServices services, ConnectionInfo co // set replication required parameter ConfigUtil.setReplicationConfigIfAbsent(this.config); this.props = new ReadOnlyProps(this.config.iterator()); + + this.guidePostsCache = GUIDE_POSTS_CACHE_PROVIDER.getGuidePostsCache(props.get(GUIDE_POSTS_CACHE_FACTORY_CLASS, + QueryServicesOptions.DEFAULT_GUIDE_POSTS_CACHE_FACTORY_CLASS), null, config); } private PMetaData newEmptyMetaData() { @@ -407,11 +413,6 @@ public void init(String url, Properties props) throws SQLException { } return; } - guidePostsCache = - (new GuidePostsCacheProvider()).getGuidePostsCache( - props.getProperty(GUIDE_POSTS_CACHE_FACTORY_CLASS, - QueryServicesOptions.DEFAULT_GUIDE_POSTS_CACHE_FACTORY_CLASS), - null, config); SQLException sqlE = null; PhoenixConnection metaConnection = null; try { @@ -506,6 +507,16 @@ public int getLowestClusterHBaseVersion() { return Integer.MAX_VALUE; // Allow everything for connectionless } + @Override + public void refreshLiveRegionServers() throws SQLException { + throw new UnsupportedOperationException(); + } + + @Override + public List getLiveRegionServers() { + throw new UnsupportedOperationException(); + } + @Override public Admin getAdmin() throws SQLException { throw new UnsupportedOperationException(); @@ -849,4 +860,10 @@ public PMetaData getMetaDataCache() { public int getConnectionCount(boolean isInternal) { return 0; } + + @Override + public void invalidateServerMetadataCache(List requests) + throws Throwable { + // No-op + } } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java index 9945896dd74..07ff39a781f 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java @@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Mutation; @@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Pair; import org.apache.phoenix.compile.MutationPlan; +import org.apache.phoenix.coprocessorclient.InvalidateServerMetadataCacheRequest; import org.apache.phoenix.coprocessorclient.MetaDataProtocol.MetaDataMutationResult; import org.apache.phoenix.execute.MutationState; import org.apache.phoenix.hbase.index.util.KeyValueBuilder; @@ -211,6 +213,16 @@ public int getLowestClusterHBaseVersion() { return getDelegate().getLowestClusterHBaseVersion(); } + @Override + public void refreshLiveRegionServers() throws SQLException { + getDelegate().refreshLiveRegionServers(); + } + + @Override + public List getLiveRegionServers() { + return getDelegate().getLiveRegionServers(); + } + @Override public Admin getAdmin() throws SQLException { return getDelegate().getAdmin(); @@ -452,4 +464,10 @@ public ConnectionLimiter getConnectionLimiter() { public int getConnectionCount(boolean isInternal) { return getDelegate().getConnectionCount(isInternal); } + + @Override + public void invalidateServerMetadataCache(List requests) + throws Throwable { + getDelegate().invalidateServerMetadataCache(requests); + } } \ No newline at end of file diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java index 0279377d5ae..df7f669c6c0 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServices.java @@ -186,6 +186,9 @@ public interface QueryServices extends SQLCloseable { public static final String INDEX_PRIOIRTY_ATTRIB = "phoenix.index.rpc.priority"; public static final String METADATA_PRIOIRTY_ATTRIB = "phoenix.metadata.rpc.priority"; public static final String SERVER_SIDE_PRIOIRTY_ATTRIB = "phoenix.serverside.rpc.priority"; + String INVALIDATE_METADATA_CACHE_PRIORITY_ATTRIB = + "phoenix.invalidate.metadata.cache.rpc.priority"; + public static final String ALLOW_LOCAL_INDEX_ATTRIB = "phoenix.index.allowLocalIndex"; // Retries when doing server side writes to SYSTEM.CATALOG @@ -251,6 +254,7 @@ public interface QueryServices extends SQLCloseable { public static final String INDEX_HANDLER_COUNT_ATTRIB = "phoenix.rpc.index.handler.count"; public static final String METADATA_HANDLER_COUNT_ATTRIB = "phoenix.rpc.metadata.handler.count"; public static final String SERVER_SIDE_HANDLER_COUNT_ATTRIB = "phoenix.rpc.serverside.handler.count"; + String INVALIDATE_CACHE_HANDLER_COUNT_ATTRIB = "phoenix.rpc.invalidate.cache.handler.count"; public static final String FORCE_ROW_KEY_ORDER_ATTRIB = "phoenix.query.force.rowkeyorder"; public static final String ALLOW_USER_DEFINED_FUNCTIONS_ATTRIB = "phoenix.functions.allowUserDefinedFunctions"; @@ -322,6 +326,13 @@ public interface QueryServices extends SQLCloseable { //Update Cache Frequency default config attribute public static final String DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB = "phoenix.default.update.cache.frequency"; + //Update Cache Frequency for indexes in PENDING_DISABLE state + public static final String UPDATE_CACHE_FREQUENCY_FOR_PENDING_DISABLED_INDEX + = "phoenix.update.cache.frequency.pending.disable.index"; + + // whether to validate last ddl timestamps during client operations + public static final String LAST_DDL_TIMESTAMP_VALIDATION_ENABLED = "phoenix.ddl.timestamp.validation.enabled"; + // Whether to enable cost-based-decision in the query optimizer public static final String COST_BASED_OPTIMIZER_ENABLED = "phoenix.costbased.optimizer.enabled"; public static final String SMALL_SCAN_THRESHOLD_ATTRIB = "phoenix.query.smallScanThreshold"; @@ -438,7 +449,14 @@ public interface QueryServices extends SQLCloseable { * Parameter to disable the server merges for hinted uncovered indexes */ String SERVER_MERGE_FOR_UNCOVERED_INDEX = "phoenix.query.global.server.merge.enable"; - + String PHOENIX_METADATA_CACHE_INVALIDATION_TIMEOUT_MS = + "phoenix.metadata.cache.invalidation.timeoutMs"; + // Default to 10 seconds. + long PHOENIX_METADATA_CACHE_INVALIDATION_TIMEOUT_MS_DEFAULT = 10 * 1000; + String PHOENIX_METADATA_INVALIDATE_CACHE_ENABLED = "phoenix.metadata.invalidate.cache.enabled"; + + String PHOENIX_METADATA_CACHE_INVALIDATION_THREAD_POOL_SIZE = + "phoenix.metadata.cache.invalidation.threadPool.size"; /** * Param to determine whether client can disable validation to figure out if any of the * descendent views extend primary key of their parents. Since this is a bit of diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java index 3dac05b8cda..6dd7a620b70 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java @@ -119,6 +119,7 @@ import java.util.Map.Entry; import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.phoenix.schema.ConnectionProperty; import org.apache.phoenix.schema.PIndexState; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Coprocessor; @@ -239,10 +240,12 @@ public class QueryServicesOptions { public static final int DEFAULT_SERVER_SIDE_PRIORITY = 500; public static final int DEFAULT_INDEX_PRIORITY = 1000; public static final int DEFAULT_METADATA_PRIORITY = 2000; + public static final int DEFAULT_INVALIDATE_METADATA_CACHE_PRIORITY = 3000; public static final boolean DEFAULT_ALLOW_LOCAL_INDEX = true; public static final int DEFAULT_INDEX_HANDLER_COUNT = 30; public static final int DEFAULT_METADATA_HANDLER_COUNT = 30; public static final int DEFAULT_SERVERSIDE_HANDLER_COUNT = 30; + public static final int DEFAULT_INVALIDATE_CACHE_HANDLER_COUNT = 10; public static final int DEFAULT_SYSTEM_MAX_VERSIONS = 1; public static final boolean DEFAULT_SYSTEM_KEEP_DELETED_CELLS = false; @@ -368,10 +371,27 @@ public class QueryServicesOptions { //Security defaults public static final boolean DEFAULT_PHOENIX_ACLS_ENABLED = false; - //default update cache frequency - public static final long DEFAULT_UPDATE_CACHE_FREQUENCY = 0; public static final int DEFAULT_SMALL_SCAN_THRESHOLD = 100; + /** + * Metadata caching configs, see https://issues.apache.org/jira/browse/PHOENIX-6883. + * Disable the boolean flags and set UCF=always to disable the caching re-design. + * + * Disable caching re-design if you use Online Data Format Change since the cutover logic + * is currently incompatible and clients may not learn about the physical table change. + * See https://issues.apache.org/jira/browse/PHOENIX-7284. + * + * Disable caching re-design if your clients will not have ADMIN perms to call region server + * RPC. See https://issues.apache.org/jira/browse/HBASE-28508 + */ + public static final long DEFAULT_UPDATE_CACHE_FREQUENCY + = (long) ConnectionProperty.UPDATE_CACHE_FREQUENCY.getValue("ALWAYS"); + public static final boolean DEFAULT_LAST_DDL_TIMESTAMP_VALIDATION_ENABLED = false; + public static final boolean DEFAULT_PHOENIX_METADATA_INVALIDATE_CACHE_ENABLED = false; + public static final String DEFAULT_UPDATE_CACHE_FREQUENCY_FOR_PENDING_DISABLED_INDEX + = Long.toString(0L); + public static final int DEFAULT_PHOENIX_METADATA_CACHE_INVALIDATION_THREAD_POOL_SIZE = 20; + // default system task handling interval in milliseconds public static final long DEFAULT_TASK_HANDLING_INTERVAL_MS = 60*1000; // 1 min public static final long DEFAULT_TASK_HANDLING_MAX_INTERVAL_MS = 30*60*1000; // 30 min diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/DelegateTable.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/DelegateTable.java index 39c4bd5e381..02efcc3ffb7 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/DelegateTable.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/DelegateTable.java @@ -423,6 +423,11 @@ public String getIndexWhere() { return delegate.getIndexWhere(); } + @Override + public Map getAncestorLastDDLTimestampMap() { + return delegate.getAncestorLastDDLTimestampMap(); + } + @Override public Expression getIndexWhereExpression(PhoenixConnection connection) throws SQLException { diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/MetaDataClient.java index 02f377f4517..16546a3d1d5 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/MetaDataClient.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/MetaDataClient.java @@ -207,6 +207,7 @@ import org.apache.phoenix.util.ClientUtil; import org.apache.phoenix.util.CDCUtil; import org.apache.phoenix.util.TaskMetaDataServiceCallBack; +import org.apache.phoenix.util.ValidateLastDDLTimestampUtil; import org.apache.phoenix.util.ViewUtil; import org.apache.phoenix.util.JacksonUtil; import org.apache.phoenix.exception.SQLExceptionCode; @@ -697,7 +698,7 @@ public MetaDataMutationResult updateCache(PName origTenantId, String schemaName, result.setTable(table); } if (result.getTable()!=null) { - addTableToCache(result); + addTableToCache(result, alwaysHitServer); } return result; } @@ -711,7 +712,7 @@ public MetaDataMutationResult updateCache(PName origTenantId, String schemaName, // Otherwise, a tenant would be required to create a VIEW first // which is not really necessary unless you want to filter or add // columns - addTableToCache(result); + addTableToCache(result, alwaysHitServer); return result; } else { // if (result.getMutationCode() == MutationCode.NEWER_TABLE_FOUND) { @@ -727,7 +728,9 @@ public MetaDataMutationResult updateCache(PName origTenantId, String schemaName, // In this case, we update the parent table which may in turn pull // in indexes to add to this table. long resolvedTime = TransactionUtil.getResolvedTime(connection, result); - if (addColumnsAndIndexesFromAncestors(result, resolvedTimestamp, true)) { + if (addColumnsIndexesAndLastDDLTimestampsFromAncestors(result, + resolvedTimestamp, true, false)) { + updateIndexesWithAncestorMap(result); connection.addTable(result.getTable(), resolvedTime); } else { // if we aren't adding the table, we still need to update the @@ -777,7 +780,18 @@ private boolean avoidRpcToGetTable(boolean alwaysHitServer, Long resolvedTimesta // What if the table is created with UPDATE_CACHE_FREQUENCY explicitly set to ALWAYS? // i.e. explicitly set to 0. We should ideally be checking for something like // hasUpdateCacheFrequency(). - if (table.getUpdateCacheFrequency() != 0L) { + + //always fetch an Index in PENDING_DISABLE state to retrieve server timestamp + //QueryOptimizer needs that to decide whether the index can be used + if (PIndexState.PENDING_DISABLE.equals(table.getIndexState())) { + effectiveUpdateCacheFreq = + (Long) ConnectionProperty.UPDATE_CACHE_FREQUENCY.getValue( + connection.getQueryServices().getProps().get( + QueryServices.UPDATE_CACHE_FREQUENCY_FOR_PENDING_DISABLED_INDEX, + QueryServicesOptions.DEFAULT_UPDATE_CACHE_FREQUENCY_FOR_PENDING_DISABLED_INDEX)); + ucfInfoForLogging = "pending-disable-index-level"; + } else if (table.getUpdateCacheFrequency() + != QueryServicesOptions.DEFAULT_UPDATE_CACHE_FREQUENCY) { effectiveUpdateCacheFreq = table.getUpdateCacheFrequency(); ucfInfoForLogging = "table-level"; } else { @@ -903,15 +917,19 @@ public MetaDataMutationResult updateCache(PName tenantId, List functionN * @param resolvedTimestamp timestamp at which child table was resolved * @param alwaysAddAncestorColumnsAndIndexes flag that determines whether we should recalculate * all inherited columns and indexes that can be used in the view and + * @param alwaysHitServerForAncestors flag that determines whether we should fetch latest + * metadata for ancestors from the server * @return true if the PTable contained by result was modified and false otherwise * @throws SQLException if the physical table cannot be found */ - private boolean addColumnsAndIndexesFromAncestors(MetaDataMutationResult result, Long resolvedTimestamp, - boolean alwaysAddAncestorColumnsAndIndexes) throws SQLException { + private boolean addColumnsIndexesAndLastDDLTimestampsFromAncestors( + MetaDataMutationResult result, Long resolvedTimestamp, + boolean alwaysAddAncestorColumnsAndIndexes, + boolean alwaysHitServerForAncestors) + throws SQLException { PTable table = result.getTable(); boolean hasIndexId = table.getViewIndexId() != null; - // only need to inherit columns and indexes for view indexes and views - if ((table.getType()==PTableType.INDEX && hasIndexId) + if (table.getType() == PTableType.INDEX || (table.getType() == PTableType.VIEW && table.getViewType() != ViewType.MAPPED)) { String tableName = null; try { @@ -919,7 +937,7 @@ private boolean addColumnsAndIndexesFromAncestors(MetaDataMutationResult result, String parentSchemaName = SchemaUtil.getSchemaNameFromFullName(parentName); tableName = SchemaUtil.getTableNameFromFullName(parentName); MetaDataMutationResult parentResult = updateCache(connection.getTenantId(), parentSchemaName, tableName, - false, resolvedTimestamp); + alwaysHitServerForAncestors, resolvedTimestamp); PTable parentTable = parentResult.getTable(); if (parentResult.getMutationCode() == MutationCode.TABLE_NOT_FOUND || parentTable == null) { // Try once more with different tenant id (connection can be global but view could be tenant @@ -946,8 +964,18 @@ private boolean addColumnsAndIndexesFromAncestors(MetaDataMutationResult result, if (!alwaysAddAncestorColumnsAndIndexes && !result.wasUpdated() && !parentResult.wasUpdated()) { return false; } - result.setTable(ViewUtil.addDerivedColumnsAndIndexesFromParent( - connection, table, parentTable)); + + // only need to inherit columns and indexes for view indexes and views + if (!table.getType().equals(PTableType.INDEX) || hasIndexId) { + PTable pTableWithDerivedColumnsAndIndexes + = ViewUtil.addDerivedColumnsAndIndexesFromParent(connection, + table, parentTable); + result.setTable(getPTableWithAncestorLastDDLTimestampMap( + pTableWithDerivedColumnsAndIndexes, parentTable)); + } else { + result.setTable(getPTableWithAncestorLastDDLTimestampMap( + table, parentTable)); + } return true; } catch (Throwable e) { TableMetricsManager.updateMetricsForSystemCatalogTableMethod(tableName, NUM_METADATA_LOOKUP_FAILURES, 1); @@ -957,6 +985,42 @@ private boolean addColumnsAndIndexesFromAncestors(MetaDataMutationResult result, return false; } + /** + * Update the indexes within this result's table with ancestor->last_ddl_timestamp map. + */ + private void updateIndexesWithAncestorMap(MetaDataMutationResult result) throws SQLException { + PTable table = result.getTable(); + if (table.getIndexes().isEmpty()) { + return; + } + List newIndexes = new ArrayList<>(table.getIndexes().size()); + for (PTable index : table.getIndexes()) { + newIndexes.add(getPTableWithAncestorLastDDLTimestampMap(index, table)); + } + result.setTable(PTableImpl.builderWithColumns(table, PTableImpl.getColumnsToClone(table)) + .setIndexes(newIndexes).build()); + } + + /** + * Creates a new PTable object from the provided pTable and with the ancestorLastDDLTimestampMap + * Copy the map of the parent and add the last_ddl_timestamp of the parent in the map. + * @param pTable + * @param parentTable + */ + private PTable getPTableWithAncestorLastDDLTimestampMap(PTable pTable, PTable parentTable) + throws SQLException { + Map ancestorMap + = new HashMap<>(parentTable.getAncestorLastDDLTimestampMap()); + // this method can be called for an index and a view which inherited this index + // from its ancestors, skip adding the view as an ancestor of the index. + if (pTable.getParentName().equals(parentTable.getName())) { + ancestorMap.put(parentTable.getKey(), parentTable.getLastDDLTimestamp()); + } + return PTableImpl.builderWithColumns(pTable, PTableImpl.getColumnsToClone(pTable)) + .setAncestorLastDDLTimestampMap(ancestorMap) + .build(); + } + private void addFunctionArgMutation(String functionName, FunctionArgument arg, PreparedStatement argUpsert, int position) throws SQLException { argUpsert.setString(1, connection.getTenantId() == null ? null : connection.getTenantId().getString()); argUpsert.setString(2, functionName); @@ -1465,14 +1529,17 @@ public MutationState createIndex(CreateIndexStatement statement, byte[][] splits Set acquiredColumnMutexSet = Sets.newHashSetWithExpectedSize(3); String physicalSchemaName = null; String physicalTableName = null; + PTable dataTable = null; try { - ColumnResolver resolver = FromCompiler.getResolver(statement, connection, statement.getUdfParseNodes()); + ColumnResolver resolver + = FromCompiler.getResolverForCreateIndex( + statement, connection, statement.getUdfParseNodes()); tableRef = resolver.getTables().get(0); Date asyncCreatedDate = null; if (statement.isAsync()) { - asyncCreatedDate = new Date(tableRef.getTimeStamp()); + asyncCreatedDate = new Date(tableRef.getCurrentTime()); } - PTable dataTable = tableRef.getTable(); + dataTable = tableRef.getTable(); boolean isTenantConnection = connection.getTenantId() != null; if (isTenantConnection) { if (dataTable.getType() != PTableType.VIEW) { @@ -1730,7 +1797,14 @@ public MutationState createIndex(CreateIndexStatement statement, byte[][] splits return buildIndexAtTimeStamp(table, statement.getTable()); } - return buildIndex(table, tableRef); + MutationState state = buildIndex(table, tableRef); + // If client is validating LAST_DDL_TIMESTAMPS, parent's last_ddl_timestamp changed + // so remove it from client's cache. It will be refreshed when table is accessed next time. + if (ValidateLastDDLTimestampUtil.getValidateLastDdlTimestampEnabled(connection)) { + connection.removeTable(connection.getTenantId(), dataTable.getName().getString(), + null, dataTable.getTimeStamp()); + } + return state; } public MutationState createCDC(CreateCDCStatement statement) throws SQLException { @@ -3020,6 +3094,7 @@ else if (!SchemaUtil.isSystemTable(Bytes.toBytes(SchemaUtil.getTableName(schemaN .setColumns(columns.values()) .setPhoenixTTL(PHOENIX_TTL_NOT_DEFINED) .setPhoenixTTLHighWaterMark(MIN_PHOENIX_TTL_HWM) + .setLastDDLTimestamp(0L) .setIndexWhere(statement.getWhereClause() == null ? null : statement.getWhereClause().toString()) .build(); @@ -3452,7 +3527,7 @@ public boolean isViewReferenced() { .setCDCIncludeScopes(cdcIncludeScopes) .build(); result = new MetaDataMutationResult(code, result.getMutationTime(), table, true); - addTableToCache(result); + addTableToCache(result, false); return table; } catch (Throwable e) { TableMetricsManager.updateMetricsForSystemCatalogTableMethod(tableNameNode.toString(), @@ -3542,7 +3617,7 @@ public boolean handleCreateTableMutationCode(MetaDataMutationResult result, Muta switch(code) { case TABLE_ALREADY_EXISTS: if (result.getTable() != null) { - addTableToCache(result); + addTableToCache(result, false); } if (!statement.ifNotExists()) { throw new TableAlreadyExistsException(schemaName, tableName, result.getTable()); @@ -3559,7 +3634,7 @@ public boolean handleCreateTableMutationCode(MetaDataMutationResult result, Muta case UNALLOWED_TABLE_MUTATION: throwsSQLExceptionUtil("CANNOT_MUTATE_TABLE",schemaName,tableName); case CONCURRENT_TABLE_MUTATION: - addTableToCache(result); + addTableToCache(result, false); throw new ConcurrentTableMutationException(schemaName, tableName); case AUTO_PARTITION_SEQUENCE_NOT_FOUND: throw new SQLExceptionInfo.Builder(SQLExceptionCode.AUTO_PARTITION_SEQUENCE_UNDEFINED) @@ -3881,7 +3956,7 @@ private MutationCode processMutationResult(String schemaName, String tableName, case COLUMN_NOT_FOUND: break; case CONCURRENT_TABLE_MUTATION: - addTableToCache(result); + addTableToCache(result, false); if (LOGGER.isDebugEnabled()) { LOGGER.debug(LogUtil.addCustomAnnotations("CONCURRENT_TABLE_MUTATION for table " + SchemaUtil.getTableName(schemaName, tableName), connection)); } @@ -4524,7 +4599,7 @@ public MutationState addColumn(PTable table, List origColumnDefs, try { MutationCode code = processMutationResult(schemaName, tableName, result); if (code == MutationCode.COLUMN_ALREADY_EXISTS) { - addTableToCache(result); + addTableToCache(result, false); if (!ifNotExists) { throw new ColumnAlreadyExistsException(schemaName, tableName, SchemaUtil.findExistingColumn(result.getTable(), columns)); } @@ -4536,7 +4611,7 @@ public MutationState addColumn(PTable table, List origColumnDefs, String fullTableName = SchemaUtil.getTableName(schemaName, tableName); long resolvedTimeStamp = TransactionUtil.getResolvedTime(connection, result); if (table.getIndexes().isEmpty() || (numPkColumnsAdded==0 && ! metaProperties.getNonTxToTx())) { - addTableToCache(result, resolvedTimeStamp); + addTableToCache(result, false, resolvedTimeStamp); table = result.getTable(); } else { // remove the table from the cache, it will be fetched from the server the @@ -4863,12 +4938,16 @@ else if (columnToDrop.isViewReferenced()) { .setColumnName(columnToDrop.getName().getString()).build().buildException(); } columnsToDrop.add(new ColumnRef(columnRef.getTableRef(), columnToDrop.getPosition())); - boolean acquiredMutex = writeCell(null, physicalSchemaName, - physicalTableName, columnToDrop.toString()); - if (!acquiredMutex) { - throw new ConcurrentTableMutationException(physicalSchemaName, physicalTableName); + // check if client is already holding a mutex from previous retry + if (!acquiredColumnMutexSet.contains(columnToDrop.toString())) { + boolean acquiredMutex = writeCell(null, physicalSchemaName, + physicalTableName, columnToDrop.toString()); + if (!acquiredMutex) { + throw new ConcurrentTableMutationException(physicalSchemaName, + physicalTableName); + } + acquiredColumnMutexSet.add(columnToDrop.toString()); } - acquiredColumnMutexSet.add(columnToDrop.toString()); } dropColumnMutations(table, tableColumnsToDrop); @@ -4958,7 +5037,7 @@ else if (columnToDrop.isViewReferenced()) { try { MutationCode code = processMutationResult(schemaName, tableName, result); if (code == MutationCode.COLUMN_NOT_FOUND) { - addTableToCache(result); + addTableToCache(result, false); if (!statement.ifExists()) { throw new ColumnNotFoundException(schemaName, tableName, Bytes.toString(result.getFamilyName()), Bytes.toString(result.getColumnName())); } @@ -5094,7 +5173,14 @@ else if (columnToDrop.isViewReferenced()) { if (retried) { throw e; } - table = connection.getTable(fullTableName); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(LogUtil.addCustomAnnotations( + "Caught ConcurrentTableMutationException for table " + + SchemaUtil.getTableName(e.getSchemaName(), e.getTableName()) + + ". Will update cache and try again...", connection)); + } + updateCache(connection.getTenantId(), + e.getSchemaName(), e.getTableName(), true); retried = true; } catch (Throwable e) { TableMetricsManager.updateMetricsForSystemCatalogTableMethod(tableName, NUM_METADATA_LOOKUP_FAILURES, 1); @@ -5204,7 +5290,7 @@ public MutationState alterIndex(AlterIndexStatement statement) throws SQLExcepti if (code == MutationCode.TABLE_ALREADY_EXISTS) { if (result.getTable() != null) { // To accommodate connection-less update of index state - addTableToCache(result); + addTableToCache(result, false); // Set so that we get the table below with the potentially modified rowKeyOrderOptimizable flag set indexRef.setTable(result.getTable()); if (newIndexState == PIndexState.BUILDING && isAsync) { @@ -5324,14 +5410,18 @@ public MutationState alterIndex(AlterIndexStatement statement) throws SQLExcepti } } - private void addTableToCache(MetaDataMutationResult result) throws SQLException { - addTableToCache(result, TransactionUtil.getResolvedTime(connection, result)); + private void addTableToCache(MetaDataMutationResult result, boolean alwaysHitServerForAncestors) + throws SQLException { + addTableToCache(result, alwaysHitServerForAncestors, + TransactionUtil.getResolvedTime(connection, result)); } - private void addTableToCache(MetaDataMutationResult result, long timestamp) throws SQLException { - addColumnsAndIndexesFromAncestors(result, null, false); - PTable table = result.getTable(); - connection.addTable(table, timestamp); + private void addTableToCache(MetaDataMutationResult result, boolean alwaysHitServerForAncestors, + long timestamp) throws SQLException { + addColumnsIndexesAndLastDDLTimestampsFromAncestors(result, null, + false, alwaysHitServerForAncestors); + updateIndexesWithAncestorMap(result); + connection.addTable(result.getTable(), timestamp); } private void addFunctionToCache(MetaDataMutationResult result) throws SQLException { diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTable.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTable.java index ae425a97fab..d1d7d5227f4 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTable.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTable.java @@ -987,6 +987,10 @@ IndexMaintainer getIndexMaintainer(PTable dataTable, PTable cdcTable, */ String getIndexWhere(); + /** + * @return the map of all ancestors to their LAST_DDL_TIMESTAMP + */ + Map getAncestorLastDDLTimestampMap(); /** * diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTableImpl.java index 05fc24a959d..9d89607528d 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTableImpl.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTableImpl.java @@ -222,6 +222,7 @@ public class PTableImpl implements PTable { private Expression indexWhereExpression; private Set indexWhereColumns; private Long maxLookbackAge; + private Map ancestorLastDDLTimestampMap; private Set cdcIncludeScopes; public static class Builder { @@ -291,7 +292,7 @@ public static class Builder { private Set cdcIncludeScopes; private String indexWhere; private Long maxLookbackAge; - + private Map ancestorLastDDLTimestampMap = new HashMap<>(); // Used to denote which properties a view has explicitly modified private BitSet viewModifiedPropSet = new BitSet(3); // Optionally set columns for the builder, but not for the actual PTable @@ -726,6 +727,11 @@ public Builder setMaxLookbackAge(Long maxLookbackAge) { return this; } + public Builder setAncestorLastDDLTimestampMap(Map map) { + this.ancestorLastDDLTimestampMap = map; + return this; + } + public Builder setCDCIncludeScopes(Set cdcIncludeScopes) { if (cdcIncludeScopes != null) { this.cdcIncludeScopes = cdcIncludeScopes; @@ -1026,6 +1032,7 @@ private PTableImpl(Builder builder) { this.cdcIncludeScopes = builder.cdcIncludeScopes; this.indexWhere = builder.indexWhere; this.maxLookbackAge = builder.maxLookbackAge; + this.ancestorLastDDLTimestampMap = builder.ancestorLastDDLTimestampMap; } // When cloning table, ignore the salt column as it will be added back in the constructor @@ -1107,7 +1114,8 @@ public static PTableImpl.Builder builderFromExisting(PTable table) { .setExternalSchemaId(table.getExternalSchemaId()) .setStreamingTopicName(table.getStreamingTopicName()) .setIndexWhere(table.getIndexWhere()) - .setMaxLookbackAge(table.getMaxLookbackAge()); + .setMaxLookbackAge(table.getMaxLookbackAge()) + .setAncestorLastDDLTimestampMap(table.getAncestorLastDDLTimestampMap()); } @Override @@ -2432,6 +2440,11 @@ public Long getMaxLookbackAge() { return maxLookbackAge; } + @Override + public Map getAncestorLastDDLTimestampMap() { + return ancestorLastDDLTimestampMap; + } + private void buildIndexWhereExpression(PhoenixConnection connection) throws SQLException { PhoenixPreparedStatement pstmt = diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTableKey.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTableKey.java index 874b62f3a85..a0204b85415 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTableKey.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/PTableKey.java @@ -20,10 +20,13 @@ import org.apache.phoenix.query.QueryConstants; import org.apache.phoenix.thirdparty.com.google.common.base.Preconditions; +import org.apache.phoenix.util.SchemaUtil; public class PTableKey { private final PName tenantId; private final String name; + private final String schemaName; + private final String tableName; public PTableKey(PName tenantId, String name) { Preconditions.checkNotNull(name); @@ -33,6 +36,8 @@ public PTableKey(PName tenantId, String name) { } else { this.name = name; } + this.schemaName = SchemaUtil.getSchemaNameFromFullName(this.name); + this.tableName = SchemaUtil.getTableNameFromFullName(this.name); } public PName getTenantId() { @@ -42,6 +47,14 @@ public PName getTenantId() { public String getName() { return name; } + + public String getSchemaName() { + return schemaName; + } + + public String getTableName() { + return tableName; + } @Override public String toString() { diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TableRef.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TableRef.java index 71a839eb383..64f13acf6b9 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TableRef.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TableRef.java @@ -91,7 +91,7 @@ public TableRef(String alias, PTable table, long upperBoundTimeStamp, long lower this.table = table; // if UPDATE_CACHE_FREQUENCY is set, always let the server set timestamps this.upperBoundTimeStamp = table.getUpdateCacheFrequency()!=0 ? QueryConstants.UNSET_TIMESTAMP : upperBoundTimeStamp; - this.currentTime = this.upperBoundTimeStamp; + this.currentTime = upperBoundTimeStamp; this.lowerBoundTimeStamp = lowerBoundTimeStamp; this.hasDynamicCols = hasDynamicCols; this.hinted = hinted; diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/util/ClientUtil.java b/phoenix-core-client/src/main/java/org/apache/phoenix/util/ClientUtil.java index a4bdc32f111..2afd4640b3f 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/util/ClientUtil.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/util/ClientUtil.java @@ -25,6 +25,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; +import com.google.protobuf.ServiceException; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.NotServingRegionException; @@ -98,7 +99,7 @@ public static SQLException parseLocalOrRemoteServerException(Throwable t) { return parseRemoteException(t); } - private static SQLException parseRemoteException(Throwable t) { + public static SQLException parseRemoteException(Throwable t) { String message = t.getLocalizedMessage(); if (message != null) { @@ -192,4 +193,17 @@ public static boolean isHBaseNamespaceAvailable(Admin admin, String schemaName) String[] hbaseNamespaces = admin.listNamespaces(); return Arrays.asList(hbaseNamespaces).contains(schemaName); } + + /** + * Convert ServiceException into an IOException + * @param se ServiceException + * @return IOException + */ + public static IOException parseServiceException(ServiceException se) { + Throwable cause = se.getCause(); + if (cause != null && cause instanceof IOException) { + return (IOException) cause; + } + return new IOException(se); + } } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/util/IndexUtil.java b/phoenix-core-client/src/main/java/org/apache/phoenix/util/IndexUtil.java index bc123bf4d25..8aa2c33e929 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/util/IndexUtil.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/util/IndexUtil.java @@ -832,6 +832,11 @@ public static int getServerSidePriority(Configuration conf) { return conf.getInt(QueryServices.SERVER_SIDE_PRIOIRTY_ATTRIB, QueryServicesOptions.DEFAULT_SERVER_SIDE_PRIORITY); } + public static int getInvalidateMetadataCachePriority(Configuration conf) { + return conf.getInt(QueryServices.INVALIDATE_METADATA_CACHE_PRIORITY_ATTRIB, + QueryServicesOptions.DEFAULT_INVALIDATE_METADATA_CACHE_PRIORITY); + } + public static void removeEmptyColumn(Mutation m, byte[] emptyCF, byte[] emptyCQ) { List cellList = m.getFamilyCellMap().get(emptyCF); if (cellList == null) { diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java b/phoenix-core-client/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java index 290368b47ca..20641e4af45 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java @@ -54,6 +54,7 @@ import org.apache.hadoop.hbase.util.Pair; import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.phoenix.compile.QueryPlan; +import org.apache.phoenix.coprocessorclient.MetaDataProtocol; import org.apache.phoenix.expression.Expression; import org.apache.phoenix.expression.LiteralExpression; import org.apache.phoenix.expression.OrderByExpression; @@ -71,6 +72,7 @@ import org.apache.phoenix.monitoring.PhoenixTableMetric; import org.apache.phoenix.monitoring.TableMetricsManager; import org.apache.phoenix.monitoring.connectionqueryservice.ConnectionQueryServicesMetricsManager; +import org.apache.phoenix.query.ConnectionQueryServices; import org.apache.phoenix.query.QueryConstants; import org.apache.phoenix.query.QueryServices; import org.apache.phoenix.schema.AmbiguousColumnException; diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/util/ValidateLastDDLTimestampUtil.java b/phoenix-core-client/src/main/java/org/apache/phoenix/util/ValidateLastDDLTimestampUtil.java new file mode 100644 index 00000000000..dbb321a2443 --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/util/ValidateLastDDLTimestampUtil.java @@ -0,0 +1,228 @@ +/* + * 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.util; + +import java.sql.SQLException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.util.ByteStringer; +import org.apache.phoenix.coprocessor.generated.RegionServerEndpointProtos; +import org.apache.phoenix.exception.StaleMetadataCacheException; +import org.apache.phoenix.jdbc.PhoenixConnection; +import org.apache.phoenix.query.QueryConstants; +import org.apache.phoenix.query.QueryServices; +import org.apache.phoenix.query.QueryServicesOptions; +import org.apache.phoenix.schema.PName; +import org.apache.phoenix.schema.PTable; +import org.apache.phoenix.schema.PTableKey; +import org.apache.phoenix.schema.PTableType; +import org.apache.phoenix.schema.TableRef; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Utility class for last ddl timestamp validation from the client. + */ +public class ValidateLastDDLTimestampUtil { + + private ValidateLastDDLTimestampUtil() {} + + private static final Logger LOGGER = LoggerFactory + .getLogger(ValidateLastDDLTimestampUtil.class); + private static final List ALLOWED_PTABLE_TYPES = Arrays.asList(new PTableType[] + {PTableType.TABLE, PTableType.VIEW, PTableType.INDEX, PTableType.SYSTEM}); + + public static String getInfoString(PName tenantId, List tableRefs) { + StringBuilder sb = new StringBuilder(); + sb.append(String.format("Tenant: %s, ", tenantId)); + for (TableRef tableRef : tableRefs) { + sb.append(String.format("{Schema: %s, Table: %s},", + tableRef.getTable().getSchemaName(), + tableRef.getTable().getTableName())); + } + return sb.toString(); + } + + /** + * Get whether last ddl timestamp validation is enabled on the connection + * @param connection + * @return true if it is enabled, false otherwise + */ + public static boolean getValidateLastDdlTimestampEnabled(PhoenixConnection connection) { + return connection.getQueryServices().getProps() + .getBoolean(QueryServices.LAST_DDL_TIMESTAMP_VALIDATION_ENABLED, + QueryServicesOptions.DEFAULT_LAST_DDL_TIMESTAMP_VALIDATION_ENABLED); + } + + /** + * Get whether last ddl timestamp validation is enabled in the Configuration + * @param config + * @return true if it is enabled, false otherwise + */ + public static boolean getValidateLastDdlTimestampEnabled(Configuration config) { + return config.getBoolean( + QueryServices.LAST_DDL_TIMESTAMP_VALIDATION_ENABLED, + QueryServicesOptions.DEFAULT_LAST_DDL_TIMESTAMP_VALIDATION_ENABLED); + } + + /** + * Verifies that table metadata for given tables is up-to-date in client cache with server. + * A random live region server is picked for invoking the RPC to validate LastDDLTimestamp. + * Retry once if there was an error performing the RPC, otherwise throw the Exception. + * + * @param allTableRefs + * @param doRetry + * @throws SQLException + */ + public static void validateLastDDLTimestamp(PhoenixConnection conn, + List allTableRefs, + boolean doRetry) throws SQLException { + List tableRefs = filterTableRefs(allTableRefs); + String infoString = getInfoString(conn.getTenantId(), tableRefs); + try (Admin admin = conn.getQueryServices().getAdmin()) { + // get all live region servers + List regionServers + = conn.getQueryServices().getLiveRegionServers(); + // pick one at random + ServerName regionServer + = regionServers.get(ThreadLocalRandom.current().nextInt(regionServers.size())); + + // RPC + RegionServerEndpointProtos.RegionServerEndpointService.BlockingInterface + service = RegionServerEndpointProtos.RegionServerEndpointService + .newBlockingStub(admin.coprocessorService(regionServer)); + RegionServerEndpointProtos.ValidateLastDDLTimestampRequest request + = getValidateDDLTimestampRequest(tableRefs); + service.validateLastDDLTimestamp(null, request); + } catch (Exception e) { + SQLException parsedException = ClientUtil.parseServerException(e); + if (parsedException instanceof StaleMetadataCacheException) { + throw parsedException; + } + //retry once for any exceptions other than StaleMetadataCacheException + LOGGER.error("Error in validating DDL timestamp for {}", infoString, parsedException); + if (doRetry) { + // update the list of live region servers + conn.getQueryServices().refreshLiveRegionServers(); + validateLastDDLTimestamp(conn, tableRefs, false); + return; + } + throw parsedException; + } + } + + /** + * Build a request for the validateLastDDLTimestamp RPC for the given tables. + * 1. For a view, we need to add all its ancestors to the request + * in case something changed in the hierarchy. + * 2. For an index, we need to add its parent table to the request + * in case the index was dropped. + * 3. Add all indexes of a table/view in case index state was changed. + * + * @param tableRefs + * @return ValidateLastDDLTimestampRequest for the table in tableRef + */ + private static RegionServerEndpointProtos.ValidateLastDDLTimestampRequest + getValidateDDLTimestampRequest(List tableRefs) { + + RegionServerEndpointProtos.ValidateLastDDLTimestampRequest.Builder requestBuilder + = RegionServerEndpointProtos.ValidateLastDDLTimestampRequest.newBuilder(); + RegionServerEndpointProtos.LastDDLTimestampRequest.Builder innerBuilder; + + for (TableRef tableRef : tableRefs) { + + // validate all ancestors of this PTable if any + // index -> base table + // view -> parent view and its ancestors + // view index -> view and its ancestors + for (Map.Entry entry + : tableRef.getTable().getAncestorLastDDLTimestampMap().entrySet()) { + innerBuilder = RegionServerEndpointProtos.LastDDLTimestampRequest.newBuilder(); + PTableKey ancestorKey = entry.getKey(); + setLastDDLTimestampRequestParameters(innerBuilder, ancestorKey, entry.getValue()); + requestBuilder.addLastDDLTimestampRequests(innerBuilder); + } + + // add the current table to the request + PTable ptable = tableRef.getTable(); + innerBuilder = RegionServerEndpointProtos.LastDDLTimestampRequest.newBuilder(); + setLastDDLTimestampRequestParameters(innerBuilder, ptable.getKey(), + ptable.getLastDDLTimestamp()); + requestBuilder.addLastDDLTimestampRequests(innerBuilder); + + // add all indexes of the current table + for (PTable idxPTable : tableRef.getTable().getIndexes()) { + innerBuilder = RegionServerEndpointProtos.LastDDLTimestampRequest.newBuilder(); + setLastDDLTimestampRequestParameters(innerBuilder, idxPTable.getKey(), + idxPTable.getLastDDLTimestamp()); + requestBuilder.addLastDDLTimestampRequests(innerBuilder); + } + } + return requestBuilder.build(); + } + + /** + * For the given PTable, set the attributes on the LastDDLTimestampRequest. + */ + private static void setLastDDLTimestampRequestParameters( + RegionServerEndpointProtos.LastDDLTimestampRequest.Builder builder, + PTableKey key, long lastDDLTimestamp) { + String tableName = key.getTableName(); + String schemaName = key.getSchemaName(); + + // view(V) with Index (VIndex) -> child view (V1) -> grand child view (V2) + // inherited view index is of the form V2#V1#VIndex, it does not exist in syscat + if (tableName.contains(QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR)) { + int lastIndexOf = tableName.lastIndexOf(QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR); + String indexFullName = tableName.substring(lastIndexOf + 1); + tableName = SchemaUtil.getTableNameFromFullName(indexFullName); + schemaName = SchemaUtil.getSchemaNameFromFullName(indexFullName); + } + + byte[] tenantIDBytes = key.getTenantId() == null + ? HConstants.EMPTY_BYTE_ARRAY + : key.getTenantId().getBytes(); + byte[] schemaBytes = (schemaName == null || schemaName.isEmpty()) + ? HConstants.EMPTY_BYTE_ARRAY + : key.getSchemaName().getBytes(); + builder.setTenantId(ByteStringer.wrap(tenantIDBytes)); + builder.setSchemaName(ByteStringer.wrap(schemaBytes)); + builder.setTableName(ByteStringer.wrap(tableName.getBytes())); + builder.setLastDDLTimestamp(lastDDLTimestamp); + } + + /** + * Filter out any TableRefs which are not tables, views or indexes. + * @param tableRefs + * @return + */ + private static List filterTableRefs(List tableRefs) { + List filteredTableRefs = tableRefs.stream() + .filter(tableRef -> ALLOWED_PTABLE_TYPES.contains(tableRef.getTable().getType())) + .collect(Collectors.toList()); + return filteredTableRefs; + } +} diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/util/ViewUtil.java b/phoenix-core-client/src/main/java/org/apache/phoenix/util/ViewUtil.java index 505b9f3ad1a..aadba4a8e66 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/util/ViewUtil.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/util/ViewUtil.java @@ -484,7 +484,8 @@ public static void addIndexesFromParent(PhoenixConnection connection, PTable vie .setTableName(modifiedIndexName) .setViewStatement(viewStatement) .setUpdateCacheFrequency(view.getUpdateCacheFrequency()) - .setTenantId(view.getTenantId()) + //retain the tenantId from the index being inherited + .setTenantId(index.getTenantId()) .setPhysicalNames(Collections.singletonList(index.getPhysicalName())) .build()); } diff --git a/phoenix-core-client/src/main/protobuf/RegionServerEndpointService.proto b/phoenix-core-client/src/main/protobuf/RegionServerEndpointService.proto new file mode 100644 index 00000000000..2d0da268ba6 --- /dev/null +++ b/phoenix-core-client/src/main/protobuf/RegionServerEndpointService.proto @@ -0,0 +1,59 @@ +/* + * 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. + */ +option java_package = "org.apache.phoenix.coprocessor.generated"; +option java_outer_classname = "RegionServerEndpointProtos"; +option java_generic_services = true; +option java_generate_equals_and_hash = true; +option optimize_for = SPEED; + +message ValidateLastDDLTimestampResponse { +} + +message ValidateLastDDLTimestampRequest { + repeated LastDDLTimestampRequest lastDDLTimestampRequests = 1; +} + +message LastDDLTimestampRequest { + // Will be HConstants.EMPTY_BYTE_ARRAY if tenantID or schema name is null. + required bytes tenantId = 1; + required bytes schemaName = 2; + required bytes tableName = 3; + required int64 lastDDLTimestamp = 4; +} + +message InvalidateServerMetadataCache { + // Will be HConstants.EMPTY_BYTE_ARRAY if tenantID or schema name is null. + required bytes tenantId = 1; + required bytes schemaName = 2; + required bytes tableName = 3; +} + +message InvalidateServerMetadataCacheResponse { +} + +message InvalidateServerMetadataCacheRequest { + repeated InvalidateServerMetadataCache invalidateServerMetadataCacheRequests = 1; +} + +service RegionServerEndpointService { + rpc validateLastDDLTimestamp(ValidateLastDDLTimestampRequest) + returns (ValidateLastDDLTimestampResponse); + + rpc invalidateServerMetadataCache(InvalidateServerMetadataCacheRequest) + returns (InvalidateServerMetadataCacheResponse); +} \ No newline at end of file diff --git a/phoenix-core-server/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcScheduler.java b/phoenix-core-server/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcScheduler.java index ea6a5c97193..0a04ad47873 100644 --- a/phoenix-core-server/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcScheduler.java +++ b/phoenix-core-server/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcScheduler.java @@ -27,12 +27,13 @@ import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; +import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_INVALIDATE_CACHE_HANDLER_COUNT; + /** * {@link RpcScheduler} that first checks to see if this is an index or metadata update before passing off the * call to the delegate {@link RpcScheduler}. */ public class PhoenixRpcScheduler extends CompatPhoenixRpcScheduler { - // copied from org.apache.hadoop.hbase.ipc.SimpleRpcScheduler in HBase 0.98.4 private static final String CALL_QUEUE_HANDLER_FACTOR_CONF_KEY = "ipc.server.callqueue.handler.factor"; private static final String CALLQUEUE_LENGTH_CONF_KEY = "ipc.server.max.callqueue.length"; @@ -41,28 +42,44 @@ public class PhoenixRpcScheduler extends CompatPhoenixRpcScheduler { private int indexPriority; private int metadataPriority; private int serverSidePriority; + private int invalidateMetadataCachePriority; private RpcExecutor indexCallExecutor; private RpcExecutor metadataCallExecutor; private RpcExecutor serverSideCallExecutor; + // Executor for invalidating server side metadata cache RPCs. + private RpcExecutor invalidateMetadataCacheCallExecutor; private int port; - public PhoenixRpcScheduler(Configuration conf, RpcScheduler delegate, int indexPriority, int metadataPriority, int serversidePriority, PriorityFunction priorityFunction, Abortable abortable) { + public PhoenixRpcScheduler(Configuration conf, RpcScheduler delegate, int indexPriority, + int metadataPriority, int serversidePriority, + int invalidateMetadataCachePriority, + PriorityFunction priorityFunction, Abortable abortable) { // copied from org.apache.hadoop.hbase.ipc.SimpleRpcScheduler in HBase 0.98.4 int indexHandlerCount = conf.getInt(QueryServices.INDEX_HANDLER_COUNT_ATTRIB, QueryServicesOptions.DEFAULT_INDEX_HANDLER_COUNT); int metadataHandlerCount = conf.getInt(QueryServices.METADATA_HANDLER_COUNT_ATTRIB, QueryServicesOptions.DEFAULT_METADATA_HANDLER_COUNT); int serverSideHandlerCount = conf.getInt(QueryServices.SERVER_SIDE_HANDLER_COUNT_ATTRIB, QueryServicesOptions.DEFAULT_SERVERSIDE_HANDLER_COUNT); + int invalidateMetadataCacheHandlerCount = conf.getInt( + QueryServices.INVALIDATE_CACHE_HANDLER_COUNT_ATTRIB, + DEFAULT_INVALIDATE_CACHE_HANDLER_COUNT); int maxIndexQueueLength = conf.getInt(CALLQUEUE_LENGTH_CONF_KEY, indexHandlerCount*DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER); int maxMetadataQueueLength = conf.getInt(CALLQUEUE_LENGTH_CONF_KEY, metadataHandlerCount*DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER); int maxServerSideQueueLength = conf.getInt(CALLQUEUE_LENGTH_CONF_KEY, serverSideHandlerCount*DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER); + int maxInvalidateMetadataCacheQueueLength = conf.getInt(CALLQUEUE_LENGTH_CONF_KEY, + invalidateMetadataCacheHandlerCount * DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER); + this.indexPriority = indexPriority; this.metadataPriority = metadataPriority; this.serverSidePriority = serversidePriority; + this.invalidateMetadataCachePriority = invalidateMetadataCachePriority; this.delegate = delegate; this.indexCallExecutor = new BalancedQueueRpcExecutor("Index", indexHandlerCount, maxIndexQueueLength, priorityFunction,conf,abortable); this.metadataCallExecutor = new BalancedQueueRpcExecutor("Metadata", metadataHandlerCount, maxMetadataQueueLength, priorityFunction,conf,abortable); this.serverSideCallExecutor = new BalancedQueueRpcExecutor("ServerSide", serverSideHandlerCount, maxServerSideQueueLength, priorityFunction,conf,abortable); + this.invalidateMetadataCacheCallExecutor = new BalancedQueueRpcExecutor( + "InvalidateMetadataCache", invalidateMetadataCacheHandlerCount, + maxInvalidateMetadataCacheQueueLength, priorityFunction, conf, abortable); } @Override @@ -77,6 +94,7 @@ public void start() { indexCallExecutor.start(port); metadataCallExecutor.start(port); serverSideCallExecutor.start(port); + invalidateMetadataCacheCallExecutor.start(port); } @Override @@ -85,6 +103,7 @@ public void stop() { indexCallExecutor.stop(); metadataCallExecutor.stop(); serverSideCallExecutor.stop(); + invalidateMetadataCacheCallExecutor.stop(); } @Override @@ -97,6 +116,8 @@ public boolean compatDispatch(CallRunner callTask) throws IOException, Interrupt return metadataCallExecutor.dispatch(callTask); } else if (serverSidePriority == priority) { return serverSideCallExecutor.dispatch(callTask); + } else if (invalidateMetadataCachePriority == priority) { + return invalidateMetadataCacheCallExecutor.dispatch(callTask); } else { return delegate.dispatch(callTask); } @@ -114,7 +135,8 @@ public int getGeneralQueueLength() { return this.delegate.getGeneralQueueLength() + this.indexCallExecutor.getQueueLength() + this.metadataCallExecutor.getQueueLength() - + this.serverSideCallExecutor.getQueueLength(); + + this.serverSideCallExecutor.getQueueLength() + + this.invalidateMetadataCacheCallExecutor.getQueueLength(); } @Override @@ -132,7 +154,8 @@ public int getActiveRpcHandlerCount() { return this.delegate.getActiveRpcHandlerCount() + this.indexCallExecutor.getActiveHandlerCount() + this.metadataCallExecutor.getActiveHandlerCount() - + this.serverSideCallExecutor.getActiveHandlerCount(); + + this.serverSideCallExecutor.getActiveHandlerCount() + + this.invalidateMetadataCacheCallExecutor.getActiveHandlerCount(); } @Override @@ -155,6 +178,11 @@ public void setMetadataExecutorForTesting(RpcExecutor executor) { this.metadataCallExecutor = executor; } + @VisibleForTesting + public void setInvalidateMetadataCacheExecutorForTesting(RpcExecutor executor) { + this.invalidateMetadataCacheCallExecutor = executor; + } + @VisibleForTesting public void setServerSideExecutorForTesting(RpcExecutor executor) { this.serverSideCallExecutor = executor; @@ -229,5 +257,4 @@ public int getActiveMetaPriorityRpcHandlerCount() { public int getActiveReplicationRpcHandlerCount() { return this.delegate.getActiveReplicationRpcHandlerCount(); } - } diff --git a/phoenix-core-server/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcSchedulerFactory.java b/phoenix-core-server/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcSchedulerFactory.java index 5a22eb2c162..75042a45dba 100644 --- a/phoenix-core-server/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcSchedulerFactory.java +++ b/phoenix-core-server/src/main/java/org/apache/hadoop/hbase/ipc/PhoenixRpcSchedulerFactory.java @@ -62,12 +62,16 @@ public RpcScheduler create(Configuration conf, PriorityFunction priorityFunction validatePriority(serverSidePriority); // validate index and metadata priorities are not the same - Preconditions.checkArgument(indexPriority != metadataPriority, "Index and Metadata priority must not be same "+ indexPriority); + Preconditions.checkArgument(indexPriority != metadataPriority, + "Index and Metadata priority must not be same " + indexPriority); LOGGER.info("Using custom Phoenix Index RPC Handling with index rpc priority " + indexPriority + " and metadata rpc priority " + metadataPriority); - PhoenixRpcScheduler scheduler = - new PhoenixRpcScheduler(conf, delegate, indexPriority, metadataPriority, serverSidePriority, priorityFunction,abortable); + int invalidateCachePriority = IndexUtil.getInvalidateMetadataCachePriority(conf); + validatePriority(invalidateCachePriority); + PhoenixRpcScheduler scheduler = new PhoenixRpcScheduler(conf, delegate, indexPriority, + metadataPriority, serverSidePriority, invalidateCachePriority, priorityFunction, + abortable); return scheduler; } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java index bcace742b92..d5f4c67ad41 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java @@ -84,6 +84,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID_DATA_TYPE_BYTES; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT_BYTES; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE_BYTES; +import static org.apache.phoenix.query.QueryServices.SKIP_SYSTEM_TABLES_EXISTENCE_CHECK; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MAX_LOOKBACK_AGE_BYTES; import static org.apache.phoenix.schema.PTable.LinkType.PHYSICAL_TABLE; import static org.apache.phoenix.schema.PTable.LinkType.VIEW_INDEX_PARENT_TABLE; @@ -179,6 +180,7 @@ import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse; import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse; import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest; +import org.apache.phoenix.coprocessorclient.InvalidateServerMetadataCacheRequest; import org.apache.phoenix.coprocessorclient.MetaDataEndpointImplConstants; import org.apache.phoenix.coprocessorclient.MetaDataProtocol; import org.apache.phoenix.coprocessorclient.TableInfo; @@ -598,6 +600,7 @@ public static PName newPName(byte[] keyBuffer, int keyOffset, int keyLength) { private boolean blockWriteRebuildIndex; private int maxIndexesPerTable; private boolean isTablesMappingEnabled; + private boolean invalidateServerCacheEnabled; // this flag denotes that we will continue to write parent table column metadata while creating // a child view and also block metadata changes that were previously propagated to children @@ -641,7 +644,9 @@ public void start(CoprocessorEnvironment env) throws IOException { new ReadOnlyProps(config.iterator())); this.allowSplittableSystemCatalogRollback = config.getBoolean(QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK, QueryServicesOptions.DEFAULT_ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK); - + this.invalidateServerCacheEnabled + = config.getBoolean(QueryServices.PHOENIX_METADATA_INVALIDATE_CACHE_ENABLED, + QueryServicesOptions.DEFAULT_PHOENIX_METADATA_INVALIDATE_CACHE_ENABLED); LOGGER.info("Starting Tracing-Metrics Systems"); // Start the phoenix trace collection Tracing.addTraceMetricsSource(); @@ -1529,7 +1534,7 @@ private PTable getTableFromCells(List tableCellList, List> allC if (indexType != IndexType.LOCAL) { parentTable = getTable(null, SchemaUtil.getSchemaNameFromFullName(famName.getBytes()).getBytes(StandardCharsets.UTF_8), SchemaUtil.getTableNameFromFullName(famName.getBytes()).getBytes(StandardCharsets.UTF_8), clientTimeStamp, clientVersion); - if (parentTable == null) { + if (parentTable == null || isTableDeleted(parentTable)) { // parentTable is not in the cache. Since famName is only logical name, we need to find the physical table. try (PhoenixConnection connection = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class)) { parentTable = connection.getTableNoCache(famName.getString()); @@ -1539,7 +1544,7 @@ private PTable getTableFromCells(List tableCellList, List> allC } } - if (parentTable == null) { + if (parentTable == null || isTableDeleted(parentTable)) { if (indexType == IndexType.LOCAL) { PName tablePhysicalName = getPhysicalTableName( env.getRegion(),null, @@ -2509,6 +2514,18 @@ public void createTable(RpcController controller, CreateTableRequest request, // table/index/views. tableMetadata.add(MetaDataUtil.getLastDDLTimestampUpdate(tableKey, clientTimeStamp, EnvironmentEdgeManager.currentTimeMillis())); + if (tableType == INDEX) { + // Invalidate the cache on each regionserver for parent table/view. + List requests = new ArrayList<>(); + requests.add(new InvalidateServerMetadataCacheRequest(tenantIdBytes, + parentSchemaName, parentTableName)); + invalidateServerMetadataCache(requests); + long currentTimestamp = EnvironmentEdgeManager.currentTimeMillis(); + // If table type is index, then update the last ddl timestamp of the parent + // table or immediate parent view. + tableMetadata.add(MetaDataUtil.getLastDDLTimestampUpdate(parentTableKey, + currentTimestamp, currentTimestamp)); + } //and if we're doing change detection on this table or view, notify the //external schema registry and get its schema id @@ -2595,6 +2612,12 @@ public void createTable(RpcController controller, CreateTableRequest request, builder.setMutationTime(currentTimeStamp); //send the newly built table back because we generated the DDL timestamp server // side and the client doesn't have it. + if (clientTimeStamp != HConstants.LATEST_TIMESTAMP) { + // if a client uses a connection with currentSCN=t to create the table, + // the table is created with timestamp 't' but the timestamp range in the scan + // used by buildTable does not include 't' due to how SCN is implemented. + clientTimeStamp += 1; + } PTable newTable = buildTable(tableKey, cacheKey, region, clientTimeStamp, clientVersion); if (newTable != null) { @@ -2641,6 +2664,7 @@ private void exportSchema(List tableMetadata, byte[] tableKey, long cl PTable parentTable = null; //if this is a view, we need to get the columns from its parent table / view if (newTable != null && newTable.getType().equals(PTableType.VIEW)) { + // TODO why creating generic connection and not getConnectionOnServer? try (PhoenixConnection conn = (PhoenixConnection) ConnectionUtil.getInputConnection(env.getConfiguration())) { newTable = ViewUtil.addDerivedColumnsAndIndexesFromAncestors(conn, newTable); @@ -2862,8 +2886,20 @@ public void dropTable(RpcController controller, DropTableRequest request, if (parentLockKey != null) { acquireLock(region, parentLockKey, locks); } - - List invalidateList = new ArrayList(); + List requests = new ArrayList<>(); + requests.add(new InvalidateServerMetadataCacheRequest(tenantIdBytes, schemaName, + tableOrViewName)); + if (pTableType == INDEX) { + requests.add(new InvalidateServerMetadataCacheRequest(tenantIdBytes, schemaName, + parentTableName)); + long currentTimestamp = EnvironmentEdgeManager.currentTimeMillis(); + // If table type is index, then update the last ddl timestamp of the parent + // table or immediate parent view. + tableMetadata.add(MetaDataUtil.getLastDDLTimestampUpdate(parentLockKey, + currentTimestamp, currentTimestamp)); + } + invalidateServerMetadataCache(requests); + List invalidateList = new ArrayList<>(); result = doDropTable(lockKey, tenantIdBytes, schemaName, tableOrViewName, parentTableName, PTableType.fromSerializedValue(tableType), tableMetadata, childLinkMutations, invalidateList, tableNamesToDelete, @@ -3274,10 +3310,15 @@ private MetaDataMutationResult mutateColumn( if (mutationResult.isPresent()) { return mutationResult.get(); } - + // We take a write row lock for tenantId, schemaName, tableOrViewName acquireLock(region, key, locks); + // Invalidate the cache from all the regionservers. + List requests = new ArrayList<>(); + requests.add(new InvalidateServerMetadataCacheRequest(tenantId, schemaName, + tableOrViewName)); + invalidateServerMetadataCache(requests); ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key); - List invalidateList = new ArrayList(); + List invalidateList = new ArrayList<>(); invalidateList.add(cacheKey); PTable table = getTableFromCache(cacheKey, clientTimeStamp, clientVersion); if (failConcurrentMutateAddColumnOneTimeForTesting) { @@ -3529,6 +3570,24 @@ private MetaDataMutationResult mutateColumn( } } + private void invalidateServerMetadataCache(List requests) + throws Throwable { + if (!this.invalidateServerCacheEnabled) { + LOGGER.info("Skip invalidating server metadata cache since conf property" + + " phoenix.metadata.invalidate.cache.enabled is set to false"); + return; + } + Properties properties = new Properties(); + // Skip checking of system table existence since the system tables should have created + // by now. + properties.setProperty(SKIP_SYSTEM_TABLES_EXISTENCE_CHECK, "true"); + try (PhoenixConnection connection = QueryUtil.getConnectionOnServer(properties, + env.getConfiguration()).unwrap(PhoenixConnection.class)) { + ConnectionQueryServices queryServices = connection.getQueryServices(); + queryServices.invalidateServerMetadataCache(requests); + } + } + private boolean hasInheritableTablePropertyChanged(PTable newTable, PTable oldTable) { return ! Objects.equals(newTable.getMaxLookbackAge(), oldTable.getMaxLookbackAge()); } @@ -3736,7 +3795,9 @@ private PTable doGetTable(byte[] tenantId, byte[] schemaName, byte[] tableName, buildTable(key, cacheKey, region, clientTimeStamp, clientVersion); return table; } finally { - if (!wasLocked && rowLock != null) rowLock.release(); + if (!wasLocked && rowLock != null) { + rowLock.release(); + } } } @@ -3796,8 +3857,6 @@ public int compare(byte[] o1, byte[] o2) { public void dropColumn(RpcController controller, final DropColumnRequest request, RpcCallback done) { List tableMetaData = null; - final List tableNamesToDelete = Lists.newArrayList(); - final List sharedTablesToDelete = Lists.newArrayList(); try { tableMetaData = ProtobufUtil.getMutations(request); PTable parentTable = request.hasParentTable() ? PTableImpl.createFromProto(request.getParentTable()) : null; @@ -3823,7 +3882,7 @@ private MetaDataMutationResult dropIndexes(RegionCoprocessorEnvironment env, PTa List invalidateList, List locks, long clientTimeStamp, List tableMetaData, PColumn columnToDelete, List tableNamesToDelete, List sharedTablesToDelete, int clientVersion) - throws IOException, SQLException { + throws Throwable { // Look for columnToDelete in any indexes. If found as PK column, get lock and drop the // index and then invalidate it // Covered columns are deleted from the index by the client @@ -3870,6 +3929,13 @@ private MetaDataMutationResult dropIndexes(RegionCoprocessorEnvironment env, PTa // that a change in index state doesn't // occur while we're dropping it. acquireLock(region, indexKey, locks); + // invalidate server metadata cache when dropping index + List requests = new ArrayList<>(); + requests.add(new InvalidateServerMetadataCacheRequest(tenantId, + index.getSchemaName().getBytes(), + index.getTableName().getBytes())); + + invalidateServerMetadataCache(requests); List childLinksMutations = Lists.newArrayList(); MetaDataMutationResult result = doDropTable(indexKey, tenantId, index.getSchemaName().getBytes(), index.getTableName().getBytes(), @@ -4117,6 +4183,10 @@ public void updateIndexState(RpcController controller, UpdateIndexStateRequest r done.run(builder.build()); return; } + List requests = new ArrayList<>(); + requests.add(new InvalidateServerMetadataCacheRequest(tenantId, schemaName, + tableName)); + invalidateServerMetadataCache(requests); getCoprocessorHost().preIndexUpdate(Bytes.toString(tenantId), SchemaUtil.getTableName(schemaName, tableName), TableName.valueOf(loadedTable.getPhysicalName().getBytes()), diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixRegionServerEndpoint.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixRegionServerEndpoint.java new file mode 100644 index 00000000000..59fd1209db4 --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/PhoenixRegionServerEndpoint.java @@ -0,0 +1,117 @@ +/* + * 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.coprocessor; + +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; + +import java.io.IOException; +import java.util.Collections; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessor; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.phoenix.cache.ServerMetadataCache; +import org.apache.phoenix.cache.ServerMetadataCacheImpl; +import org.apache.phoenix.coprocessor.generated.RegionServerEndpointProtos; +import org.apache.phoenix.coprocessorclient.metrics.MetricsMetadataCachingSource; +import org.apache.phoenix.coprocessorclient.metrics.MetricsPhoenixCoprocessorSourceFactory; +import org.apache.phoenix.protobuf.ProtobufUtil; +import org.apache.phoenix.util.ClientUtil; +import org.apache.phoenix.util.SchemaUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is first implementation of RegionServer coprocessor introduced by Phoenix. + */ +public class PhoenixRegionServerEndpoint + extends RegionServerEndpointProtos.RegionServerEndpointService + implements RegionServerCoprocessor { + private static final Logger LOGGER = LoggerFactory.getLogger(PhoenixRegionServerEndpoint.class); + private MetricsMetadataCachingSource metricsSource; + protected Configuration conf; + + @Override + public void start(CoprocessorEnvironment env) throws IOException { + this.conf = env.getConfiguration(); + this.metricsSource = MetricsPhoenixCoprocessorSourceFactory + .getInstance().getMetadataCachingSource(); + } + + @Override + public void validateLastDDLTimestamp(RpcController controller, + RegionServerEndpointProtos.ValidateLastDDLTimestampRequest request, + RpcCallback done) { + metricsSource.incrementValidateTimestampRequestCount(); + ServerMetadataCache cache = getServerMetadataCache(); + for (RegionServerEndpointProtos.LastDDLTimestampRequest lastDDLTimestampRequest + : request.getLastDDLTimestampRequestsList()) { + byte[] tenantID = lastDDLTimestampRequest.getTenantId().toByteArray(); + byte[] schemaName = lastDDLTimestampRequest.getSchemaName().toByteArray(); + byte[] tableName = lastDDLTimestampRequest.getTableName().toByteArray(); + long clientLastDDLTimestamp = lastDDLTimestampRequest.getLastDDLTimestamp(); + String tenantIDStr = Bytes.toString(tenantID); + String fullTableName = SchemaUtil.getTableName(schemaName, tableName); + try { + VerifyLastDDLTimestamp.verifyLastDDLTimestamp(cache, tenantID, schemaName, + tableName, clientLastDDLTimestamp); + } catch (Throwable t) { + String errorMsg = String.format("Verifying last ddl timestamp FAILED for " + + "tenantID: %s, fullTableName: %s", tenantIDStr, fullTableName); + LOGGER.error(errorMsg, t); + IOException ioe = ClientUtil.createIOException(errorMsg, t); + ProtobufUtil.setControllerException(controller, ioe); + //If an index was dropped and a client tries to query it, we will validate table + //first and encounter stale metadata, if we don't break the coproc will run into + //table not found error since it will not be able to validate the dropped index. + //this should be fine for views too since we will update the entire hierarchy. + break; + } + } + } + + @Override + public void invalidateServerMetadataCache(RpcController controller, + RegionServerEndpointProtos.InvalidateServerMetadataCacheRequest request, + RpcCallback done) { + for (RegionServerEndpointProtos.InvalidateServerMetadataCache invalidateCacheRequest + : request.getInvalidateServerMetadataCacheRequestsList()) { + byte[] tenantID = invalidateCacheRequest.getTenantId().toByteArray(); + byte[] schemaName = invalidateCacheRequest.getSchemaName().toByteArray(); + byte[] tableName = invalidateCacheRequest.getTableName().toByteArray(); + String fullTableName = SchemaUtil.getTableName(schemaName, tableName); + String tenantIDStr = Bytes.toString(tenantID); + LOGGER.info("PhoenixRegionServerEndpoint invalidating the cache for tenantID: {}," + + " tableName: {}", tenantIDStr, fullTableName); + ServerMetadataCache cache = getServerMetadataCache(); + cache.invalidate(tenantID, schemaName, tableName); + } + } + + @Override + public Iterable getServices() { + return Collections.singletonList(this); + } + + public ServerMetadataCache getServerMetadataCache() { + return ServerMetadataCacheImpl.getInstance(conf); + } +} \ No newline at end of file diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/VerifyLastDDLTimestamp.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/VerifyLastDDLTimestamp.java new file mode 100644 index 00000000000..4c5bf09704a --- /dev/null +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/VerifyLastDDLTimestamp.java @@ -0,0 +1,69 @@ +/* + * 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.coprocessor; + +import java.sql.SQLException; + +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.phoenix.cache.ServerMetadataCache; +import org.apache.phoenix.exception.StaleMetadataCacheException; +import org.apache.phoenix.util.SchemaUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Client provides last DDL timestamp of tables/views/indexes included in read/write operation + * This verifies that client has the latest version of LAST_DDL_TIMESTAMP version. + * If client's provided LAST_DDL_TIMESTAMP is less than what is present in SYSTEM.CATALOG + * then it throws StaleMetadataCacheException. + */ +public class VerifyLastDDLTimestamp { + private static final Logger LOGGER = LoggerFactory.getLogger(VerifyLastDDLTimestamp.class); + + private VerifyLastDDLTimestamp() { + // Not to be instantiated. + } + + /** + * Verify that LAST_DDL_TIMESTAMP provided by the client is up to date. If it is stale it will + * throw StaleMetadataCacheException. + * + * @param tenantID tenant id + * @param schemaName schema name + * @param tableName table name + * @param clientLastDDLTimestamp last ddl timestamp provided by client + * @param cache ServerMetadataCache + * @throws SQLException StaleMetadataCacheException if client provided timestamp + * is stale. + */ + public static void verifyLastDDLTimestamp(ServerMetadataCache cache, byte[] tenantID, + byte[] schemaName, byte[] tableName, long clientLastDDLTimestamp) + throws SQLException { + long lastDDLTimestamp = cache.getLastDDLTimestampForTable(tenantID, schemaName, tableName); + // Is it possible to have client last ddl timestamp greater than server side? + if (clientLastDDLTimestamp < lastDDLTimestamp) { + LOGGER.error("Stale metadata for LAST_DDL_TIMESTAMP for tenantID: {}, schema: {}," + + " table: {}, client provided timestamp: {}, server timestamp: {}", + Bytes.toString(tenantID), Bytes.toString(schemaName), + Bytes.toString(tableName), clientLastDDLTimestamp, lastDDLTimestamp); + String fullTableName = SchemaUtil.getTableName(schemaName, tableName); + throw new StaleMetadataCacheException("Stale metadata cache for table name: " + + fullTableName); + } + } +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/schema/transform/Transform.java b/phoenix-core-server/src/main/java/org/apache/phoenix/schema/transform/Transform.java index 53f95537cf6..b216c9d7e36 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/schema/transform/Transform.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/schema/transform/Transform.java @@ -121,6 +121,12 @@ public static void removeTransformRecord( ).execute(); } + /** + * Disable caching re-design if you use Online Data Format Change since the cutover logic + * is currently incompatible and clients may not learn about the physical table change. + * See https://issues.apache.org/jira/browse/PHOENIX-6883 and + * https://issues.apache.org/jira/browse/PHOENIX-7284. + */ public static void doCutover(PhoenixConnection connection, SystemTransformRecord systemTransformRecord) throws Exception{ String tenantId = systemTransformRecord.getTenantId(); String schema = systemTransformRecord.getSchemaName(); diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/util/ServerUtil.java b/phoenix-core-server/src/main/java/org/apache/phoenix/util/ServerUtil.java index 2b364e28651..c88e7c12d3f 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/util/ServerUtil.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/util/ServerUtil.java @@ -88,7 +88,7 @@ private static Table getTableFromSingletonPool(RegionCoprocessorEnvironment env, return conn.getTable(tableName); } catch (RuntimeException t) { // handle cases that an IOE is wrapped inside a RuntimeException like HTableInterface#createHTableInterface - if(t.getCause() instanceof IOException) { + if (t.getCause() instanceof IOException) { throw (IOException)t.getCause(); } else { throw t; diff --git a/phoenix-core/src/it/java/org/apache/phoenix/cache/ServerMetadataCacheIT.java b/phoenix-core/src/it/java/org/apache/phoenix/cache/ServerMetadataCacheIT.java new file mode 100644 index 00000000000..1ecf1843c98 --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/cache/ServerMetadataCacheIT.java @@ -0,0 +1,1921 @@ +/* + * 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.cache; + +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessor; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.phoenix.coprocessorclient.metrics.MetricsMetadataCachingSource; +import org.apache.phoenix.coprocessorclient.metrics.MetricsPhoenixCoprocessorSourceFactory; +import org.apache.phoenix.end2end.IndexToolIT; +import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; +import org.apache.phoenix.end2end.ParallelStatsDisabledIT; +import org.apache.phoenix.end2end.PhoenixRegionServerEndpointTestImpl; +import org.apache.phoenix.end2end.ServerMetadataCacheTestImpl; +import org.apache.phoenix.jdbc.PhoenixConnection; +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; +import org.apache.phoenix.jdbc.PhoenixResultSet; +import org.apache.phoenix.jdbc.PhoenixStatement; +import org.apache.phoenix.monitoring.GlobalClientMetrics; +import org.apache.phoenix.query.ConnectionQueryServices; +import org.apache.phoenix.query.QueryConstants; +import org.apache.phoenix.query.QueryServices; +import org.apache.phoenix.schema.ColumnNotFoundException; +import org.apache.phoenix.schema.PIndexState; +import org.apache.phoenix.schema.PTable; +import org.apache.phoenix.schema.PTableKey; +import org.apache.phoenix.schema.TableNotFoundException; +import org.apache.phoenix.schema.types.PVarchar; +import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; +import org.apache.phoenix.util.PhoenixRuntime; +import org.apache.phoenix.util.PropertiesUtil; +import org.apache.phoenix.util.QueryUtil; +import org.apache.phoenix.util.ReadOnlyProps; +import org.apache.phoenix.util.SchemaUtil; +import org.apache.phoenix.util.TestUtil; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.mockito.Mockito; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Map; +import java.util.Properties; +import java.util.Random; + +import static org.apache.hadoop.hbase.coprocessor.CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY; +import static org.apache.phoenix.query.ConnectionQueryServicesImpl.INVALIDATE_SERVER_METADATA_CACHE_EX_MESSAGE; +import static org.apache.phoenix.query.QueryServices.PHOENIX_METADATA_INVALIDATE_CACHE_ENABLED; +import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +// End to end tests for metadata caching re-design. +@Category(NeedsOwnMiniClusterTest.class) +public class ServerMetadataCacheIT extends ParallelStatsDisabledIT { + + private final Random RANDOM = new Random(42); + + private static ServerName serverName; + + @BeforeClass + public static synchronized void doSetup() throws Exception { + Map props = Maps.newHashMapWithExpectedSize(1); + props.put(QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB, "NEVER"); + props.put(QueryServices.LAST_DDL_TIMESTAMP_VALIDATION_ENABLED, Boolean.toString(true)); + props.put(PHOENIX_METADATA_INVALIDATE_CACHE_ENABLED, Boolean.toString(true)); + props.put(QueryServices.TASK_HANDLING_INTERVAL_MS_ATTRIB, + Long.toString(Long.MAX_VALUE)); + props.put(QueryServices.TASK_HANDLING_INITIAL_DELAY_MS_ATTRIB, + Long.toString(Long.MAX_VALUE)); + setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator())); + assertEquals(1, getUtility().getHBaseCluster().getNumLiveRegionServers()); + serverName = getUtility().getHBaseCluster().getRegionServer(0).getServerName(); + } + + @Before + public void resetMetrics() { + GlobalClientMetrics.GLOBAL_CLIENT_STALE_METADATA_CACHE_EXCEPTION_COUNTER.getMetric().reset(); + } + + @After + public void resetMetadataCache() { + ServerMetadataCacheTestImpl.resetCache(); + } + + /** + * Get the server metadata cache instance from the endpoint loaded on the region server. + */ + private ServerMetadataCacheTestImpl getServerMetadataCache() { + String phoenixRegionServerEndpoint = config.get(REGIONSERVER_COPROCESSOR_CONF_KEY); + assertNotNull(phoenixRegionServerEndpoint); + RegionServerCoprocessor coproc = getUtility().getHBaseCluster() + .getRegionServer(0) + .getRegionServerCoprocessorHost() + .findCoprocessor(phoenixRegionServerEndpoint); + assertNotNull(coproc); + ServerMetadataCache cache = ((PhoenixRegionServerEndpointTestImpl)coproc).getServerMetadataCache(); + assertNotNull(cache); + return (ServerMetadataCacheTestImpl)cache; + } + + /** + * Make sure cache is working fine for base table. + * @throws Exception + */ + @Test + public void testCacheForBaseTable() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String tableNameStr = generateUniqueName(); + PTable pTable; + // use a spyed ConnectionQueryServices so we can verify calls to getTable + ConnectionQueryServices spyCQS = Mockito.spy(driver.getConnectionQueryServices(getUrl(), + PropertiesUtil.deepCopy(TEST_PROPERTIES))); + try(Connection conn = spyCQS.connect(getUrl(), props)) { + conn.setAutoCommit(false); + // Create a test table. + createTable(conn, tableNameStr); + pTable = PhoenixRuntime.getTableNoCache(conn, + tableNameStr);// --> First call to CQSI#getTable + ServerMetadataCacheTestImpl cache = getServerMetadataCache(); + // Override the connection to use in ServerMetadataCache + cache.setConnectionForTesting(conn); + byte[] tableName = Bytes.toBytes(tableNameStr); + long lastDDLTimestampFromCache = cache.getLastDDLTimestampForTable( + null, null, tableName); // --> Second call to CQSI#getTable + // Make sure the lastDDLTimestamp are the same. + assertEquals(pTable.getLastDDLTimestamp().longValue(), lastDDLTimestampFromCache); + // Verify that we made 2 calls to CQSI#getTable. + verify(spyCQS, times(2)).getTable( + any(), any(), eq(tableName), anyLong(), anyLong()); + // Make the same call 2 times to make sure it returns from the cache. + cache.getLastDDLTimestampForTable(null, null, tableName); + cache.getLastDDLTimestampForTable(null, null, tableName); + // Both the above 2 calls were served from the cache. + verify(spyCQS, times(2)).getTable( + any(), any(), eq(tableName), anyLong(), anyLong()); + } + } + + /** + * Make sure cache is working fine for global view. + * @throws Exception + */ + @Test + public void testCacheForGlobalView() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String tableNameStr = generateUniqueName(); + PTable viewTable; + // use a spyed ConnectionQueryServices so we can verify calls to getTable + ConnectionQueryServices spyCQS = Mockito.spy(driver.getConnectionQueryServices(getUrl(), + PropertiesUtil.deepCopy(TEST_PROPERTIES))); + try (Connection conn = spyCQS.connect(getUrl(), props)) { + conn.setAutoCommit(false); + // Create a test table. + createTable(conn, tableNameStr); + // Create view on table. + String whereClause = " WHERE v1 = 1000"; + String viewNameStr = generateUniqueName(); + createViewWhereClause(conn, tableNameStr, viewNameStr, whereClause); + viewTable = PhoenixRuntime.getTableNoCache(conn, viewNameStr); // --> First call to CQSI#getTable + ServerMetadataCacheTestImpl cache = getServerMetadataCache();; + // Override the connection to use in ServerMetadataCache + cache.setConnectionForTesting(conn); + + long lastDDLTimestampFromCache = cache.getLastDDLTimestampForTable( + null, null, Bytes.toBytes(viewNameStr)); // --> Second call to CQSI#getTable + + byte[] viewNameBytes = Bytes.toBytes(viewNameStr); + // Make sure the lastDDLTimestamp are the same. + assertEquals(viewTable.getLastDDLTimestamp().longValue(), lastDDLTimestampFromCache); + // Verify that we made 2 calls to CQSI#getTable. + verify(spyCQS, times(2)).getTable( + any(), any(), eq(viewNameBytes), anyLong(), anyLong()); + // Make the same call 2 times to make sure it returns from the cache. + cache.getLastDDLTimestampForTable(null, null, viewNameBytes); + cache.getLastDDLTimestampForTable(null, null, viewNameBytes); + verify(spyCQS, times(2)).getTable( + any(), any(), eq(viewNameBytes), anyLong(), anyLong()); + } + } + + /** + * Make sure cache is working fine for tenant view. + * @throws Exception + */ + @Test + public void testCacheForTenantView() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String tableNameStr = generateUniqueName(); + try (Connection conn = DriverManager.getConnection(getUrl(), props)) { + conn.setAutoCommit(false); + // Create a test table. + createTable(conn, tableNameStr); + } + String tenantId = "T_" + generateUniqueName(); + Properties tenantProps = PropertiesUtil.deepCopy(TEST_PROPERTIES); + tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId); + PTable tenantViewTable; + // Create view on table. + String whereClause = " WHERE v1 = 1000"; + String tenantViewNameStr = generateUniqueName(); + ConnectionQueryServices spyCQS = Mockito.spy(driver.getConnectionQueryServices(getUrl(), + PropertiesUtil.deepCopy(TEST_PROPERTIES))); + try (Connection conn = spyCQS.connect(getUrl(), tenantProps)) { + createViewWhereClause(conn, tableNameStr, tenantViewNameStr, whereClause); + tenantViewTable = PhoenixRuntime.getTableNoCache(conn, + tenantViewNameStr); // --> First call to CQSI#getTable + ServerMetadataCacheTestImpl cache = getServerMetadataCache();; + // Override the connection to use in ServerMetadataCache + cache.setConnectionForTesting(conn); + byte[] tenantIDBytes = Bytes.toBytes(tenantId); + long lastDDLTimestampFromCache = cache.getLastDDLTimestampForTable(tenantIDBytes, + null, Bytes.toBytes(tenantViewNameStr)); // --> Second call to CQSI#getTable + assertEquals(tenantViewTable.getLastDDLTimestamp().longValue(), + lastDDLTimestampFromCache); + byte[] tenantViewNameBytes = Bytes.toBytes(tenantViewNameStr); + // Verify that we made 2 calls to CQSI#getTable. + verify(spyCQS, times(2)).getTable( + any(), any(), eq(tenantViewNameBytes), anyLong(), anyLong()); + // Make the same call 2 times to make sure it returns from the cache. + cache.getLastDDLTimestampForTable(tenantIDBytes, + null, Bytes.toBytes(tenantViewNameStr)); + cache.getLastDDLTimestampForTable(tenantIDBytes, + null, Bytes.toBytes(tenantViewNameStr)); + verify(spyCQS, times(2)).getTable( + any(), any(), eq(tenantViewNameBytes), anyLong(), anyLong()); + } + } + + /** + * Make sure we are invalidating the cache for table with no tenant connection, no schema name + * and valid table name. + * @throws Exception + */ + @Test + public void testInvalidateCacheForBaseTable() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String tableNameStr = generateUniqueName(); + PTable pTable; + try (Connection conn = DriverManager.getConnection(getUrl(), props)) { + conn.setAutoCommit(false); + // Create a test table. + createTable(conn, tableNameStr); + pTable = PhoenixRuntime.getTableNoCache(conn, tableNameStr); + ServerMetadataCacheTestImpl cache = getServerMetadataCache();; + // Override the connection to use in ServerMetadataCache + cache.setConnectionForTesting(conn); + byte[] tableName = Bytes.toBytes(tableNameStr); + long lastDDLTimestampFromCache = cache.getLastDDLTimestampForTable( + null, null, tableName); + assertEquals(pTable.getLastDDLTimestamp().longValue(), lastDDLTimestampFromCache); + // Invalidate the cache for this table. + cache.invalidate(null, null, tableName); + assertNull(cache.getLastDDLTimestampForTableFromCacheOnly(null, null, tableName)); + } + } + + /** + * Make sure we are invalidating the cache for table with no tenant connection, + * valid schema name and table name. + * @throws Exception + */ + @Test + public void testInvalidateCacheForBaseTableWithSchemaName() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String schemaName = generateUniqueName(); + String tableName = generateUniqueName(); + String fullTableName = SchemaUtil.getTableName(schemaName, tableName); + PTable pTable; + try (Connection conn = DriverManager.getConnection(getUrl(), props)) { + conn.setAutoCommit(false); + // Create a test table. + createTable(conn, fullTableName); + pTable = PhoenixRuntime.getTableNoCache(conn, fullTableName); + ServerMetadataCacheTestImpl cache = getServerMetadataCache();; + // Override the connection to use in ServerMetadataCache + cache.setConnectionForTesting(conn); + byte[] tableNameBytes = Bytes.toBytes(fullTableName); + long lastDDLTimestampFromCache = cache.getLastDDLTimestampForTable( + null, Bytes.toBytes(schemaName), Bytes.toBytes(tableName)); + assertEquals(pTable.getLastDDLTimestamp().longValue(), lastDDLTimestampFromCache); + // Invalidate the cache for this table. + cache.invalidate(null, Bytes.toBytes(schemaName), Bytes.toBytes(tableName)); + assertNull(cache.getLastDDLTimestampForTableFromCacheOnly(null, + Bytes.toBytes(schemaName), Bytes.toBytes(tableName))); + } + } + + /** + * Make sure we are invalidating the cache for view with tenant connection. + * @throws Exception + */ + @Test + public void testInvalidateCacheForTenantView() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String tableNameStr = generateUniqueName(); + try (Connection conn = DriverManager.getConnection(getUrl(), props)) { + conn.setAutoCommit(false); + // Create a test table. + createTable(conn, tableNameStr); + } + String tenantId = "T_" + generateUniqueName(); + Properties tenantProps = PropertiesUtil.deepCopy(TEST_PROPERTIES); + tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId); + PTable tenantViewTable; + // Create view on table. + String whereClause = " WHERE V1 = 1000"; + String tenantViewNameStr = generateUniqueName(); + try (Connection conn = DriverManager.getConnection(getUrl(), tenantProps)) { + createViewWhereClause(conn, tableNameStr, tenantViewNameStr, whereClause); + tenantViewTable = PhoenixRuntime.getTableNoCache(conn, tenantViewNameStr); + ServerMetadataCacheTestImpl cache = getServerMetadataCache();; + // Override the connection to use in ServerMetadataCache + cache.setConnectionForTesting(conn); + byte[] tenantIDBytes = Bytes.toBytes(tenantId); + byte[] tenantViewNameBytes = Bytes.toBytes(tenantViewNameStr); + long lastDDLTimestampFromCache = cache.getLastDDLTimestampForTable( + tenantIDBytes, null, tenantViewNameBytes); + assertEquals(tenantViewTable.getLastDDLTimestamp().longValue(), + lastDDLTimestampFromCache); + // Invalidate the cache for this table. + cache.invalidate(tenantIDBytes, null, tenantViewNameBytes); + assertNull(cache.getLastDDLTimestampForTableFromCacheOnly( + tenantIDBytes, null, tenantViewNameBytes)); + } + } + + /** + * Make sure we are invalidating the cache for table with no tenant connection, no schema name + * and valid table name when we run alter statement. + * @throws Exception + */ + @Test + public void testInvalidateCacheForBaseTableWithAlterStatement() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String tableNameStr = generateUniqueName(); + byte[] tableNameBytes = Bytes.toBytes(tableNameStr); + PTable pTable; + ServerMetadataCacheTestImpl cache = getServerMetadataCache();; + try (Connection conn = DriverManager.getConnection(getUrl(), props)) { + conn.setAutoCommit(false); + // Create a test table. + createTable(conn, tableNameStr); + pTable = PhoenixRuntime.getTableNoCache(conn, tableNameStr); + long lastDDLTimestamp = pTable.getLastDDLTimestamp(); + assertEquals(lastDDLTimestamp, + cache.getLastDDLTimestampForTable(null, null, tableNameBytes)); + String alterDDLStmt = "ALTER TABLE " + tableNameStr + " SET DISABLE_WAL = true"; + conn.createStatement().execute(alterDDLStmt); + // The above alter statement will invalidate the last ddl timestamp from metadata cache. + // Notice that we are using cache#getLastDDLTimestampForTableFromCacheOnly which will + // read the last ddl timestamp only from the cache and return null if not present in + // the cache. + assertNull(cache.getLastDDLTimestampForTableFromCacheOnly(null, null, tableNameBytes)); + // This will load the cache with the latest last ddl timestamp value. + long lastDDLTimestampAfterAlterStmt = cache.getLastDDLTimestampForTable(null, + null, tableNameBytes); + assertNotNull(lastDDLTimestampAfterAlterStmt); + // Make sure that the last ddl timestamp value after ALTER statement + // is greater than previous one. + assertTrue(lastDDLTimestampAfterAlterStmt > lastDDLTimestamp); + } + } + + /** + * Make sure we are invalidating the cache for table with no tenant connection, no schema name + * and valid table name when we run drop table statement. + * @throws Exception + */ + @Test + public void testInvalidateCacheForBaseTableWithDropTableStatement() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String tableNameStr = generateUniqueName(); + byte[] tableNameBytes = Bytes.toBytes(tableNameStr); + PTable pTable; + ServerMetadataCacheTestImpl cache = getServerMetadataCache();; + try (Connection conn = DriverManager.getConnection(getUrl(), props)) { + conn.setAutoCommit(false); + // Create a test table. + createTable(conn, tableNameStr); + pTable = PhoenixRuntime.getTableNoCache(conn, tableNameStr); + long lastDDLTimestamp = pTable.getLastDDLTimestamp(); + assertEquals(lastDDLTimestamp, + cache.getLastDDLTimestampForTable(null, null, tableNameBytes)); + assertNotNull(cache.getLastDDLTimestampForTableFromCacheOnly(null, null, + tableNameBytes)); + String alterDDLStmt = "DROP TABLE " + tableNameStr; + conn.createStatement().execute(alterDDLStmt); + // The above alter statement will invalidate the last ddl timestamp from metadata cache. + // Notice that we are using cache#getLastDDLTimestampForTableFromCacheOnly which will + // read the last ddl timestamp only from the cache and return null if not present in + // the cache. + assertNull(cache.getLastDDLTimestampForTableFromCacheOnly(null, null, tableNameBytes)); + } + } + + /** + * Make sure we are invalidating the cache for table with no tenant connection, no schema name + * and valid table name when we run update index statement. + * @throws Exception + */ + @Test + public void testInvalidateCacheForBaseTableWithUpdateIndexStatement() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url = QueryUtil.getConnectionUrl(props, config, "client"); + String tableNameStr = "TBL_" + generateUniqueName(); + String indexNameStr = "IND_" + generateUniqueName(); + byte[] indexNameBytes = Bytes.toBytes(indexNameStr); + PTable indexTable; + ServerMetadataCacheTestImpl cache = getServerMetadataCache();; + try (Connection conn = DriverManager.getConnection(url, props)) { + conn.setAutoCommit(false); + // Create a test table. + createTable(conn, tableNameStr); + String indexDDLStmt = "CREATE INDEX " + indexNameStr + " ON " + tableNameStr + "(v1)"; + conn.createStatement().execute(indexDDLStmt); + TestUtil.waitForIndexState(conn, indexNameStr, PIndexState.ACTIVE); + indexTable = PhoenixRuntime.getTableNoCache(conn, indexNameStr); + long lastDDLTimestamp = indexTable.getLastDDLTimestamp(); + assertEquals(lastDDLTimestamp, + cache.getLastDDLTimestampForTable(null, null, indexNameBytes)); + Thread.sleep(1); + // Disable an index. This should change the LAST_DDL_TIMESTAMP. + String disableIndexDDL = "ALTER INDEX " + indexNameStr + " ON " + tableNameStr + + " DISABLE"; + conn.createStatement().execute(disableIndexDDL); + TestUtil.waitForIndexState(conn, indexNameStr, PIndexState.DISABLE); + // The above alter index statement will invalidate the last ddl timestamp from metadata + // cache. Notice that we are using cache#getLastDDLTimestampForTableFromCacheOnly which + // will read the last ddl timestamp only from the cache and return null if not present + // in the cache. + assertNull(cache.getLastDDLTimestampForTableFromCacheOnly(null, null, indexNameBytes)); + // This will load the cache with the latest last ddl timestamp value. + long lastDDLTimestampAfterUpdateIndexStmt = cache.getLastDDLTimestampForTable(null, + null, indexNameBytes); + assertNotNull(lastDDLTimestampAfterUpdateIndexStmt); + // Make sure that the last ddl timestamp value after ALTER statement + // is greater than previous one. + assertTrue(lastDDLTimestampAfterUpdateIndexStmt > lastDDLTimestamp); + } + } + + /** + * Test that we invalidate the cache for parent table and update the last ddl timestamp + * of the parent table while we add an index. + * Test that we invalidate the cache for parent table and index when we drop an index. + * Also we update the last ddl timestamp for parent table when we drop an index. + * @throws Exception + */ + @Test + public void testUpdateLastDDLTimestampTableAfterIndexCreation() throws Exception { + String tableName = generateUniqueName(); + byte[] tableNameBytes = Bytes.toBytes(tableName); + String indexName = generateUniqueName(); + byte[] indexNameBytes = Bytes.toBytes(indexName); + ServerMetadataCacheTestImpl cache = getServerMetadataCache();; + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.setAutoCommit(true); + createTable(conn, tableName); + long tableLastDDLTimestampBeforeIndexCreation = getLastDDLTimestamp(tableName); + // Populate the cache + assertNotNull(cache.getLastDDLTimestampForTable(null, null, tableNameBytes)); + Thread.sleep(1); + createIndex(conn, tableName, indexName, "v1"); + // Make sure that we have invalidated the last ddl timestamp for parent table + // on all regionservers after we create an index. + assertNull(cache.getLastDDLTimestampForTableFromCacheOnly(null, null, tableNameBytes)); + long tableLastDDLTimestampAfterIndexCreation = getLastDDLTimestamp(tableName); + assertNotNull(tableLastDDLTimestampAfterIndexCreation); + assertTrue(tableLastDDLTimestampAfterIndexCreation > + tableLastDDLTimestampBeforeIndexCreation); + long indexLastDDLTimestampAfterCreation = getLastDDLTimestamp(indexName); + // Make sure that last ddl timestamp is cached on the regionserver. + assertNotNull(indexLastDDLTimestampAfterCreation); + // Adding a sleep for 1 ms so that we get new last ddl timestamp. + Thread.sleep(1); + dropIndex(conn, tableName, indexName); + // Make sure that we invalidate the cache on regionserver for base table and an index + // after we dropped an index. + assertNull(cache.getLastDDLTimestampForTableFromCacheOnly(null, null, tableNameBytes)); + assertNull(cache.getLastDDLTimestampForTableFromCacheOnly(null, null, indexNameBytes)); + long tableLastDDLTimestampAfterIndexDeletion = getLastDDLTimestamp(tableName); + // Verify that last ddl timestamp after index deletion is greater than + // the previous last ddl timestamp. + assertNotNull(tableLastDDLTimestampAfterIndexDeletion); + assertTrue(tableLastDDLTimestampAfterIndexDeletion > + tableLastDDLTimestampAfterIndexCreation); + } + } + + /** + * Test that we invalidate the cache of the immediate parent view + * and update the last ddl timestamp of the immediate parent view while we add an index. + * Test that we invalidate the cache for parent view and view index when we drop an index. + * Also we update the last ddl timestamp for parent view when we drop an index. + * @throws Exception + */ + @Test + public void testUpdateLastDDLTimestampViewAfterIndexCreation() throws Exception { + String tableName = "T_" + generateUniqueName(); + String globalViewName = "GV_" + generateUniqueName(); + byte[] globalViewNameBytes = Bytes.toBytes(globalViewName); + String globalViewIndexName = "GV_IDX_" + generateUniqueName(); + byte[] globalViewIndexNameBytes = Bytes.toBytes(globalViewIndexName); + + ServerMetadataCacheTestImpl cache = getServerMetadataCache();; + try(Connection conn = DriverManager.getConnection(getUrl()); + Statement stmt = conn.createStatement()) { + String whereClause = " WHERE v1 < 1000"; + createTable(conn, tableName); + createViewWhereClause(conn, tableName, globalViewName, whereClause); + // Populate the cache + assertNotNull(cache.getLastDDLTimestampForTable(null, null, globalViewNameBytes)); + long viewLastDDLTimestampBeforeIndexCreation = getLastDDLTimestamp(globalViewName); + createIndex(conn, globalViewName, globalViewIndexName, "v1"); + + // Make sure that we have invalidated the last ddl timestamp for parent global view + // on all regionserver after we create a view index. + assertNull(cache.getLastDDLTimestampForTableFromCacheOnly(null, null, + globalViewNameBytes)); + long viewLastDDLTimestampAfterIndexCreation = getLastDDLTimestamp(globalViewName); + assertTrue(viewLastDDLTimestampAfterIndexCreation > + viewLastDDLTimestampBeforeIndexCreation); + long indexLastDDLTimestampAfterCreation = getLastDDLTimestamp(globalViewIndexName); + // Make sure that last ddl timestamp is cached on the regionserver. + assertNotNull(indexLastDDLTimestampAfterCreation); + // Adding a sleep for 1 ms so that we get new last ddl timestamp. + Thread.sleep(1); + dropIndex(conn, globalViewName, globalViewIndexName); + // Make sure that we invalidate the cache on regionservers for view and its index after + // we drop a view index. + assertNull(cache.getLastDDLTimestampForTableFromCacheOnly(null, null, + globalViewNameBytes)); + assertNull(cache.getLastDDLTimestampForTableFromCacheOnly(null, null, + globalViewIndexNameBytes)); + long viewLastDDLTimestampAfterIndexDeletion = getLastDDLTimestamp(globalViewName); + // Verify that last ddl timestamp of view after index deletion is greater than + // the previous last ddl timestamp. + assertNotNull(viewLastDDLTimestampAfterIndexDeletion); + assertTrue(viewLastDDLTimestampAfterIndexDeletion > + viewLastDDLTimestampAfterIndexCreation); + } + } + + /** + * Client-1 creates a table, upserts data and alters the table. + * Client-2 queries the table before and after the alter. + * Check queries work successfully in both cases and verify number of addTable invocations. + */ + @Test + public void testSelectQueryWithOldDDLTimestamp() throws SQLException { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + int expectedNumCacheUpdates; + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + // create table with UCF=never and upsert data using client-1 + createTable(conn1, tableName); + upsert(conn1, tableName, true); + + // select query from client-2 works to populate client side metadata cache + // there should be 1 update to the client cache + query(conn2, tableName); + expectedNumCacheUpdates = 1; + Mockito.verify(spyCqs2, Mockito.times(expectedNumCacheUpdates)) + .addTable(any(PTable.class), anyLong()); + + // add column using client-1 to update last ddl timestamp + alterTableAddColumn(conn1, tableName, "newCol1"); + + // reset the spy CQSI object + Mockito.reset(spyCqs2); + + // select query from client-2 with old ddl timestamp works + // there should be one update to the client cache + //verify client got a StaleMetadataCacheException + query(conn2, tableName); + expectedNumCacheUpdates = 1; + Mockito.verify(spyCqs2, Mockito.times(expectedNumCacheUpdates)) + .addTable(any(PTable.class), anyLong()); + Assert.assertEquals("Client should have encountered a StaleMetadataCacheException", + 1, GlobalClientMetrics.GLOBAL_CLIENT_STALE_METADATA_CACHE_EXCEPTION_COUNTER.getMetric().getValue()); + + // select query from client-2 with latest ddl timestamp works + // there should be no more updates to client cache + //verify client did not get another StaleMetadataCacheException + query(conn2, tableName); + Mockito.verify(spyCqs2, Mockito.times(expectedNumCacheUpdates)) + .addTable(any(PTable.class), anyLong()); + Assert.assertEquals("Client should have encountered a StaleMetadataCacheException", + 1, GlobalClientMetrics.GLOBAL_CLIENT_STALE_METADATA_CACHE_EXCEPTION_COUNTER.getMetric().getValue()); + } + } + + /** + * Test DDL timestamp validation retry logic in case of any exception + * from Server other than StaleMetadataCacheException. + */ + @Test + public void testSelectQueryServerSideExceptionInValidation() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + ServerMetadataCacheTestImpl cache = null; + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + // create table and upsert using client-1 + createTable(conn1, tableName); + upsert(conn1, tableName, true); + + // Instrument ServerMetadataCache to throw a SQLException once + cache = getServerMetadataCache();; + ServerMetadataCacheTestImpl spyCache = Mockito.spy(cache); + Mockito.doThrow(new SQLException("FAIL")).doCallRealMethod().when(spyCache) + .getLastDDLTimestampForTable(any(), any(), eq(Bytes.toBytes(tableName))); + ServerMetadataCacheTestImpl.setInstance(serverName, spyCache); + + // query using client-2 should succeed + query(conn2, tableName); + + // verify live region servers were refreshed + Mockito.verify(spyCqs2, Mockito.times(1)).refreshLiveRegionServers(); + } + } + + /** + * Test Select query works when ddl timestamp validation with old timestamp encounters an exception. + * Verify that the list of live region servers was refreshed when ddl timestamp validation is retried. + * Verify that the client cache was updated after encountering StaleMetadataCacheException. + */ + @Test + public void testSelectQueryWithOldDDLTimestampWithExceptionRetry() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + int expectedNumCacheUpdates; + ServerMetadataCacheTestImpl cache = null; + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + // create table and upsert using client-1 + createTable(conn1, tableName); + upsert(conn1, tableName, true); + + // query using client-2 to populate cache + query(conn2, tableName); + expectedNumCacheUpdates = 1; + Mockito.verify(spyCqs2, Mockito.times(expectedNumCacheUpdates)) + .addTable(any(PTable.class), anyLong()); + + // add column using client-1 to update last ddl timestamp + alterTableAddColumn(conn1, tableName, "newCol1"); + + // reset the spy CQSI object + Mockito.reset(spyCqs2); + + // Instrument ServerMetadataCache to throw a SQLException once + cache = getServerMetadataCache();; + ServerMetadataCacheTestImpl spyCache = Mockito.spy(cache); + Mockito.doThrow(new SQLException("FAIL")).doCallRealMethod().when(spyCache) + .getLastDDLTimestampForTable(any(), any(), eq(Bytes.toBytes(tableName))); + ServerMetadataCacheTestImpl.setInstance(serverName, spyCache); + + // query using client-2 should succeed, one cache update + query(conn2, tableName); + expectedNumCacheUpdates = 1; + Mockito.verify(spyCqs2, Mockito.times(expectedNumCacheUpdates)) + .addTable(any(PTable.class), anyLong()); + + // verify live region servers were refreshed + Mockito.verify(spyCqs2, Mockito.times(1)).refreshLiveRegionServers(); + } + } + + /** + * Test Select Query fails in case DDL timestamp validation throws SQLException twice. + */ + @Test + public void testSelectQueryFails() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + ServerMetadataCacheTestImpl cache = null; + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + // create table and upsert using client-1 + createTable(conn1, tableName); + upsert(conn1, tableName, true); + + // Instrument ServerMetadataCache to throw a SQLException twice + cache = getServerMetadataCache();; + ServerMetadataCacheTestImpl spyCache = Mockito.spy(cache); + SQLException e = new SQLException("FAIL"); + Mockito.doThrow(e).when(spyCache) + .getLastDDLTimestampForTable(any(), any(), eq(Bytes.toBytes(tableName))); + ServerMetadataCacheTestImpl.setInstance(serverName, spyCache); + + // query using client-2 should fail + query(conn2, tableName); + Assert.fail("Query should have thrown Exception"); + } + catch (Exception e) { + Assert.assertTrue("SQLException was not thrown when last ddl timestamp validation encountered errors twice.", e instanceof SQLException); + } + } + + + /** + * Client-1 creates a table, 2 level of views on it and alters the first level view. + * Client-2 queries the second level view, verify that there were 3 cache updates in client-2, + * one each for the two views and base table. + */ + @Test + public void testSelectQueryOnView() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + int expectedNumCacheUpdates; + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + // create table using client-1 + createTable(conn1, tableName); + upsert(conn1, tableName, true); + + // create 2 level of views using client-1 + String view1 = generateUniqueName(); + String view2 = generateUniqueName(); + createView(conn1, tableName, view1); + createView(conn1, view1, view2); + + // query second level view using client-2 + query(conn2, view2); + expectedNumCacheUpdates = 3; // table, view1, view2 + Mockito.verify(spyCqs2, Mockito.times(expectedNumCacheUpdates)) + .addTable(any(PTable.class), anyLong()); + + // alter first level view using client-1 to update its last ddl timestamp + alterViewAddColumn(conn1, view1, "foo"); + + // reset the spy CQSI object + Mockito.reset(spyCqs2); + + // query second level view + query(conn2, view2); + + // verify there was a getTable RPC for the view and all its ancestors + Mockito.verify(spyCqs2, Mockito.times(1)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(tableName)), + anyLong(), anyLong()); + Mockito.verify(spyCqs2, Mockito.times(1)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(view1)), + anyLong(), anyLong()); + Mockito.verify(spyCqs2, Mockito.times(1)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(view2)), + anyLong(), anyLong()); + + // verify that the view and all its ancestors were updated in the client cache + expectedNumCacheUpdates = 3; // table, view1, view2 + Mockito.verify(spyCqs2, Mockito.times(expectedNumCacheUpdates)) + .addTable(any(PTable.class), anyLong()); + } + } + + /** + * Verify queries on system tables work as we will validate last ddl timestamps for them also. + */ + @Test + public void testSelectQueryOnSystemTables() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url = QueryUtil.getConnectionUrl(props, config, "client"); + ConnectionQueryServices cqs = driver.getConnectionQueryServices(url, props); + + try (Connection conn = cqs.connect(url, props)) { + query(conn, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME); + query(conn, PhoenixDatabaseMetaData.SYSTEM_TASK_NAME); + query(conn, PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME); + query(conn, PhoenixDatabaseMetaData.SYSTEM_LOG_NAME); + } + } + + /** + * https://issues.apache.org/jira/browse/PHOENIX-7167 + * Use the default connection to query system tables to confirm + * that the PTable object for SYSTEM tables is correctly bootstrapped. + */ + @Test + public void testSystemTablesBootstrap() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url = QueryUtil.getConnectionUrl(props, config); + ConnectionQueryServices cqs = driver.getConnectionQueryServices(url, props); + + try (Connection conn = cqs.connect(url, props)) { + query(conn, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME); + query(conn, PhoenixDatabaseMetaData.SYSTEM_TASK_NAME); + query(conn, PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME); + query(conn, PhoenixDatabaseMetaData.SYSTEM_LOG_NAME); + } + } + + /** + * Test that a client does not see TableNotFoundException when trying to validate + * LAST_DDL_TIMESTAMP for a view and its parent after the table was altered and removed from + * the client's cache. + */ + @Test + public void testQueryViewAfterParentRemovedFromCache() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url = QueryUtil.getConnectionUrl(props, config); + ConnectionQueryServices cqs = driver.getConnectionQueryServices(url, props); + String tableName = generateUniqueName(); + String viewName = generateUniqueName(); + try (Connection conn = cqs.connect(url, props)) { + createTable(conn, tableName); + createView(conn, tableName, viewName); + query(conn, viewName); + // this removes the parent table from the client cache + alterTableDropColumn(conn, tableName, "v2"); + query(conn, viewName); + } catch (TableNotFoundException e) { + fail("TableNotFoundException should not be encountered by client."); + } + } + + /** + * Test query on index with stale last ddl timestamp. + * Client-1 creates a table and an index on it. Client-2 queries table to populate its cache. + * Client-1 alters a property on the index. Client-2 queries the table again. + * Verify that the second query works and the index metadata was updated in the client cache. + */ + @Test + public void testSelectQueryAfterAlterIndex() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + String indexName = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + //client-1 creates a table and an index on it + createTable(conn1, tableName); + createIndex(conn1, tableName, indexName, "v1"); + TestUtil.waitForIndexState(conn1, indexName, PIndexState.ACTIVE); + + //client-2 populates its cache, 1 getTable and 1 addTable call for the table + query(conn2, tableName); + Mockito.verify(spyCqs2, Mockito.times(1)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(tableName)), + anyLong(), anyLong()); + Mockito.verify(spyCqs2, Mockito.times(1)) + .addTable(any(PTable.class), anyLong()); + + //client-1 updates index property + alterIndexChangeState(conn1, tableName, indexName, " REBUILD"); + + //client-2's query using the index should work + PhoenixStatement stmt = conn2.createStatement().unwrap(PhoenixStatement.class); + stmt.executeQuery("SELECT k FROM " + tableName + " WHERE v1=1"); + Assert.assertEquals("Query on secondary key should have used index.", indexName, stmt.getQueryPlan().getTableRef().getTable().getTableName().toString()); + + //verify client-2 cache was updated with the index and base table metadata + Mockito.verify(spyCqs2, Mockito.times(2)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(tableName)), + anyLong(), anyLong()); + Mockito.verify(spyCqs2, Mockito.times(1)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(indexName)), + anyLong(), anyLong()); + Mockito.verify(spyCqs2, Mockito.times(3)) + .addTable(any(PTable.class), anyLong()); + + //client-2 queries again with latest metadata + //verify no more getTable/addTable calls + queryWithIndex(conn2, tableName); + Mockito.verify(spyCqs2, Mockito.times(2)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(tableName)), + anyLong(), anyLong()); + Mockito.verify(spyCqs2, Mockito.times(3)) + .addTable(any(PTable.class), anyLong()); + } + } + + /** + * Test that a client can learn about a newly created index. + * Client-1 creates a table, client-2 queries the table to populate its cache. + * Client-1 creates an index on the table. Client-2 queries the table using the index. + * Verify that client-2 uses the index for the query. + */ + @Test + public void testSelectQueryAddIndex() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + String indexName = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + //client-1 creates table + createTable(conn1, tableName); + + //client-2 populates its cache + query(conn2, tableName); + + //client-1 creates an index on the table + createIndex(conn1, tableName, indexName, "v1"); + TestUtil.waitForIndexState(conn1, indexName, PIndexState.ACTIVE); + + //client-2 query should be able to use this index + PhoenixStatement stmt = conn2.createStatement().unwrap(PhoenixStatement.class); + ResultSet rs = stmt.executeQuery("SELECT k FROM " + tableName + " WHERE v1=1"); + Assert.assertEquals("Query on secondary key should have used index.", indexName, stmt.getQueryPlan().getContext().getCurrentTable().getTable().getName().getString()); + } + } + + /** + * Test that a client can learn about a dropped index. + * Client-1 creates a table and an index, client-2 queries the table to populate its cache. + * Client-1 drops the index. Client-2 queries the table with index hint. + * Verify that client-2 uses the data table for the query. + */ + @Test + public void testSelectQueryDropIndex() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + String indexName = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + //client-1 creates table and index on it + createTable(conn1, tableName); + createIndex(conn1, tableName, indexName, "v1"); + + //client-2 populates its cache + query(conn2, tableName); + + //client-1 drops the index + dropIndex(conn1, tableName, indexName); + + //client-2 queries should use data table and not run into table not found error even when index hint is given + PhoenixStatement stmt = conn2.createStatement().unwrap(PhoenixStatement.class); + ResultSet rs = stmt.executeQuery("SELECT /*+ INDEX(" + tableName + " " + indexName + ") */ * FROM " + tableName + " WHERE v1=1"); + Assert.assertEquals("Query should have used data table since index was dropped", tableName, stmt.getQueryPlan().getContext().getCurrentTable().getTable().getName().getString()); + } + } + + /** + * Test the case when a client upserts into multiple tables before calling commit. + * Verify that last ddl timestamp was validated for all involved tables only once. + */ + @Test + public void testUpsertMultipleTablesWithOldDDLTimestamp() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName1 = generateUniqueName(); + String tableName2 = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + //client-1 creates 2 tables + createTable(conn1, tableName1); + createTable(conn1, tableName2); + + //client-2 populates its cache, 1 getTable call for each table + query(conn2, tableName1); + query(conn2, tableName2); + + //client-1 alters one of the tables + alterTableAddColumn(conn1, tableName2, "col3"); + + //client-2 upserts multiple rows to both tables before calling commit + //verify the table metadata was fetched for each table + multiTableUpsert(conn2, tableName1, tableName2); + Mockito.verify(spyCqs2, Mockito.times(2)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(tableName1)), + anyLong(), anyLong()); + Mockito.verify(spyCqs2, Mockito.times(2)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(tableName2)), + anyLong(), anyLong()); + } + } + + /** + * Test upserts into a multi-level view hierarchy. + */ + @Test + public void testUpsertViewWithOldDDLTimestamp() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + String viewName1 = generateUniqueName(); + String viewName2 = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + //client-1 creates a table and views + createTable(conn1, tableName); + createView(conn1, tableName, viewName1); + createView(conn1, viewName1, viewName2); + + //client-2 populates its cache, 1 getTable RPC each for table, view1, view2 + query(conn2, viewName2); + Mockito.verify(spyCqs2, Mockito.times(1)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(tableName)), + anyLong(), anyLong()); + Mockito.verify(spyCqs2, Mockito.times(1)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(viewName1)), + anyLong(), anyLong()); + Mockito.verify(spyCqs2, Mockito.times(1)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(viewName2)), + anyLong(), anyLong()); + + //client-1 alters first level view + alterViewAddColumn(conn1, viewName1, "col3"); + + //client-2 upserts into second level view + //verify there was a getTable RPC for the view and all its ancestors + //verify that the client got a StaleMetadataCacheException + upsert(conn2, viewName2, true); + + Mockito.verify(spyCqs2, Mockito.times(2)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(tableName)), + anyLong(), anyLong()); + Mockito.verify(spyCqs2, Mockito.times(2)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(viewName1)), + anyLong(), anyLong()); + Mockito.verify(spyCqs2, Mockito.times(2)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(viewName2)), + anyLong(), anyLong()); + Assert.assertEquals("Client should have encountered a StaleMetadataCacheException", + 1, GlobalClientMetrics.GLOBAL_CLIENT_STALE_METADATA_CACHE_EXCEPTION_COUNTER.getMetric().getValue()); + //client-2 upserts into first level view + //verify no getTable RPCs + //verify that the client did not get a StaleMetadataCacheException + upsert(conn2, viewName1, true); + + Mockito.verify(spyCqs2, Mockito.times(2)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(tableName)), + anyLong(), anyLong()); + Mockito.verify(spyCqs2, Mockito.times(2)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(viewName1)), + anyLong(), anyLong()); + Mockito.verify(spyCqs2, Mockito.times(2)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(viewName2)), + anyLong(), anyLong()); + Assert.assertEquals("Client should not have encountered another StaleMetadataCacheException", + 1, GlobalClientMetrics.GLOBAL_CLIENT_STALE_METADATA_CACHE_EXCEPTION_COUNTER.getMetric().getValue()); + } + } + + /** + * Test that upserts into a table which was dropped throws a TableNotFoundException. + */ + @Test + public void testUpsertDroppedTable() throws SQLException { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + // client-1 creates tables and executes upserts + createTable(conn1, tableName); + upsert(conn1, tableName, false); + upsert(conn1, tableName, false); + upsert(conn1, tableName, false); + + // client-2 drops the table + conn2.createStatement().execute("DROP TABLE " + tableName); + + //client-1 commits + conn1.commit(); + Assert.fail("Commit should have failed with TableNotFoundException"); + } + catch (Exception e) { + Assert.assertTrue("TableNotFoundException was not thrown when table was dropped concurrently with upserts.", e instanceof TableNotFoundException); + } + } + + /** + * Client-1 creates a table and executes some upserts. + * Client-2 drops a column for which client-1 had executed upserts. + * Client-1 calls commit. Verify that client-1 gets ColumnNotFoundException + */ + @Test + public void testUpsertDroppedTableColumn() throws SQLException { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + // client-1 creates tables and executes upserts + createTable(conn1, tableName); + upsert(conn1, tableName, false); + upsert(conn1, tableName, false); + upsert(conn1, tableName, false); + + // client-2 drops a column + alterTableDropColumn(conn2, tableName, "v1"); + + //client-1 commits + conn1.commit(); + Assert.fail("Commit should have failed with ColumnNotFoundException"); + } + catch (Exception e) { + Assert.assertTrue("ColumnNotFoundException was not thrown when column was dropped concurrently with upserts.", e instanceof ColumnNotFoundException); + } + } + + /** + * Client-1 creates a table and executes some upserts. + * Client-2 adds a column to the table. + * Client-1 calls commit. Verify that client-1 does not get any errors. + */ + @Test + public void testUpsertAddTableColumn() throws SQLException { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + // client-1 creates tables and executes upserts + createTable(conn1, tableName); + upsert(conn1, tableName, false); + upsert(conn1, tableName, false); + upsert(conn1, tableName, false); + + // client-2 adds a column + alterTableAddColumn(conn2, tableName, "v5"); + + //client-1 commits + conn1.commit(); + } + } + + /** + * Client-1 creates a table and executes some upserts. + * Client-2 creates an index on the table. + * Client-1 calls commit. Verify that index mutations were correctly generated + */ + @Test + public void testConcurrentUpsertIndexCreation() throws SQLException { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + String indexName = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + // client-1 creates tables and executes upserts + createTable(conn1, tableName); + upsert(conn1, tableName, false); + upsert(conn1, tableName, false); + upsert(conn1, tableName, false); + + // client-2 creates an index + createIndex(conn2, tableName, indexName, "v1"); + + //client-1 commits + upsert(conn1, tableName, false); + upsert(conn1, tableName, false); + conn1.commit(); + + //verify index rows + int tableCount, indexCount; + ResultSet rs = conn1.createStatement().executeQuery("SELECT COUNT(*) FROM " + tableName); + rs.next(); + tableCount = rs.getInt(1); + + rs = conn1.createStatement().executeQuery("SELECT COUNT(*) FROM " + indexName); + rs.next(); + indexCount = rs.getInt(1); + + Assert.assertEquals("All index mutations were not generated when index was created concurrently with upserts.", tableCount, indexCount); + } + } + + /** + * Client-1 creates a table, index and executes some upserts. + * Client-2 drops the index on the table. + * Client-1 calls commit. Verify that client-1 does not see any errors + */ + @Test + public void testConcurrentUpsertDropIndex() throws SQLException { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + String indexName = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + // client-1 creates tables, index and executes upserts + createTable(conn1, tableName); + createIndex(conn1, tableName, indexName, "v1"); + upsert(conn1, tableName, false); + upsert(conn1, tableName, false); + upsert(conn1, tableName, false); + + // client-2 drops the index + dropIndex(conn2, tableName, indexName); + + //client-1 commits + upsert(conn1, tableName, false); + upsert(conn1, tableName, false); + conn1.commit(); + } + } + /** + * Client-1 creates a table, index in disabled state and executes some upserts. + * Client-2 marks the index as Rebuild. + * Client-1 calls commit. Verify that index mutations were correctly generated + */ + @Test + public void testConcurrentUpsertIndexStateChange() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + String indexName = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + // client-1 creates tables and executes upserts + createTable(conn1, tableName); + createIndex(conn1, tableName, indexName, "v1"); + alterIndexChangeState(conn1, tableName, indexName, " DISABLE"); + upsert(conn1, tableName, false); + upsert(conn1, tableName, false); + upsert(conn1, tableName, false); + + // client-2 creates an index + alterIndexChangeState(conn2, tableName, indexName, " REBUILD"); + + //client-1 commits + upsert(conn1, tableName, false); + upsert(conn1, tableName, false); + conn1.commit(); + + //verify index rows + int tableCount, indexCount; + ResultSet rs = conn1.createStatement().executeQuery("SELECT COUNT(*) FROM " + tableName); + rs.next(); + tableCount = rs.getInt(1); + + rs = conn1.createStatement().executeQuery("SELECT COUNT(*) FROM " + indexName); + rs.next(); + indexCount = rs.getInt(1); + + Assert.assertEquals("All index mutations were not generated when index was created concurrently with upserts.", tableCount, indexCount); + } + } + + /** + * Test that a client can not create an index on a column after another client dropped the column. + */ + @Test + public void testClientCannotCreateIndexOnDroppedColumn() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + String indexName = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + createTable(conn1, tableName); + alterTableDropColumn(conn2, tableName, "v2"); + createIndex(conn1, tableName, indexName, "v2"); + fail("Client should not be able to create index on dropped column."); + } + catch (ColumnNotFoundException expected) { + } + } + + /** + * Test that upserts into a view whose parent was dropped throws a TableNotFoundException. + */ + @Test + public void testConcurrentUpsertDropView() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + String viewName1 = generateUniqueName(); + String viewName2 = generateUniqueName(); + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + ConnectionQueryServices spyCqs2 = Mockito.spy(driver.getConnectionQueryServices(url2, props)); + + try (Connection conn1 = spyCqs1.connect(url1, props); + Connection conn2 = spyCqs2.connect(url2, props)) { + + //client-1 creates tables and views + createTable(conn1, tableName); + createView(conn1, tableName, viewName1); + createView(conn1, viewName1, viewName2); + + //client-2 upserts into second level view + upsert(conn2, viewName2, false); + + //client-1 drop first level view + dropView(conn1, viewName1, true); + + //client-2 upserts into second level view and commits + upsert(conn2, viewName2, true); + } + catch (Exception e) { + Assert.assertTrue("TableNotFoundException was not thrown when parent view " + + "was dropped (cascade) concurrently with upserts.", + e instanceof TableNotFoundException); + } + } + + /** + * Test server side metrics are populated correctly. + * Client-1 creates a table and creates an index on it. + * Client-2 queries the table. + */ + @Test + public void testServerSideMetrics() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + String indexName = generateUniqueName(); + ConnectionQueryServices cqs1 = driver.getConnectionQueryServices(url1, props); + ConnectionQueryServices cqs2 = driver.getConnectionQueryServices(url2, props); + MetricsMetadataCachingSource metricsSource + = MetricsPhoenixCoprocessorSourceFactory.getInstance().getMetadataCachingSource(); + + //take a snapshot of current metric values + MetricsMetadataCachingSource.MetadataCachingMetricValues oldMetricValues + = metricsSource.getCurrentMetricValues(); + + long cacheHit = 0; + long cacheMiss = 0; + long validateDDLRequestCount = 0; + long cacheInvOpsCount = 0; + long cacheInvSuccessCount = 0; + long cacheInvFailureCount = 0; + long cacheInvRpcTimeCount = 0; + long cacheInvTotalTimeCount = 0; + + try (Connection conn1 = cqs1.connect(url1, props); + Connection conn2 = cqs2.connect(url2, props)) { + + // no metric changes + createTable(conn1, tableName); + + // client validates table, regionserver does not find table in its cache + query(conn2, tableName); + validateDDLRequestCount++; + cacheMiss++; + + // last_ddl_timestamp is bumped for the table + // cache invalidation operation succeeds for table + // cache invalidation operation succeeds for index state change + // only one region server in tests for cache invalidation RPC + createIndex(conn1, tableName, indexName, "v1"); + cacheInvOpsCount += 2; + cacheInvRpcTimeCount += 2; + cacheInvTotalTimeCount += 2; + cacheInvSuccessCount += 2; + + // client validates only table since it does not know about the index yet + // regionserver does not find table in its cache + query(conn2, tableName); + validateDDLRequestCount++; + cacheMiss++; + + // client validates both index and table this time + // regionserver finds table but does not find index in its cache + query(conn2, tableName); + validateDDLRequestCount++; + cacheHit++; //table + cacheMiss++; //index + + // client validates index and table again + // regionserver finds both index and table in its cache + query(conn2, tableName); + validateDDLRequestCount++; + cacheHit += 2; + + MetricsMetadataCachingSource.MetadataCachingMetricValues newMetricValues + = metricsSource.getCurrentMetricValues(); + + assertEquals("Incorrect number of cache hits on region server.", cacheHit, + newMetricValues.getCacheHitCount() - oldMetricValues.getCacheHitCount()); + + assertEquals("Incorrect number of cache misses on region server.", cacheMiss, + newMetricValues.getCacheMissCount() - oldMetricValues.getCacheMissCount()); + + assertEquals("Incorrect number of validate ddl timestamp requests.", + validateDDLRequestCount, + newMetricValues.getValidateDDLTimestampRequestsCount() + - oldMetricValues.getValidateDDLTimestampRequestsCount()); + + assertEquals("Incorrect number of cache invalidation ops count.", + cacheInvOpsCount, + newMetricValues.getCacheInvalidationOpsCount() + - oldMetricValues.getCacheInvalidationOpsCount()); + + assertEquals("Incorrect number of successful cache invalidation ops count.", + cacheInvSuccessCount, + newMetricValues.getCacheInvalidationSuccessCount() + - oldMetricValues.getCacheInvalidationSuccessCount()); + + assertEquals("Incorrect number of failed cache invalidation ops count.", + cacheInvFailureCount, + newMetricValues.getCacheInvalidationFailureCount() + - oldMetricValues.getCacheInvalidationFailureCount()); + + assertEquals("Incorrect number of cache invalidation RPC times.", + cacheInvRpcTimeCount, + newMetricValues.getCacheInvalidationRpcTimeCount() + - oldMetricValues.getCacheInvalidationRpcTimeCount()); + + assertEquals("Incorrect number of cache invalidation total times.", + cacheInvTotalTimeCount, + newMetricValues.getCacheInvalidationTotalTimeCount() + - oldMetricValues.getCacheInvalidationTotalTimeCount()); + } + } + + /* + Tests that invalidate server metadata cache fails on a non server connection. + */ + @Test + public void testInvalidateMetadataCacheOnNonServerConnection() { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + try (PhoenixConnection conn = DriverManager.getConnection(getUrl(), props) + .unwrap(PhoenixConnection.class)) { + ConnectionQueryServices cqs = conn.getQueryServices(); + cqs.invalidateServerMetadataCache(null); + fail("Shouldn't come here"); + } catch (Throwable t) { + assertNotNull(t); + assertTrue(t.getMessage().contains(INVALIDATE_SERVER_METADATA_CACHE_EX_MESSAGE)); + } + } + + /** + * Test that a query on the column of a view which was previously dropped + * throws a ColumnNotFoundException. Use the same client to drop the column. + */ + @Test + public void testDroppedTableColumnNotVisibleToViewUsingSameClient() throws Exception { + testDroppedTableColumnNotVisibleToView(true); + } + + /** + * Test that a query on the column of a view which was previously dropped + * throws a ColumnNotFoundException. Use a different client to drop the column. + */ + @Test + public void testDroppedTableColumnNotVisibleToViewUsingDifferentClients() throws Exception { + testDroppedTableColumnNotVisibleToView(false); + } + + public void testDroppedTableColumnNotVisibleToView(boolean useSameClient) throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + String tableName = generateUniqueName(); + String viewName1 = generateUniqueName(); + String viewName2 = generateUniqueName(); + ConnectionQueryServices cqs1 = driver.getConnectionQueryServices(url1, props); + ConnectionQueryServices cqs2 = driver.getConnectionQueryServices(url2, props); + try (Connection conn = cqs1.connect(url1, props); + Connection conn2 = useSameClient ? conn : cqs2.connect(url2, props)) { + createTable(conn, tableName); + createView(conn, tableName, viewName1); + createView(conn, viewName1, viewName2); + query(conn2, viewName2); + + alterTableDropColumn(conn, tableName, "v2"); + query(conn2, tableName); + + conn2.createStatement().execute("SELECT v2 FROM " + viewName2); + fail("Column dropped from base table should not be visible to view."); + } catch (ColumnNotFoundException expected) { + } + } + + /** + * Test that ancestor->last_ddl_timestamp is populated in a new client. + * @throws Exception + */ + @Test + public void testAncestorLastDDLMapPopulatedInDifferentClient() throws Exception { + String SCHEMA1 = generateUniqueName(); + String SCHEMA2 = generateUniqueName(); + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String baseTable = SchemaUtil.getTableName(SCHEMA1, generateUniqueName()); + String index = generateUniqueName(); + String view = SchemaUtil.getTableName(SCHEMA2, generateUniqueName()); + String viewIndex = generateUniqueName(); + String baseTable2 = SchemaUtil.getTableName(SCHEMA1, generateUniqueName()); + String index2 = generateUniqueName(); + String view2 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName()); + String viewIndex2 = generateUniqueName(); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String url2 = QueryUtil.getConnectionUrl(props, config, "client2"); + ConnectionQueryServices cqs1 = driver.getConnectionQueryServices(url1, props); + ConnectionQueryServices cqs2 = driver.getConnectionQueryServices(url2, props); + try (Connection conn = cqs1.connect(url1, props); + Connection conn2 = cqs2.connect(url2, props)) { + //client-1 creates tables, views, indexes and view indexes + createTable(conn, baseTable); + createView(conn, baseTable, view); + createIndex(conn, baseTable, index, "v2"); + createIndex(conn, view, viewIndex, "v1"); + createTable(conn, baseTable2); + createView(conn, baseTable2, view2); + createIndex(conn, baseTable2, index2, "v2"); + createIndex(conn, view2, viewIndex2, "v1"); + + //client-2 queries the view + query(conn2, view); + + PTable basePTable = PhoenixRuntime.getTable(conn2, baseTable); + PTable viewPTable = PhoenixRuntime.getTable(conn2, view); + PTable viewIndexPTable = PhoenixRuntime.getTable(conn2, SchemaUtil.getTableName(SCHEMA2, viewIndex)); + PTable indexPTable = PhoenixRuntime.getTable(conn2, SchemaUtil.getTableName(SCHEMA1, index)); + + //verify view has base table in ancestor map + Map map = viewPTable.getAncestorLastDDLTimestampMap(); + assertEquals(basePTable.getLastDDLTimestamp(), map.get(basePTable.getKey())); + + //verify view index has base table and view in ancestor map + map = viewIndexPTable.getAncestorLastDDLTimestampMap(); + assertEquals(2, map.size()); + assertEquals(basePTable.getLastDDLTimestamp(), map.get(basePTable.getKey())); + assertEquals(viewPTable.getLastDDLTimestamp(), map.get(viewPTable.getKey())); + + //verify index has only base table in ancestor map + map = indexPTable.getAncestorLastDDLTimestampMap(); + assertEquals(1, map.size()); + assertEquals(basePTable.getLastDDLTimestamp(), map.get(basePTable.getKey())); + + //also verify index PTable within base table has the map + assertEquals(1, basePTable.getIndexes().size()); + map = basePTable.getIndexes().get(0).getAncestorLastDDLTimestampMap(); + assertEquals(1, map.size()); + assertEquals(basePTable.getLastDDLTimestamp(), map.get(basePTable.getKey())); + + //verify client-2 sees maps directly through PhoenixRuntime, no query on baseTable2 or view2 + PTable basePTable2 = PhoenixRuntime.getTable(conn2, baseTable2); + map = basePTable2.getAncestorLastDDLTimestampMap(); + assertEquals(0, map.size()); + assertEquals(1, basePTable2.getIndexes().size()); + map = basePTable2.getIndexes().get(0).getAncestorLastDDLTimestampMap(); + assertEquals(basePTable2.getLastDDLTimestamp(), map.get(basePTable2.getKey())); + + PTable viewPTable2 = PhoenixRuntime.getTable(conn2, view2); + map = viewPTable2.getAncestorLastDDLTimestampMap(); + assertEquals(basePTable2.getLastDDLTimestamp(), map.get(basePTable2.getKey())); + assertEquals(2, viewPTable2.getIndexes().size()); + for (PTable indexOfView : viewPTable2.getIndexes()) { + // inherited index + if (indexOfView.getTableName().getString().equals(index2)) { + map = indexOfView.getAncestorLastDDLTimestampMap(); + assertEquals(basePTable2.getLastDDLTimestamp(), map.get(basePTable2.getKey())); + } else { + // view index + map = indexOfView.getAncestorLastDDLTimestampMap(); + assertEquals(basePTable2.getLastDDLTimestamp(), map.get(basePTable2.getKey())); + assertEquals(viewPTable2.getLastDDLTimestamp(), map.get(viewPTable2.getKey())); + } + } + } + } + + /** + * Test that tenant connections are able to learn about state change of an inherited index + * on their tenant views with different names. + */ + @Test + public void testInheritedIndexOnTenantViewsDifferentNames() throws Exception { + testInheritedIndexOnTenantViews(false); + } + + /** + * Test that tenant connections are able to learn about state change of an inherited index + * on their tenant views with same names. + */ + @Test + public void testInheritedIndexOnTenantViewsSameNames() throws Exception { + testInheritedIndexOnTenantViews(true); + } + + public void testInheritedIndexOnTenantViews(boolean sameTenantViewNames) throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url = QueryUtil.getConnectionUrl(props, config, "client1"); + ConnectionQueryServices cqs = driver.getConnectionQueryServices(url, props); + String baseTableName = generateUniqueName(); + String globalViewName = generateUniqueName(); + String globalViewIndexName = generateUniqueName(); + String tenantViewName1 = generateUniqueName(); + String tenantViewName2 = sameTenantViewNames ? tenantViewName1 : generateUniqueName(); + try (Connection conn = cqs.connect(url, props)) { + // create table, view and view index + conn.createStatement().execute("CREATE TABLE " + baseTableName + + " (TENANT_ID CHAR(9) NOT NULL, KP CHAR(3) NOT NULL, PK CHAR(3) NOT NULL, KV CHAR(2), KV2 CHAR(2) " + + "CONSTRAINT PK PRIMARY KEY(TENANT_ID, KP, PK)) MULTI_TENANT=true,UPDATE_CACHE_FREQUENCY=NEVER"); + conn.createStatement().execute("CREATE VIEW " + globalViewName + + " AS SELECT * FROM " + baseTableName + " WHERE KP = '001'"); + conn.createStatement().execute("CREATE INDEX " + globalViewIndexName + " on " + + globalViewName + " (KV) " + " INCLUDE (KV2) ASYNC"); + String tenantId1 = "tenantId1"; + String tenantId2 = "tenantId2"; + Properties tenantProps1 = PropertiesUtil.deepCopy(TEST_PROPERTIES); + Properties tenantProps2 = PropertiesUtil.deepCopy(TEST_PROPERTIES); + tenantProps1.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId1); + tenantProps2.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId2); + + //create tenant views and upsert one row, this updates all the timestamps in the client's cache + try (Connection tenantConn1 = cqs.connect(url, tenantProps1); + Connection tenantConn2 = cqs.connect(url, tenantProps2)) { + tenantConn1.createStatement().execute("CREATE VIEW " + tenantViewName1 + " AS SELECT * FROM " + globalViewName); + tenantConn1.createStatement().execute("UPSERT INTO " + tenantViewName1 + " (PK, KV, KV2) VALUES " + "('PK1', 'KV', '01')"); + tenantConn1.commit(); + + tenantConn2.createStatement().execute("CREATE VIEW " + tenantViewName2 + " AS SELECT * FROM " + globalViewName); + tenantConn2.createStatement().execute("UPSERT INTO " + tenantViewName2 + " (PK, KV, KV2) VALUES " + "('PK2', 'KV', '02')"); + tenantConn2.commit(); + } + // build global view index + IndexToolIT.runIndexTool(false, "", globalViewName, + globalViewIndexName); + + // query on secondary key should use inherited index for all tenant views. + try (Connection tenantConn1 = cqs.connect(url, tenantProps1); + Connection tenantConn2 = cqs.connect(url, tenantProps2)) { + + String query1 = "SELECT KV2 FROM " + tenantViewName1 + " WHERE KV = 'KV'"; + String query2 = "SELECT KV2 FROM " + tenantViewName2 + " WHERE KV = 'KV'"; + + ResultSet rs = tenantConn1.createStatement().executeQuery(query1); + assertPlan((PhoenixResultSet) rs, "", + tenantViewName1 + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + globalViewIndexName); + assertTrue(rs.next()); + assertEquals("01", rs.getString(1)); + + rs = tenantConn2.createStatement().executeQuery(query2); + assertPlan((PhoenixResultSet) rs, "", + tenantViewName2 + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + globalViewIndexName); + assertTrue(rs.next()); + assertEquals("02", rs.getString(1)); + } + } + } + + /** + * Test that client always refreshes its cache for DDL operations. + * create view -> refresh base table + * create child view -> refresh parent view and base table + * add/drop column on table -> refresh table + * add/drop column on view -> refresh view and base table + */ + @Test + public void testCacheUpdatedBeforeDDLOperations() throws Exception { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String url1 = QueryUtil.getConnectionUrl(props, config, "client1"); + String tableName = generateUniqueName(); + String indexName = generateUniqueName(); + String viewName = generateUniqueName(); + String childViewName = generateUniqueName(); + int numTableRPCs = 0, numViewRPCs = 0; + ConnectionQueryServices spyCqs1 = Mockito.spy(driver.getConnectionQueryServices(url1, props)); + try (Connection conn1 = spyCqs1.connect(url1, props)) { + // create table + createTable(conn1, tableName); + + // create index, getTable RPCs for base table + createIndex(conn1, tableName, indexName, "v2"); + // getting current time, + // create index(compile+execute), + // alter index state after building(compile+execute) + numTableRPCs += 5; + assertNumGetTableRPC(spyCqs1, tableName, numTableRPCs); + + + // create a view, getTable RPC for base table + createView(conn1, tableName, viewName); + numTableRPCs++; + assertNumGetTableRPC(spyCqs1, tableName, numTableRPCs); + + // create a child view, getTable RPC for parent view and base table + createView(conn1, viewName, childViewName); + numTableRPCs++; + numViewRPCs++; + assertNumGetTableRPC(spyCqs1, tableName, numTableRPCs); + assertNumGetTableRPC(spyCqs1, viewName, numViewRPCs); + + + // add and drop column, 2 getTable RPCs for base table + alterTableAddColumn(conn1, tableName, "newcol1"); + numTableRPCs++; + alterTableDropColumn(conn1, tableName, "newcol1"); + numTableRPCs++; + assertNumGetTableRPC(spyCqs1, tableName, numTableRPCs); + + // add and drop column, 2 getTable RPCs for view + alterViewAddColumn(conn1, viewName, "newcol2"); + numViewRPCs++; + numTableRPCs++; + alterViewDropColumn(conn1, viewName, "newcol2"); + numViewRPCs++; + numTableRPCs++; + assertNumGetTableRPC(spyCqs1, viewName, numViewRPCs); + assertNumGetTableRPC(spyCqs1, tableName, numTableRPCs); + } + } + + + //Helper methods + public static void assertNumGetTableRPC(ConnectionQueryServices spyCqs, String tableName, int numExpectedRPCs) throws SQLException { + Mockito.verify(spyCqs, Mockito.times(numExpectedRPCs)).getTable(eq(null), + any(byte[].class), eq(PVarchar.INSTANCE.toBytes(tableName)), + anyLong(), anyLong()); + } + public static void assertPlan(PhoenixResultSet rs, String schemaName, String tableName) { + PTable table = rs.getContext().getCurrentTable().getTable(); + assertTrue(table.getSchemaName().getString().equals(schemaName) && + table.getTableName().getString().equals(tableName)); + } + + private long getLastDDLTimestamp(String tableName) throws SQLException { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + // Need to use different connection than what is used for creating table or indexes. + String url = QueryUtil.getConnectionUrl(props, config, "client1"); + try (Connection conn = DriverManager.getConnection(url)) { + PTable table = PhoenixRuntime.getTableNoCache(conn, tableName); + return table.getLastDDLTimestamp(); + } + } + + private void createTable(Connection conn, String tableName) throws SQLException { + conn.createStatement().execute("CREATE TABLE " + tableName + + "(k INTEGER NOT NULL PRIMARY KEY, v1 INTEGER, v2 INTEGER)"); + } + + private void createView(Connection conn, String parentName, String viewName) throws SQLException { + conn.createStatement().execute("CREATE VIEW " + viewName + " AS SELECT * FROM " + parentName); + } + + private void createViewWhereClause(Connection conn, String parentName, String viewName, String whereClause) throws SQLException { + conn.createStatement().execute("CREATE VIEW " + viewName + + " AS SELECT * FROM "+ parentName + whereClause); + } + + private void createIndex(Connection conn, String tableName, String indexName, String col) throws SQLException { + conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + tableName + "(" + col + ")"); + } + + private void upsert(Connection conn, String tableName, boolean doCommit) throws SQLException { + conn.createStatement().execute("UPSERT INTO " + tableName + + " (k, v1, v2) VALUES ("+ RANDOM.nextInt() +", " + RANDOM.nextInt() + ", " + RANDOM.nextInt() +")"); + if (doCommit) { + conn.commit(); + } + } + + private void query(Connection conn, String tableName) throws SQLException { + ResultSet rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + tableName); + rs.next(); + } + + private void queryWithIndex(Connection conn, String tableName) throws SQLException { + ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM " + tableName + " WHERE v1=1"); + rs.next(); + } + + private void alterTableAddColumn(Connection conn, String tableName, String columnName) throws SQLException { + conn.createStatement().execute("ALTER TABLE " + tableName + " ADD IF NOT EXISTS " + + columnName + " INTEGER"); + } + + private void alterTableDropColumn(Connection conn, String tableName, String columnName) throws SQLException { + conn.createStatement().execute("ALTER TABLE " + tableName + " DROP COLUMN " + columnName); + } + + private void alterViewAddColumn(Connection conn, String viewName, String columnName) throws SQLException { + conn.createStatement().execute("ALTER VIEW " + viewName + " ADD IF NOT EXISTS " + + columnName + " INTEGER"); + } + + private void alterViewDropColumn(Connection conn, String viewName, String columnName) throws SQLException { + conn.createStatement().execute("ALTER VIEW " + viewName + " DROP COLUMN " + columnName); + } + + private void alterIndexChangeState(Connection conn, String tableName, String indexName, String state) throws SQLException, InterruptedException { + conn.createStatement().execute("ALTER INDEX " + indexName + " ON " + tableName + state); + } + + private void dropIndex(Connection conn, String tableName, String indexName) throws SQLException { + conn.createStatement().execute("DROP INDEX " + indexName + " ON " + tableName); + } + + private void dropView(Connection conn, String viewName, boolean cascade) throws SQLException { + String sql = "DROP VIEW " + viewName; + if (cascade) { + sql += " CASCADE"; + } + conn.createStatement().execute(sql); + } + + private void multiTableUpsert(Connection conn, String tableName1, String tableName2) throws SQLException { + conn.createStatement().execute("UPSERT INTO " + tableName1 + + " (k, v1, v2) VALUES ("+ RANDOM.nextInt() +", " + RANDOM.nextInt() + ", " + RANDOM.nextInt() +")"); + conn.createStatement().execute("UPSERT INTO " + tableName1 + + " (k, v1, v2) VALUES ("+ RANDOM.nextInt() +", " + RANDOM.nextInt() + ", " + RANDOM.nextInt() +")"); + conn.createStatement().execute("UPSERT INTO " + tableName2 + + " (k, v1, v2) VALUES ("+ RANDOM.nextInt() +", " + RANDOM.nextInt() + ", " + RANDOM.nextInt() +")"); + conn.createStatement().execute("UPSERT INTO " + tableName1 + + " (k, v1, v2) VALUES ("+ RANDOM.nextInt() +", " + RANDOM.nextInt() + ", " + RANDOM.nextInt() +")"); + conn.createStatement().execute("UPSERT INTO " + tableName2 + + " (k, v1, v2) VALUES ("+ RANDOM.nextInt() +", " + RANDOM.nextInt() + ", " + RANDOM.nextInt() +")"); + conn.commit(); + } +} 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 12106b27536..f02f358cd4d 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 @@ -21,6 +21,7 @@ import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; @@ -1108,7 +1109,8 @@ public void testDroppingIndexedColDropsViewIndex() throws Exception { String viewIndex2 = generateUniqueName(); String fullNameViewIndex1 = SchemaUtil.getTableName(viewSchemaName, viewIndex1); String fullNameViewIndex2 = SchemaUtil.getTableName(viewSchemaName, viewIndex2); - + List fullViewIndexNames = Arrays.asList(fullNameViewIndex1, fullNameViewIndex2); + conn.setAutoCommit(false); viewConn.setAutoCommit(false); String ddlFormat = @@ -1149,10 +1151,8 @@ public void testDroppingIndexedColDropsViewIndex() throws Exception { byte[] viewIndexPhysicalTable = viewIndex.getPhysicalName().getBytes(); assertNotNull("Can't find view index", viewIndex); assertEquals("Unexpected number of indexes ", 2, view.getIndexes().size()); - assertEquals("Unexpected index ", fullNameViewIndex1 , view.getIndexes().get(0).getName() - .getString()); - assertEquals("Unexpected index ", fullNameViewIndex2 , view.getIndexes().get(1).getName() - .getString()); + assertTrue("Expected index not found ", fullViewIndexNames.contains(view.getIndexes().get(0).getName().getString())); + assertTrue("Expected index not found ", fullViewIndexNames.contains(view.getIndexes().get(1).getName().getString())); assertEquals("Unexpected salt buckets", view.getBucketNum(), view.getIndexes().get(0).getBucketNum()); assertEquals("Unexpected salt buckets", view.getBucketNum(), @@ -1181,14 +1181,15 @@ public void testDroppingIndexedColDropsViewIndex() throws Exception { } pconn = viewConn.unwrap(PhoenixConnection.class); - view = pconn.getTable(new PTableKey(tenantId, viewOfTable )); + view = pconn.getTableNoCache(viewOfTable); + assertEquals("Unexpected number of indexes ", 1, view.getIndexes().size()); + assertEquals("Unexpected index ", fullNameViewIndex2 , view.getIndexes().get(0).getName().getString()); + assertNotEquals("Dropped index should not be in view metadata ", fullNameViewIndex1 , view.getIndexes().get(0).getName().getString()); try { - viewIndex = pconn.getTable(new PTableKey(tenantId, fullNameViewIndex1 )); + viewIndex = pconn.getTableNoCache(fullNameViewIndex1); fail("View index should have been dropped"); } catch (TableNotFoundException e) { } - assertEquals("Unexpected number of indexes ", 1, view.getIndexes().size()); - assertEquals("Unexpected index ", fullNameViewIndex2 , view.getIndexes().get(0).getName().getString()); // verify that the physical index view table is *not* dropped conn.unwrap(PhoenixConnection.class).getQueryServices().getTableDescriptor(viewIndexPhysicalTable); 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 3d25c1a9321..dba8c650451 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 @@ -305,7 +305,7 @@ public void testValidateAttributes() throws Exception { "create table IF NOT EXISTS " + tableName + " ( id char(1) NOT NULL," + " col1 integer NOT NULL," + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1))" - + " APPEND_ONLY_SCHEMA = true"); + + " APPEND_ONLY_SCHEMA = true, UPDATE_CACHE_FREQUENCY=always"); fail("UPDATE_CACHE_FREQUENCY attribute must not be set to ALWAYS if APPEND_ONLY_SCHEMA is true"); } catch (SQLException e) { assertEquals(SQLExceptionCode.UPDATE_CACHE_FREQUENCY_INVALID.getErrorCode(), diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BackwardCompatibilityIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BackwardCompatibilityIT.java index 3864292cbde..6348497119a 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BackwardCompatibilityIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BackwardCompatibilityIT.java @@ -136,6 +136,7 @@ public synchronized void cleanUpAfterTest() throws Exception { DriverManager.deregisterDriver(PhoenixDriver.INSTANCE); } finally { hbaseTestUtil.shutdownMiniCluster(); + ServerMetadataCacheTestImpl.resetCache(); } System.setProperty("java.io.tmpdir", tmpDir); assertFalse("refCount leaked", refCountLeaked); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionIT.java index 243b3c730a1..0eba0b8f517 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionIT.java @@ -23,6 +23,7 @@ import static org.junit.Assume.assumeTrue; import java.sql.Connection; +import java.sql.Driver; import java.sql.DriverManager; import java.sql.ResultSet; import java.sql.SQLException; @@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.util.VersionInfo; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixDriver; +import org.apache.phoenix.jdbc.PhoenixTestDriver; import org.apache.phoenix.mapreduce.util.ConnectionUtil; import org.apache.phoenix.query.ConfigurationFactory; import org.apache.phoenix.util.InstanceResolver; @@ -58,6 +60,7 @@ public static synchronized void setUp() throws Exception { conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/hbase-test"); hbaseTestUtil.startMiniCluster(); Class.forName(PhoenixDriver.class.getName()); + DriverManager.registerDriver(new PhoenixTestDriver()); InstanceResolver.clearSingletons(); // Make sure the ConnectionInfo doesn't try to pull a default Configuration InstanceResolver.getSingleton(ConfigurationFactory.class, new ConfigurationFactory() { diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionUtilIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionUtilIT.java index 1937d052d6f..d8454a8ff53 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionUtilIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConnectionUtilIT.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals; import java.sql.Connection; +import java.sql.DriverManager; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; @@ -30,6 +31,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HConstants; import org.apache.phoenix.jdbc.PhoenixDriver; +import org.apache.phoenix.jdbc.PhoenixTestDriver; import org.apache.phoenix.mapreduce.util.ConnectionUtil; import org.junit.BeforeClass; import org.junit.Test; @@ -49,6 +51,7 @@ public static synchronized void setUp() throws Exception { conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, "/hbase-test"); hbaseTestUtil.startMiniCluster(); Class.forName(PhoenixDriver.class.getName()); + DriverManager.registerDriver(new PhoenixTestDriver()); } @Test diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java index d7473e5baf4..ceaa1402bbc 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java @@ -906,7 +906,8 @@ public void testCreateTableNoUpdateCacheFreq() throws Exception { Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES); String createTableString = "CREATE TABLE " + tableName + " (k VARCHAR PRIMARY KEY, " + "v1 VARCHAR, v2 VARCHAR)"; - verifyUCFValueInSysCat(tableName, createTableString, props, 0L); + verifyUCFValueInSysCat(tableName, createTableString, props, + QueryServicesOptions.DEFAULT_UPDATE_CACHE_FREQUENCY); } @Test diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/FailingPhoenixRegionServerEndpoint.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/FailingPhoenixRegionServerEndpoint.java new file mode 100644 index 00000000000..5f33610c294 --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/FailingPhoenixRegionServerEndpoint.java @@ -0,0 +1,91 @@ +/* + * 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.end2end; + +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import org.apache.phoenix.coprocessor.PhoenixRegionServerEndpoint; +import org.apache.phoenix.coprocessor.generated.RegionServerEndpointProtos; +import org.apache.phoenix.protobuf.ProtobufUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +import static org.apache.phoenix.query.QueryServices.PHOENIX_METADATA_CACHE_INVALIDATION_TIMEOUT_MS; +import static org.apache.phoenix.query.QueryServices.PHOENIX_METADATA_CACHE_INVALIDATION_TIMEOUT_MS_DEFAULT; + +public class FailingPhoenixRegionServerEndpoint extends PhoenixRegionServerEndpoint { + private static final Logger LOGGER = LoggerFactory.getLogger(FailingPhoenixRegionServerEndpoint.class); + + private boolean throwException; + private boolean shouldSleep; + private boolean failFirstAndThenSucceed; + private int attempt = 0; + + @Override + public void invalidateServerMetadataCache(RpcController controller, + RegionServerEndpointProtos.InvalidateServerMetadataCacheRequest request, + RpcCallback done) { + long metadataCacheInvalidationTimeoutMs = conf.getLong( + PHOENIX_METADATA_CACHE_INVALIDATION_TIMEOUT_MS, + PHOENIX_METADATA_CACHE_INVALIDATION_TIMEOUT_MS_DEFAULT); + + if (throwException == true) { + IOException ioe = new IOException("On purpose"); + ProtobufUtil.setControllerException(controller, ioe); + return; + } else if (shouldSleep) { + try { + // Sleeping for 2 seconds more than metadataCacheInvalidationTimeoutMs. + Thread.sleep(metadataCacheInvalidationTimeoutMs + 2000); + } catch (InterruptedException e) { + LOGGER.warn("Exception while sleeping in FailingPhoenixRegionServerEndpoint", e); + } + } else if (failFirstAndThenSucceed) { + if (attempt == 0) { + IOException ioe = new IOException("On purpose"); + ProtobufUtil.setControllerException(controller, ioe); + attempt++; + } + } + else { + LOGGER.info("Invalidating server metadata cache"); + } + } + + public void throwException() { + reset(); + this.throwException = true; + } + + public void sleep() { + reset(); + this.shouldSleep = true; + } + + public void failFirstAndThenSucceed() { + reset(); + failFirstAndThenSucceed = true; + } + private void reset() { + this.shouldSleep = false; + this.throwException = false; + this.failFirstAndThenSucceed = false; + } +} diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/InvalidateMetadataCacheIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InvalidateMetadataCacheIT.java new file mode 100644 index 00000000000..b8d1d55732a --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/InvalidateMetadataCacheIT.java @@ -0,0 +1,166 @@ +/* + * 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.end2end; + +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.phoenix.query.BaseTest; +import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; +import org.apache.phoenix.util.PropertiesUtil; +import org.apache.phoenix.util.ReadOnlyProps; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.util.Map; +import java.util.Properties; + +import static org.apache.hadoop.hbase.coprocessor.CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY; +import static org.apache.phoenix.query.QueryServices.PHOENIX_METADATA_CACHE_INVALIDATION_TIMEOUT_MS; +import static org.apache.phoenix.query.QueryServices.PHOENIX_METADATA_INVALIDATE_CACHE_ENABLED; +import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES; +import static org.junit.Assert.fail; + +@Category({NeedsOwnMiniClusterTest.class }) +public class InvalidateMetadataCacheIT extends BaseTest { + private static final Logger LOGGER = LoggerFactory.getLogger(InvalidateMetadataCacheIT.class); + + @BeforeClass + public static synchronized void doSetup() throws Exception { + NUM_SLAVES_BASE = 2; + Map props = Maps.newHashMapWithExpectedSize(1); + // to fail fast in case of exception. + props.put("hbase.client.retries.number", String.valueOf(0)); + props.put(PHOENIX_METADATA_INVALIDATE_CACHE_ENABLED, "true"); + props.put(REGIONSERVER_COPROCESSOR_CONF_KEY, + FailingPhoenixRegionServerEndpoint.class.getName()); + // Setting phoenix metadata cache invalidation timeout to a small number to fail fast. + props.put(PHOENIX_METADATA_CACHE_INVALIDATION_TIMEOUT_MS, String.valueOf(2000)); + setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator())); + } + + /** + * Add FailingPhoenixRegionServerEndpoint as regionserver co-processor. + * Make one of the regionserver sleep in invalidateServerMetadataCache method. This will trigger + * TimeoutException in MetadataEndpointImpl#invalidateServerMetadataCacheWithRetries method. + * Make sure that ALTER TABLE ADD COLUMN statement fails. + */ + @Test + public void testAddColumnWithTimeout() { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String dataTableFullName = generateUniqueName(); + String ddl = getCreateTableStmt(dataTableFullName); + HRegionServer regionServerZero = utility.getMiniHBaseCluster().getRegionServer(0); + FailingPhoenixRegionServerEndpoint coprocForRS0 = + getFailingPhoenixRegionServerEndpoint(regionServerZero); + coprocForRS0.sleep(); + try(Connection conn = DriverManager.getConnection(getUrl(), props)) { + conn.createStatement().execute(ddl); + conn.createStatement().execute("ALTER TABLE " + dataTableFullName + + " ADD CF.col2 integer"); + fail("Shouldn't reach here"); + } catch (Exception e) { + LOGGER.error("Exception while adding column", e); + // This is expected + } + } + + /** + * Add FailingPhoenixRegionServerEndpoint as regionserver co-processor. + * Make one of the regionserver throw Exception in invalidateServerMetadataCache method. + * Make sure that ALTER TABLE ADD COLUMN statement fails. + */ + @Test + public void testAddColumnWithOneRSFailing() { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String dataTableFullName = generateUniqueName(); + String ddl = getCreateTableStmt(dataTableFullName); + HRegionServer regionServerZero = utility.getMiniHBaseCluster().getRegionServer(0); + FailingPhoenixRegionServerEndpoint coprocForRS0 = + getFailingPhoenixRegionServerEndpoint(regionServerZero); + coprocForRS0.throwException(); + try(Connection conn = DriverManager.getConnection(getUrl(), props)) { + conn.createStatement().execute(ddl); + conn.createStatement().execute("ALTER TABLE " + dataTableFullName + + " ADD CF.col2 integer"); + fail("Shouldn't reach here"); + } catch (Exception e) { + LOGGER.error("Exception while adding column", e); + // This is expected + } + } + + /** + * Add FailingPhoenixRegionServerEndpoint as regionserver co-processor. + * Make one of the regionserver throw Exception in the first attempt and succeed on retry. + * Make sure that ALTER TABLE ADD COLUMN statement succeeds on retry. + */ + @Test + public void testAddColumnWithOneRSSucceedingOnRetry() { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String dataTableFullName = generateUniqueName(); + String ddl = getCreateTableStmt(dataTableFullName); + HRegionServer regionServerZero = utility.getMiniHBaseCluster().getRegionServer(0); + FailingPhoenixRegionServerEndpoint coprocForRS0 = + getFailingPhoenixRegionServerEndpoint(regionServerZero); + coprocForRS0.failFirstAndThenSucceed(); + try(Connection conn = DriverManager.getConnection(getUrl(), props)) { + conn.createStatement().execute(ddl); + conn.createStatement().execute("ALTER TABLE " + dataTableFullName + + " ADD CF.col2 integer"); + } catch (Throwable e) { + fail("Shouldn't reach here"); + } + } + + /** + * Add FailingPhoenixRegionServerEndpoint as regionserver co-processor. + * Do not throw Exception or sleep in invalidateServerMetadataCache method for any regionservers + * Make sure that ALTER TABLE ADD COLUMN statement succeeds. + */ + @Test + public void testAddColumnWithBothRSPassing() { + Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); + String dataTableFullName = generateUniqueName(); + String ddl = getCreateTableStmt(dataTableFullName); + try(Connection conn = DriverManager.getConnection(getUrl(), props)) { + conn.createStatement().execute(ddl); + conn.createStatement().execute("ALTER TABLE " + dataTableFullName + + " ADD CF.col2 integer"); + } catch (Throwable t) { + fail("Shouldn't reach here"); + } + } + + private String getCreateTableStmt(String tableName) { + return "CREATE TABLE " + tableName + + " (a_string varchar not null, col1 integer" + + " CONSTRAINT pk PRIMARY KEY (a_string)) "; + } + + private FailingPhoenixRegionServerEndpoint getFailingPhoenixRegionServerEndpoint( + HRegionServer regionServer) { + FailingPhoenixRegionServerEndpoint coproc = regionServer + .getRegionServerCoprocessorHost() + .findCoprocessor(FailingPhoenixRegionServerEndpoint.class); + return coproc; + } +} \ No newline at end of file diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameBaseIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameBaseIT.java index 7039a85873f..bae0589b681 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameBaseIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameBaseIT.java @@ -31,6 +31,7 @@ import org.apache.phoenix.schema.PTable; import org.apache.phoenix.schema.types.PInteger; import org.apache.phoenix.util.ByteUtil; +import org.apache.phoenix.util.EnvironmentEdgeManager; import org.apache.phoenix.util.PropertiesUtil; import org.apache.phoenix.util.QueryUtil; import org.apache.phoenix.util.ReadOnlyProps; @@ -530,13 +531,16 @@ protected void validateIndex(Connection connection, String tableName, boolean is } public static void renameAndDropPhysicalTable(Connection conn, String tenantId, String schema, String tableName, String physicalName, boolean isNamespaceEnabled) throws Exception { + // if client is validating last_ddl_timestamp, this change in physical table name should be visible to the client + // UPDATE LAST_DDL_TIMESTAMP of the table and clear the server metadata cache on region servers + long lastDDLTimestamp = EnvironmentEdgeManager.currentTimeMillis(); String changeName = String.format( - "UPSERT INTO SYSTEM.CATALOG (TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, PHYSICAL_TABLE_NAME) VALUES (%s, %s, '%s', NULL, NULL, '%s')", - tenantId, schema==null ? null : ("'" + schema + "'"), tableName, physicalName); + "UPSERT INTO SYSTEM.CATALOG (TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, PHYSICAL_TABLE_NAME, LAST_DDL_TIMESTAMP) VALUES (%s, %s, '%s', NULL, NULL, '%s', %d)", + tenantId, schema==null ? null : ("'" + schema + "'"), tableName, physicalName, lastDDLTimestamp); conn.createStatement().execute(changeName); conn.commit(); - + ServerMetadataCacheTestImpl.resetCache(); String fullTableName = SchemaUtil.getTableName(schema, tableName); if (isNamespaceEnabled && !(Strings.isNullOrEmpty(schema) || NULL_STRING.equals(schema))) { fullTableName = schema + NAMESPACE_SEPARATOR + tableName; diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameExtendedIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameExtendedIT.java index 496d2bd2620..b99f66d1ca0 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameExtendedIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/LogicalTableNameExtendedIT.java @@ -209,21 +209,26 @@ public void testHint() throws Exception { createIndexOnTable(conn, tableName, indexName2); populateTable(conn, tableName, 1, 2); - // Test hint String tableSelect = "SELECT V1,V2,V3 FROM " + tableName; ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + tableSelect); - assertEquals(true, QueryUtil.getExplainPlan(rs).contains(indexName)); + String plan = QueryUtil.getExplainPlan(rs); + // plan should use one of the indexes + assertEquals(true, plan.contains(indexName) || plan.contains(indexName2)); + // Test hint for the other index + String hintedIndex = QueryUtil.getExplainPlan(rs).contains(indexName) + ? indexName2 + : indexName; try (Admin admin = conn.unwrap(PhoenixConnection.class).getQueryServices() .getAdmin()) { - String snapshotName = new StringBuilder(indexName2).append("-Snapshot").toString(); - admin.snapshot(snapshotName, TableName.valueOf(indexName2)); - String newName = "NEW_" + indexName2; + String snapshotName = new StringBuilder(hintedIndex).append("-Snapshot").toString(); + admin.snapshot(snapshotName, TableName.valueOf(hintedIndex)); + String newName = "NEW_" + hintedIndex; admin.cloneSnapshot(snapshotName, TableName.valueOf(newName)); - renameAndDropPhysicalTable(conn, "NULL", null, indexName2, newName, true); + renameAndDropPhysicalTable(conn, "NULL", null, hintedIndex, newName, true); } - String indexSelect = "SELECT /*+ INDEX(" + tableName + " " + indexName2 + ")*/ V1,V2,V3 FROM " + tableName; + String indexSelect = "SELECT /*+ INDEX(" + tableName + " " + hintedIndex + ")*/ V1,V2,V3 FROM " + tableName; rs = conn.createStatement().executeQuery("EXPLAIN " + indexSelect); - assertEquals(true, QueryUtil.getExplainPlan(rs).contains(indexName2)); + assertEquals(true, QueryUtil.getExplainPlan(rs).contains(hintedIndex)); rs = conn.createStatement().executeQuery(indexSelect); assertEquals(true, rs.next()); } 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 b75d0a3258c..918f7a19280 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 @@ -16,6 +16,7 @@ */ package org.apache.phoenix.end2end; +import static org.apache.hadoop.hbase.coprocessor.CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -41,10 +42,12 @@ import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.phoenix.coprocessor.PhoenixRegionServerEndpoint; import org.apache.phoenix.coprocessorclient.MetaDataProtocol; import org.apache.phoenix.exception.UpgradeInProgressException; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; +import org.apache.phoenix.jdbc.PhoenixTestDriver; import org.apache.phoenix.query.BaseTest; import org.apache.phoenix.query.ConnectionQueryServices; import org.apache.phoenix.query.ConnectionQueryServicesImpl; @@ -54,6 +57,7 @@ import org.apache.phoenix.util.ReadOnlyProps; import org.apache.phoenix.util.SchemaUtil; import org.junit.After; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -85,9 +89,15 @@ public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest { final UserGroupInformation user4 = UserGroupInformation.createUserForTesting("user4", new String[0]); + @BeforeClass + public static synchronized void registerTestDriver() throws SQLException { + DriverManager.registerDriver(new PhoenixTestDriver()); + } + public final void doSetup(boolean systemMappingEnabled) throws Exception { testUtil = new HBaseTestingUtility(); Configuration conf = testUtil.getConfiguration(); + conf.set(REGIONSERVER_COPROCESSOR_CONF_KEY, PhoenixRegionServerEndpointTestImpl.class.getName()); enableNamespacesOnServer(conf, systemMappingEnabled); configureRandomHMasterPort(conf); testUtil.startMiniCluster(1); @@ -98,6 +108,7 @@ public void tearDownMiniCluster() { try { if (testUtil != null) { boolean refCountLeaked = isAnyStoreRefCountLeaked(); + ServerMetadataCacheTestImpl.resetCache(); testUtil.shutdownMiniCluster(); testUtil = null; assertFalse("refCount leaked", refCountLeaked); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PartialResultServerConfigurationIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PartialResultServerConfigurationIT.java index d529f6ebfdc..2cc4629cd7e 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PartialResultServerConfigurationIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PartialResultServerConfigurationIT.java @@ -25,6 +25,7 @@ import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixDriver; +import org.apache.phoenix.jdbc.PhoenixTestDriver; import org.apache.phoenix.query.ConnectionQueryServices.Feature; import org.apache.phoenix.query.QueryServices; import org.apache.phoenix.util.PhoenixRuntime; @@ -77,6 +78,7 @@ public static synchronized void setUp() throws Exception { url = PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum; DriverManager.registerDriver(PhoenixDriver.INSTANCE); + DriverManager.registerDriver(new PhoenixTestDriver()); } @AfterClass @@ -84,6 +86,7 @@ public static synchronized void tearDownAfterClass() throws Exception { try { DriverManager.deregisterDriver(PhoenixDriver.INSTANCE); } finally { + ServerMetadataCacheTestImpl.resetCache(); hbaseTestUtil.shutdownMiniCluster(); } } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRegionServerEndpointIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRegionServerEndpointIT.java new file mode 100644 index 00000000000..89969392901 --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRegionServerEndpointIT.java @@ -0,0 +1,193 @@ +/* + * 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.end2end; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.ipc.ServerRpcController; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.util.ByteStringer; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.phoenix.coprocessor.PhoenixRegionServerEndpoint; +import org.apache.phoenix.coprocessor.generated.RegionServerEndpointProtos; +import org.apache.phoenix.exception.StaleMetadataCacheException; +import org.apache.phoenix.query.BaseTest; +import org.apache.phoenix.schema.PTable; +import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; +import org.apache.phoenix.util.ClientUtil; +import org.apache.phoenix.util.PhoenixRuntime; +import org.apache.phoenix.util.PropertiesUtil; +import org.apache.phoenix.util.ReadOnlyProps; +import org.apache.phoenix.util.ServerUtil; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.nio.charset.StandardCharsets; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.util.Map; +import java.util.Properties; + +import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +@Category({NeedsOwnMiniClusterTest.class }) +public class PhoenixRegionServerEndpointIT extends BaseTest { + @BeforeClass + public static synchronized void doSetup() throws Exception { + Map props = Maps.newHashMapWithExpectedSize(1); + setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator())); + } + + // Tests that PhoenixRegionServerEndpoint validates the last ddl timestamp for base table. + @Test + public void testValidateLastDDLTimestampNoException() throws SQLException { + HRegionServer regionServer = utility.getMiniHBaseCluster().getRegionServer(0); + PhoenixRegionServerEndpoint coprocessor = getPhoenixRegionServerEndpoint(regionServer); + assertNotNull(coprocessor); + ServerRpcController controller = new ServerRpcController(); + String tableNameStr = generateUniqueName(); + Properties props = new Properties(); + try (Connection conn = DriverManager.getConnection(getUrl(), props)) { + conn.setAutoCommit(false); + String ddl = getCreateTableStmt(tableNameStr); + // Create a test table. + conn.createStatement().execute(ddl); + conn.commit(); + + PTable pTable = PhoenixRuntime.getTable(conn, tableNameStr); + long lastDDLTimestamp = pTable.getLastDDLTimestamp(); + RegionServerEndpointProtos.ValidateLastDDLTimestampRequest request = getRequest( + HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, + tableNameStr.getBytes(StandardCharsets.UTF_8), lastDDLTimestamp); + // Call coprocessor#validateLastDDLTimestamp to validate + // client provided last ddl timestamp + coprocessor.validateLastDDLTimestamp(controller, request, null); + assertFalse(controller.failed()); + } + } + + // Tests that PhoenixRegionServerEndpoint throws StaleMetadataCacheException if client + // provided last ddl timestamp is less than server maintained last ddl timestamp. + @Test + public void testValidateLastDDLTimestampWithException() throws SQLException { + HRegionServer regionServer = utility.getMiniHBaseCluster().getRegionServer(0); + PhoenixRegionServerEndpoint coprocessor = getPhoenixRegionServerEndpoint(regionServer); + assertNotNull(coprocessor); + ServerRpcController controller = new ServerRpcController(); + String tableNameStr = generateUniqueName(); + Properties props = new Properties(); + try (Connection conn = DriverManager.getConnection(getUrl(), props)) { + conn.setAutoCommit(false); + String ddl = getCreateTableStmt(tableNameStr); + // Create a test table. + conn.createStatement().execute(ddl); + conn.commit(); + + PTable pTable = PhoenixRuntime.getTable(conn, tableNameStr); + long lastDDLTimestamp = pTable.getLastDDLTimestamp(); + RegionServerEndpointProtos.ValidateLastDDLTimestampRequest request = getRequest( + HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, + tableNameStr.getBytes(StandardCharsets.UTF_8), lastDDLTimestamp - 1); + // Call coprocessor#validateLastDDLTimestamp to validate client provided + // last ddl timestamp and make sure it throws an StaleMetadataCacheException + coprocessor.validateLastDDLTimestamp(controller, request, null); + assertTrue(controller.failed()); + Exception exception = controller.getFailedOn(); + Exception parsedException = ClientUtil.parseRemoteException(exception); + assertTrue(parsedException instanceof StaleMetadataCacheException); + } + } + + // Tests that PhoenixRegionServerEndpoint validates the last ddl timestamp for tenant owned + // views + @Test + public void testValidateLastDDLTimestampWithTenantID() throws SQLException { + HRegionServer regionServer = utility.getMiniHBaseCluster().getRegionServer(0); + PhoenixRegionServerEndpoint coprocessor = getPhoenixRegionServerEndpoint(regionServer); + assertNotNull(coprocessor); + ServerRpcController controller = new ServerRpcController(); + String tableNameStr = generateUniqueName(); + Properties props = new Properties(); + try (Connection conn = DriverManager.getConnection(getUrl(), props)) { + conn.setAutoCommit(false); + String ddl = getCreateTableStmt(tableNameStr); + // Create a test table. + conn.createStatement().execute(ddl); + } + String tenantId = "T_" + generateUniqueName(); + Properties tenantProps = PropertiesUtil.deepCopy(TEST_PROPERTIES); + tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId); + PTable tenantViewTable; + // Create view on table. + String whereClause = " WHERE COL1 = 1000"; + String tenantViewNameStr = generateUniqueName(); + try (Connection conn = DriverManager.getConnection(getUrl(), tenantProps)) { + conn.createStatement().execute(getCreateViewStmt(tenantViewNameStr, + tableNameStr, whereClause)); + tenantViewTable = PhoenixRuntime.getTableNoCache(conn, tenantViewNameStr); + + byte[] tenantIDBytes = Bytes.toBytes(tenantId); + byte[] tenantViewNameBytes = Bytes.toBytes(tenantViewNameStr); + long lastDDLTimestamp = tenantViewTable.getLastDDLTimestamp(); + RegionServerEndpointProtos.ValidateLastDDLTimestampRequest request = getRequest( + tenantIDBytes, HConstants.EMPTY_BYTE_ARRAY, tenantViewNameBytes, + lastDDLTimestamp); + // Call coprocessor#validateLastDDLTimestamp to validate client provided + // last ddl timestamp for tenant owned views. + coprocessor.validateLastDDLTimestamp(controller, request, null); + assertFalse(controller.failed()); + } + } + + private String getCreateTableStmt(String tableName) { + return "CREATE TABLE " + tableName + + " (a_string varchar not null, col1 integer" + + " CONSTRAINT pk PRIMARY KEY (a_string)) "; + } + + private String getCreateViewStmt(String viewName, String fullTableName, String whereClause) { + String viewStmt = "CREATE VIEW " + viewName + + " AS SELECT * FROM "+ fullTableName + whereClause; + return viewStmt; + } + + private PhoenixRegionServerEndpoint getPhoenixRegionServerEndpoint(HRegionServer regionServer) { + PhoenixRegionServerEndpoint coproc = regionServer + .getRegionServerCoprocessorHost() + .findCoprocessor(PhoenixRegionServerEndpoint.class); + return coproc; + } + + private RegionServerEndpointProtos.ValidateLastDDLTimestampRequest getRequest(byte[] tenantID, + byte[] schemaName, byte[] tableName, long lastDDLTimestamp) { + RegionServerEndpointProtos.ValidateLastDDLTimestampRequest.Builder requestBuilder + = RegionServerEndpointProtos.ValidateLastDDLTimestampRequest.newBuilder(); + RegionServerEndpointProtos.LastDDLTimestampRequest.Builder innerBuilder + = RegionServerEndpointProtos.LastDDLTimestampRequest.newBuilder(); + innerBuilder.setTenantId(ByteStringer.wrap(tenantID)); + innerBuilder.setSchemaName(ByteStringer.wrap(schemaName)); + innerBuilder.setTableName(ByteStringer.wrap(tableName)); + innerBuilder.setLastDDLTimestamp(lastDDLTimestamp); + requestBuilder.addLastDDLTimestampRequests(innerBuilder); + return requestBuilder.build(); + } +} diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRegionServerEndpointTestImpl.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRegionServerEndpointTestImpl.java new file mode 100644 index 00000000000..c7b6a9414ab --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRegionServerEndpointTestImpl.java @@ -0,0 +1,45 @@ +/* + * 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.end2end; + +import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.coprocessor.RegionServerCoprocessorEnvironment; +import org.apache.phoenix.cache.ServerMetadataCache; +import org.apache.phoenix.coprocessor.PhoenixRegionServerEndpoint; + +import java.io.IOException; + +/** + * PhoenixRegionServerEndpoint for integration tests. + * Uses {@link ServerMetadataCacheTestImpl} to support keeping multiple cache instances. + */ +public class PhoenixRegionServerEndpointTestImpl extends PhoenixRegionServerEndpoint { + protected ServerName serverName; + + @Override + public void start(CoprocessorEnvironment env) throws IOException { + super.start(env); + this.serverName = ((RegionServerCoprocessorEnvironment)env).getServerName(); + } + + @Override + public ServerMetadataCache getServerMetadataCache() { + return ServerMetadataCacheTestImpl.getInstance(conf, serverName); + } +} diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ServerMetadataCacheTestImpl.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ServerMetadataCacheTestImpl.java new file mode 100644 index 00000000000..c919f76db60 --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ServerMetadataCacheTestImpl.java @@ -0,0 +1,89 @@ +/* + * 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.end2end; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.ServerName; +import org.apache.phoenix.cache.ServerMetadataCacheImpl; +import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr; +import org.apache.phoenix.util.QueryUtil; +import org.apache.phoenix.util.SchemaUtil; + +import java.sql.Connection; +import java.sql.SQLException; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +/** + * Implementation of {@link ServerMetadataCache} for Integration Tests. + * Supports keeping more than one instanceof the cache keyed on the regionserver ServerName. + * + * PhoenixRegionServerEndpoint is a region server coproc. There is a 1-1 correspondence between + * PhoenixRegionServerEndpoint and ServerMetadataCache. In ITs we can have multiple regionservers + * per cluster so we need multiple instances of ServerMetadataCache in the same jvm. Tests using + * HighAvailabilityTestingUtility create 2 clusters so we need to have one instance of + * ServerMetadataCache for each regionserver in each cluster. + */ +public class ServerMetadataCacheTestImpl extends ServerMetadataCacheImpl { + private static volatile Map INSTANCES = new HashMap<>(); + private Connection connectionForTesting; + + ServerMetadataCacheTestImpl(Configuration conf) { + super(conf); + } + + public static ServerMetadataCacheTestImpl getInstance(Configuration conf, ServerName serverName) { + ServerMetadataCacheTestImpl result = INSTANCES.get(serverName); + if (result == null) { + synchronized (ServerMetadataCacheTestImpl.class) { + result = INSTANCES.get(serverName); + if (result == null) { + result = new ServerMetadataCacheTestImpl(conf); + INSTANCES.put(serverName, result); + } + } + } + return result; + } + + public static void setInstance(ServerName serverName, ServerMetadataCacheTestImpl cache) { + INSTANCES.put(serverName, cache); + } + + public Long getLastDDLTimestampForTableFromCacheOnly(byte[] tenantID, byte[] schemaName, + byte[] tableName) { + byte[] tableKey = SchemaUtil.getTableKey(tenantID, schemaName, tableName); + ImmutableBytesPtr tableKeyPtr = new ImmutableBytesPtr(tableKey); + return lastDDLTimestampMap.getIfPresent(tableKeyPtr); + } + + public void setConnectionForTesting(Connection connection) { + this.connectionForTesting = connection; + } + + public static void resetCache() { + INSTANCES.clear(); + } + + @Override + protected Connection getConnection(Properties properties) throws SQLException { + return connectionForTesting != null ? connectionForTesting + : super.getConnection(properties); + } +} diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogIT.java index 33bb50b4ad5..852e98e67c3 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SplitSystemCatalogIT.java @@ -28,7 +28,6 @@ import org.apache.phoenix.util.ReadOnlyProps; import org.apache.phoenix.util.SchemaUtil; import org.junit.BeforeClass; -import org.junit.experimental.categories.Category; import org.apache.phoenix.thirdparty.com.google.common.collect.Lists; import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablesCreationOnConnectionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablesCreationOnConnectionIT.java index 8680325ed92..1ff648b805f 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablesCreationOnConnectionIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablesCreationOnConnectionIT.java @@ -17,6 +17,7 @@ */ package org.apache.phoenix.end2end; +import static org.apache.hadoop.hbase.coprocessor.CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_MUTEX_FAMILY_NAME_BYTES; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_MUTEX_HBASE_TABLE_NAME; @@ -51,6 +52,8 @@ import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.ipc.ServerRpcController; import org.apache.hadoop.hbase.ipc.controller.ServerToServerRpcController; +import org.apache.phoenix.cache.ServerMetadataCacheImpl; +import org.apache.phoenix.coprocessor.PhoenixRegionServerEndpoint; import org.apache.phoenix.coprocessorclient.MetaDataProtocol; import org.apache.phoenix.exception.SQLExceptionCode; import org.apache.phoenix.exception.UpgradeRequiredException; @@ -69,6 +72,7 @@ import org.apache.phoenix.util.UpgradeUtil; import org.junit.After; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -174,6 +178,10 @@ void resetCQS() { } } + @BeforeClass + public static synchronized void registerTestDriver() throws SQLException { + DriverManager.registerDriver(new PhoenixTestDriver()); + } @Before public void resetVariables() { setOldTimestampToInduceUpgrade = false; @@ -192,6 +200,7 @@ public synchronized void tearDownMiniCluster() { refCountLeaked = BaseTest.isAnyStoreRefCountLeaked(testUtil.getAdmin()); } testUtil.shutdownMiniCluster(); + ServerMetadataCacheTestImpl.resetCache(); testUtil = null; assertFalse("refCount leaked", refCountLeaked); } @@ -661,6 +670,7 @@ private void startMiniClusterWithToggleNamespaceMapping(String isNamespaceMappin conf.set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, isNamespaceMappingEnabled); // Avoid multiple clusters trying to bind to the master's info port (16010) conf.setInt(HConstants.MASTER_INFO_PORT, -1); + conf.set(REGIONSERVER_COPROCESSOR_CONF_KEY, PhoenixRegionServerEndpointTestImpl.class.getName()); testUtil.startMiniCluster(1); } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpdateCacheAcrossDifferentClientsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpdateCacheAcrossDifferentClientsIT.java index 23606d8684c..76f5a701eb0 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpdateCacheAcrossDifferentClientsIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpdateCacheAcrossDifferentClientsIT.java @@ -38,6 +38,7 @@ import java.sql.DriverManager; import java.sql.PreparedStatement; import java.sql.ResultSet; +import java.sql.SQLException; import java.util.Properties; import static org.junit.Assert.assertEquals; @@ -104,17 +105,19 @@ public void testUpdateCacheFrequencyWithAddAndDropTable() throws Exception { } catch (TableNotFoundException e) { //Expected } - rs = conn2.createStatement().executeQuery("select * from "+tableName); try { + rs = conn2.createStatement().executeQuery("select * from "+tableName); rs.next(); fail("Should throw org.apache.hadoop.hbase.TableNotFoundException since the latest metadata " + "wasn't fetched"); - } catch (PhoenixIOException ex) { - boolean foundHBaseTableNotFound = false; - for(Throwable throwable : Throwables.getCausalChain(ex)) { - if(org.apache.hadoop.hbase.TableNotFoundException.class.equals(throwable.getClass())) { - foundHBaseTableNotFound = true; - break; + } catch (SQLException ex) { + boolean foundHBaseTableNotFound = (ex instanceof TableNotFoundException); + if (!foundHBaseTableNotFound) { + for(Throwable throwable : Throwables.getCausalChain(ex)) { + if(org.apache.hadoop.hbase.TableNotFoundException.class.equals(throwable.getClass())) { + foundHBaseTableNotFound = true; + break; + } } } assertTrue("Should throw org.apache.hadoop.hbase.TableNotFoundException since the latest" + diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java index c59726adfbe..2c5478fa7f9 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java @@ -68,7 +68,6 @@ import org.apache.phoenix.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellUtil; -import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeepDeletedCells; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; @@ -731,13 +730,14 @@ public void testLastDDLTimestampBootstrap() throws Exception { } private void nullDDLTimestamps(Connection conn) throws SQLException { + //ignore system tables since that can interfere with other tests. String pkCols = TENANT_ID + ", " + TABLE_SCHEM + ", " + TABLE_NAME + ", " + COLUMN_NAME + ", " + COLUMN_FAMILY; String upsertSql = "UPSERT INTO " + SYSTEM_CATALOG_NAME + " (" + pkCols + ", " + LAST_DDL_TIMESTAMP + ")" + " " + "SELECT " + pkCols + ", NULL FROM " + SYSTEM_CATALOG_NAME + " " + - "WHERE " + TABLE_TYPE + " IS NOT NULL"; + "WHERE " + TABLE_TYPE + " " + " != '" + PTableType.SYSTEM.getSerializedValue() + "'"; conn.createStatement().execute(upsertSql); conn.commit(); } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewMetadataIT.java index cb07031ac58..c32a8e2d4fd 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewMetadataIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewMetadataIT.java @@ -86,7 +86,10 @@ import org.apache.phoenix.schema.ColumnAlreadyExistsException; import org.apache.phoenix.schema.PColumn; import org.apache.phoenix.schema.PName; +import org.apache.phoenix.schema.PNameImpl; import org.apache.phoenix.schema.PTable; +import org.apache.phoenix.schema.PTableImpl; +import org.apache.phoenix.schema.PTableKey; import org.apache.phoenix.schema.PTableType; import org.apache.phoenix.schema.TableAlreadyExistsException; import org.apache.phoenix.schema.TableNotFoundException; @@ -1375,6 +1378,120 @@ public void testCreateViewDefinesPKConstraint() throws Exception { assertPKs(rs, new String[] {"K1", "K2", "K3", "K4"}); } + @Test + public void testAncestorLastDDLMapPopulatedInViewAndIndexHierarchy() throws SQLException { + String baseTable = SchemaUtil.getTableName(SCHEMA1, generateUniqueName()); + String view1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName()); + String view2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName()); + String view3 = SchemaUtil.getTableName(SCHEMA4, generateUniqueName()); + String view4 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName()); + String index1 = generateUniqueName(); + String index2 = generateUniqueName(); + String tenant1 = TENANT1; + String tenant2 = TENANT2; + /* baseTable + / | \ \ + view1(tenant1) view3(tenant2) index1(global) view4(global) + / + view2(tenant1) + / + index2(tenant1) + */ + try (Connection conn = DriverManager.getConnection(getUrl())) { + String baseTableDDL = "CREATE TABLE " + baseTable + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR CONSTRAINT NAME_PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT = true "; + conn.createStatement().execute(baseTableDDL); + String index1DDL = "CREATE INDEX " + index1 + " ON " + baseTable + "(V1)"; + conn.createStatement().execute(index1DDL); + + + try (Connection tenant1Conn = getTenantConnection(tenant1)) { + String view1DDL = "CREATE VIEW " + view1 + " AS SELECT * FROM " + baseTable; + tenant1Conn.createStatement().execute(view1DDL); + + String view2DDL = "CREATE VIEW " + view2 + " AS SELECT * FROM " + view1; + tenant1Conn.createStatement().execute(view2DDL); + + String index2DDL = "CREATE INDEX " + index2 + " ON " + view2 + "(V1)"; + tenant1Conn.createStatement().execute(index2DDL); + } + + try (Connection tenant2Conn = getTenantConnection(tenant2)) { + String view3DDL = "CREATE VIEW " + view3 + " AS SELECT * FROM " + baseTable; + tenant2Conn.createStatement().execute(view3DDL); + } + + String view4DDL = "CREATE VIEW " + view4 + " AS SELECT * FROM " + baseTable; + conn.createStatement().execute(view4DDL); + + //validate ancestor->last_ddl_timestamps maps + PTable basePTable = PhoenixRuntime.getTable(conn, baseTable); + Long baseTableLastDDLTimestamp = basePTable.getLastDDLTimestamp(); + PTableKey baseTableKey = new PTableKey(null, baseTable); + //base table map should be empty + Map map = basePTable.getAncestorLastDDLTimestampMap(); + assertEquals(0, map.size()); + + //global view + map = PhoenixRuntime.getTable(conn, view4).getAncestorLastDDLTimestampMap(); + assertEquals(1, map.size()); + assertEquals(baseTableLastDDLTimestamp, map.get(baseTableKey)); + + //global index in cache and in parent PTable + PTable index1PTable = PhoenixRuntime.getTable(conn, SchemaUtil.getTableName(SCHEMA1, index1)); + map = index1PTable.getAncestorLastDDLTimestampMap(); + assertEquals(baseTableLastDDLTimestamp, map.get(baseTableKey)); + assertEquals(1, basePTable.getIndexes().size()); + map = basePTable.getIndexes().get(0).getAncestorLastDDLTimestampMap(); + assertEquals(baseTableLastDDLTimestamp, map.get(baseTableKey)); + + //tenant2 view + try (Connection tenant2Conn = getTenantConnection(tenant2)) { + map = PhoenixRuntime.getTable(tenant2Conn, view3).getAncestorLastDDLTimestampMap(); + assertEquals(1, map.size()); + assertEquals(baseTableLastDDLTimestamp, map.get(baseTableKey)); + } + try (Connection tenant1Conn = getTenantConnection(tenant1)) { + //tenant1 view + PTable view1PTable = PhoenixRuntime.getTable(tenant1Conn, view1); + map = view1PTable.getAncestorLastDDLTimestampMap(); + assertEquals(1, map.size()); + assertEquals(baseTableLastDDLTimestamp, map.get(baseTableKey)); + //tenant1 child view + PTableKey view1Key = new PTableKey(view1PTable.getTenantId(), view1); + map = PhoenixRuntime.getTable(tenant1Conn, view2).getAncestorLastDDLTimestampMap(); + assertEquals(2, map.size()); + assertEquals(baseTableLastDDLTimestamp, map.get(baseTableKey)); + assertEquals(view1PTable.getLastDDLTimestamp(), map.get(view1Key)); + //tenant1 child view index in cache and in child view PTable + PTable view2PTable = PhoenixRuntime.getTable(tenant1Conn, view2); + PTableKey view2Key = new PTableKey(view2PTable.getTenantId(), view2); + PTable index2PTable = PhoenixRuntime.getTable(tenant1Conn, SchemaUtil.getTableName(SCHEMA3, index2)); + map = index2PTable.getAncestorLastDDLTimestampMap(); + assertEquals(baseTableLastDDLTimestamp, map.get(baseTableKey)); + assertEquals(view2PTable.getLastDDLTimestamp(), map.get(view2Key)); + assertEquals(2, view2PTable.getIndexes().size()); + for (PTable index : view2PTable.getIndexes()) { + // inherited index + if (index.getTableName().getString().equals(index1)) { + map = index.getAncestorLastDDLTimestampMap(); + assertEquals(baseTableLastDDLTimestamp, map.get(baseTableKey)); + } else { + // view index + map = index.getAncestorLastDDLTimestampMap(); + assertEquals(baseTableLastDDLTimestamp, map.get(baseTableKey)); + assertEquals(view2PTable.getLastDDLTimestamp(), map.get(view2Key)); + } + } + } + } + } + + private Connection getTenantConnection(String tenantId) throws SQLException { + Properties tenantProps = new Properties(); + tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId); + return DriverManager.getConnection(getUrl(), tenantProps); + } + @Test public void testAlterViewAndViewIndexMaxLookbackAgeFails() throws Exception { String schemaName = generateUniqueName(); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropIndexDuringUpsertIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropIndexDuringUpsertIT.java index c1527dd3f38..5e88f9bbac4 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropIndexDuringUpsertIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropIndexDuringUpsertIT.java @@ -43,6 +43,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; +import org.apache.phoenix.end2end.ServerMetadataCacheTestImpl; import org.apache.phoenix.jdbc.PhoenixTestDriver; import org.apache.phoenix.query.BaseTest; import org.apache.phoenix.query.QueryServices; @@ -101,6 +102,7 @@ public void tearDown() throws Exception { service.shutdownNow(); destroyDriver(driver); } finally { + ServerMetadataCacheTestImpl.resetCache(); util.shutdownMiniCluster(); } } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexAsyncThresholdIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexAsyncThresholdIT.java index 51f77fae711..4732ebd7110 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexAsyncThresholdIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexAsyncThresholdIT.java @@ -17,6 +17,7 @@ */ package org.apache.phoenix.end2end.index; +import org.apache.phoenix.end2end.ServerMetadataCacheTestImpl; import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseConfiguration; @@ -123,6 +124,7 @@ public static synchronized void tearDownMiniCluster() throws Exception { } catch (Throwable t) { logger.error("Exception caught when shutting down mini cluster", t); } finally { + ServerMetadataCacheTestImpl.resetCache(); ConnectionFactory.shutdown(); } assertFalse("refCount leaked", refCountLeaked); @@ -160,9 +162,7 @@ public void testAsyncIndexCreation() throws Exception { exception = (SQLException) e; } connection.commit(); - PTableKey key = new PTableKey(null, this.tableName); - PMetaData metaCache = connection.unwrap(PhoenixConnection.class).getMetaDataCache(); - List indexes = metaCache.getTableRef(key).getTable().getIndexes(); + List indexes = connection.unwrap(PhoenixConnection.class).getTable(this.tableName).getIndexes(); if (!overThreshold) { if (this.mode == Mode.ASYNC) { assertEquals(PIndexState.BUILDING, indexes.get(0).getIndexState()); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java index 79a82094d86..9b7b2777793 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java @@ -17,6 +17,7 @@ */ package org.apache.phoenix.end2end.index; +import static org.apache.hadoop.hbase.coprocessor.CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY; import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -54,6 +55,7 @@ import org.apache.phoenix.coprocessor.MetaDataRegionObserver; import org.apache.phoenix.coprocessor.MetaDataRegionObserver.BuildIndexScheduleTask; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; +import org.apache.phoenix.end2end.PhoenixRegionServerEndpointTestImpl; import org.apache.phoenix.execute.CommitException; import org.apache.phoenix.hbase.index.write.IndexWriterUtils; import org.apache.phoenix.index.PhoenixIndexFailurePolicy; @@ -176,6 +178,7 @@ protected static Map getServerProps(){ * because we want to control it's execution ourselves */ serverProps.put(QueryServices.INDEX_REBUILD_TASK_INITIAL_DELAY, Long.toString(Long.MAX_VALUE)); + serverProps.put(REGIONSERVER_COPROCESSOR_CONF_KEY, PhoenixRegionServerEndpointTestImpl.class.getName()); return serverProps; } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ReplicationWithWALAnnotationIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ReplicationWithWALAnnotationIT.java index ee4e387d4b6..de24e4c3858 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ReplicationWithWALAnnotationIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ReplicationWithWALAnnotationIT.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.phoenix.coprocessor.ReplicationSinkEndpoint; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; +import org.apache.phoenix.end2end.ServerMetadataCacheTestImpl; import org.apache.phoenix.execute.MutationState; import org.apache.phoenix.hbase.index.IndexRegionObserver; import org.apache.phoenix.query.BaseTest; @@ -126,8 +127,12 @@ public static synchronized void setUpBeforeClass() throws Exception { @AfterClass public static void afterClass() throws Exception { - utility1.shutdownMiniCluster(); - utility2.shutdownMiniCluster(); + try { + utility1.shutdownMiniCluster(); + utility2.shutdownMiniCluster(); + } finally { + ServerMetadataCacheTestImpl.resetCache(); + } } private static void setupConfigsAndStartCluster() throws Exception { diff --git a/phoenix-core/src/it/java/org/apache/phoenix/execute/UpsertSelectOverlappingBatchesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/execute/UpsertSelectOverlappingBatchesIT.java index 4994a4b7d60..66c48566bb2 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/execute/UpsertSelectOverlappingBatchesIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/execute/UpsertSelectOverlappingBatchesIT.java @@ -52,6 +52,7 @@ import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.phoenix.end2end.ServerMetadataCacheTestImpl; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; import org.apache.phoenix.query.BaseTest; import org.apache.phoenix.query.QueryServices; @@ -88,6 +89,7 @@ public static synchronized void doSetup() throws Exception { @AfterClass public static synchronized void tearDownClass() throws Exception { SlowBatchRegionObserver.SLOW_MUTATE = false; + ServerMetadataCacheTestImpl.resetCache(); getUtility().shutdownMiniCluster(); } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/FailForUnsupportedHBaseVersionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/FailForUnsupportedHBaseVersionsIT.java index ab66e03955d..9ea6771fedc 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/FailForUnsupportedHBaseVersionsIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/hbase/index/FailForUnsupportedHBaseVersionsIT.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.VersionInfo; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; +import org.apache.phoenix.end2end.ServerMetadataCacheTestImpl; import org.apache.phoenix.hbase.index.covered.ColumnGroup; import org.apache.phoenix.hbase.index.covered.CoveredColumn; import org.apache.phoenix.hbase.index.covered.CoveredColumnIndexSpecifierBuilder; @@ -160,6 +161,7 @@ public void testDoesNotStartRegionServerForUnsupportedCompressionAndVersion() th } finally { // cleanup + ServerMetadataCacheTestImpl.resetCache(); util.shutdownMiniCluster(); } } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/iterate/ScannerLeaseRenewalIT.java b/phoenix-core/src/it/java/org/apache/phoenix/iterate/ScannerLeaseRenewalIT.java index 94700e04cf3..42876eae604 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/iterate/ScannerLeaseRenewalIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/iterate/ScannerLeaseRenewalIT.java @@ -42,6 +42,7 @@ import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixDriver; +import org.apache.phoenix.jdbc.PhoenixTestDriver; import org.apache.phoenix.query.ConnectionQueryServices.Feature; import org.apache.phoenix.util.PhoenixRuntime; import org.apache.phoenix.util.PropertiesUtil; @@ -87,6 +88,7 @@ public static synchronized void setUp() throws Exception { // use round robin iterator driverProps.put(FORCE_ROW_KEY_ORDER_ATTRIB, Boolean.toString(false)); DriverManager.registerDriver(PhoenixDriver.INSTANCE); + DriverManager.registerDriver(new PhoenixTestDriver()); try (PhoenixConnection phxConn = DriverManager.getConnection(url, driverProps).unwrap(PhoenixConnection.class)) { // run test methods only if we are at the hbase version that supports lease renewal. Assume.assumeTrue(phxConn.getQueryServices().supportsFeature(Feature.RENEW_LEASE)); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityTestingUtility.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityTestingUtility.java index 1cb6c0ba799..526b83e292e 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityTestingUtility.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/HighAvailabilityTestingUtility.java @@ -18,6 +18,9 @@ package org.apache.phoenix.jdbc; import org.apache.hadoop.hbase.StartMiniClusterOption; +import org.apache.phoenix.end2end.PhoenixRegionServerEndpointTestImpl; +import org.apache.phoenix.end2end.ServerMetadataCacheTestImpl; +import org.apache.phoenix.query.QueryServicesOptions; import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList; import org.apache.commons.lang3.RandomUtils; @@ -52,6 +55,7 @@ import java.util.concurrent.atomic.AtomicReference; import static org.apache.hadoop.hbase.HConstants.*; +import static org.apache.hadoop.hbase.coprocessor.CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY; import static org.apache.hadoop.hbase.ipc.RpcClient.*; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY; import static org.apache.hadoop.test.GenericTestUtils.waitFor; @@ -472,6 +476,7 @@ public void close() throws IOException { admin1.close(); admin2.close(); try { + ServerMetadataCacheTestImpl.resetCache(); hbaseCluster1.shutdownMiniCluster(); hbaseCluster2.shutdownMiniCluster(); } catch (Exception e) { @@ -534,6 +539,10 @@ private static void setUpDefaultHBaseConfig(Configuration conf) { // Hadoop cluster settings to avoid failing tests conf.setInt(DFS_REPLICATION_KEY, 1); // we only need one replica for testing + + // Phoenix Region Server Endpoint needed for metadata caching + conf.set(REGIONSERVER_COPROCESSOR_CONF_KEY, + PhoenixRegionServerEndpointTestImpl.class.getName()); } } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/LoggingSingleConnectionLimiterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/LoggingSingleConnectionLimiterIT.java index fb0ad2871d9..f5b1330defd 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/LoggingSingleConnectionLimiterIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/LoggingSingleConnectionLimiterIT.java @@ -88,7 +88,7 @@ public static void doSetup() throws Exception { String zkQuorum = "localhost:" + hBaseTestingUtility.getZkCluster().getClientPort(); url = PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum; DriverManager.registerDriver(PhoenixDriver.INSTANCE); - + DriverManager.registerDriver(new PhoenixTestDriver()); String profileName = "setup"; final String urlWithPrinc = url + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + profileName + PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR; diff --git a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionIT.java index 7ecacd2261b..5772da8f9e8 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionIT.java @@ -97,6 +97,7 @@ public class ParallelPhoenixConnectionIT { public static void setUpBeforeClass() throws Exception { CLUSTERS.start(); DriverManager.registerDriver(PhoenixDriver.INSTANCE); + DriverManager.registerDriver(new PhoenixTestDriver()); GLOBAL_PROPERTIES.setProperty(AUTO_COMMIT_ATTRIB, "true"); GLOBAL_PROPERTIES.setProperty(QueryServices.COLLECT_REQUEST_LEVEL_METRICS, String.valueOf(true)); GLOBAL_PROPERTIES.setProperty(QueryServices.LOG_LEVEL, LogLevel.DEBUG.name()); //Need logging for query metrics diff --git a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixTableLevelMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixTableLevelMetricsIT.java index ab00c2ace41..0031957cfa3 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixTableLevelMetricsIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixTableLevelMetricsIT.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException; import org.apache.hbase.thirdparty.com.google.common.collect.Maps; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; +import org.apache.phoenix.end2end.ServerMetadataCacheTestImpl; import org.apache.phoenix.exception.PhoenixIOException; import org.apache.phoenix.exception.SQLExceptionInfo; import org.apache.phoenix.execute.CommitException; @@ -37,6 +38,7 @@ import org.apache.phoenix.query.ConnectionQueryServices; import org.apache.phoenix.query.ConnectionQueryServicesImpl; import org.apache.phoenix.query.QueryServices; +import org.apache.phoenix.query.QueryServicesOptions; import org.apache.phoenix.query.QueryServicesTestImpl; import org.apache.phoenix.util.EnvironmentEdge; import org.apache.phoenix.util.EnvironmentEdgeManager; @@ -45,6 +47,7 @@ import org.apache.phoenix.util.DelayedOrFailingRegionServer; import org.apache.phoenix.util.ReadOnlyProps; +import org.apache.phoenix.util.ValidateLastDDLTimestampUtil; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; @@ -53,6 +56,8 @@ import org.junit.Test; import org.junit.experimental.categories.Category; +import javax.annotation.concurrent.GuardedBy; + import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME; import static org.apache.phoenix.exception.SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY; import static org.apache.phoenix.exception.SQLExceptionCode.GET_TABLE_REGIONS_FAIL; @@ -132,6 +137,7 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; @@ -152,6 +158,7 @@ public class PhoenixTableLevelMetricsIT extends BaseTest { @BeforeClass public static void doSetup() throws Exception { final Configuration conf = HBaseConfiguration.create(); + setUpConfigForMiniCluster(conf); conf.set(QueryServices.TABLE_LEVEL_METRICS_ENABLED, String.valueOf(true)); conf.set(QueryServices.METRIC_PUBLISHER_ENABLED, String.valueOf(true)); conf.set(QueryServices.COLLECT_REQUEST_LEVEL_METRICS, String.valueOf(true)); @@ -170,7 +177,7 @@ public class PhoenixTableLevelMetricsIT extends BaseTest { return copy; } }); - hbaseTestUtil = new HBaseTestingUtility(); + hbaseTestUtil = new HBaseTestingUtility(conf); hbaseTestUtil.startMiniCluster(1, 1, null, null, DelayedOrFailingRegionServer.class); // establish url and quorum. Need to use PhoenixDriver and not PhoenixTestDriver String zkQuorum = "localhost:" + hbaseTestUtil.getZkCluster().getClientPort(); @@ -189,6 +196,8 @@ public class PhoenixTableLevelMetricsIT extends BaseTest { } } catch (Exception e) { // ignore + } finally { + ServerMetadataCacheTestImpl.resetCache(); } } @@ -1273,14 +1282,21 @@ public void testMetricsWithIndexUsage() throws Exception { assertTrue(metricExists); metricExists = false; //assert BaseTable is not being queried - for (PhoenixTableMetric metric : getPhoenixTableClientMetrics().get(dataTable)) { - if (metric.getMetricType().equals(SELECT_SQL_COUNTER)) { - metricExists = true; - assertMetricValue(metric, SELECT_SQL_COUNTER, 0, CompareOp.EQ); - break; + //if client is validating last_ddl_timestamps with ucf=never, + //there will be no metrics for base table (like getTable RPC times/counts). + if (ValidateLastDDLTimestampUtil + .getValidateLastDdlTimestampEnabled(conn.unwrap(PhoenixConnection.class))) { + assertFalse(getPhoenixTableClientMetrics().containsKey(dataTable)); + } else { + for (PhoenixTableMetric metric : getPhoenixTableClientMetrics().get(dataTable)) { + if (metric.getMetricType().equals(SELECT_SQL_COUNTER)) { + metricExists = true; + assertMetricValue(metric, SELECT_SQL_COUNTER, 0, CompareOp.EQ); + break; + } } + assertTrue(metricExists); } - assertTrue(metricExists); } } } @@ -1613,11 +1629,16 @@ public List getTableRegions(byte[] tableName, byte[] startRowKe * Custom driver to return a custom QueryServices object */ public static class PhoenixMetricsTestingDriver extends PhoenixTestDriver { - private ConnectionQueryServices cqs; + @GuardedBy("this") + private final Map + connectionQueryServicesMap = new HashMap<>(); + + private final QueryServices qsti; private ReadOnlyProps overrideProps; public PhoenixMetricsTestingDriver(ReadOnlyProps props) { overrideProps = props; + qsti = new QueryServicesTestImpl(getDefaultProps(), overrideProps); } @Override public boolean acceptsURL(String url) { @@ -1625,17 +1646,16 @@ public PhoenixMetricsTestingDriver(ReadOnlyProps props) { } @Override public synchronized ConnectionQueryServices getConnectionQueryServices(String url, - Properties info) throws SQLException { - if (cqs == null) { - QueryServicesTestImpl qsti = - new QueryServicesTestImpl(getDefaultProps(), overrideProps); - cqs = - new PhoenixMetricsTestingQueryServices( - qsti, - ConnectionInfo.create(url, qsti.getProps(), info), info); - cqs.init(url, info); + Properties info) throws SQLException { + ConnectionInfo connInfo = ConnectionInfo.create(url, null, info); + ConnectionQueryServices connectionQueryServices = connectionQueryServicesMap.get(connInfo); + if (connectionQueryServices != null) { + return connectionQueryServices; } - return cqs; + connectionQueryServices = new PhoenixMetricsTestingQueryServices(qsti, connInfo, info); + connectionQueryServices.init(url, info); + connectionQueryServicesMap.put(connInfo, connectionQueryServices); + return connectionQueryServices; } } } \ No newline at end of file diff --git a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/connectionqueryservice/ConnectionQueryServicesMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/connectionqueryservice/ConnectionQueryServicesMetricsIT.java index 454662b28aa..0b3b6236a0a 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/connectionqueryservice/ConnectionQueryServicesMetricsIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/connectionqueryservice/ConnectionQueryServicesMetricsIT.java @@ -21,6 +21,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; +import org.apache.phoenix.end2end.ServerMetadataCacheTestImpl; import org.apache.phoenix.jdbc.ConnectionInfo; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixDriver; @@ -127,6 +128,8 @@ public static void tearDownMiniCluster() { } } catch (Exception e) { // ignore + } finally { + ServerMetadataCacheTestImpl.resetCache(); } } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/query/MaxConcurrentConnectionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/query/MaxConcurrentConnectionsIT.java index 5adeeb18e34..dacd07ee199 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/query/MaxConcurrentConnectionsIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/query/MaxConcurrentConnectionsIT.java @@ -18,10 +18,14 @@ package org.apache.phoenix.query; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; +import org.apache.phoenix.end2end.ServerMetadataCacheTestImpl; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixDriver; +import org.apache.phoenix.jdbc.PhoenixTestDriver; import org.apache.phoenix.util.DelayedRegionServer; import org.apache.phoenix.util.PhoenixRuntime; import org.junit.AfterClass; @@ -56,7 +60,9 @@ public class MaxConcurrentConnectionsIT extends BaseTest { @BeforeClass public static void setUp() throws Exception { - hbaseTestUtil = new HBaseTestingUtility(); + final Configuration conf = HBaseConfiguration.create(); + setUpConfigForMiniCluster(conf); + hbaseTestUtil = new HBaseTestingUtility(conf); hbaseTestUtil.startMiniCluster(1,1,null,null,DelayedRegionServer.class); // establish url and quorum. Need to use PhoenixDriver and not PhoenixTestDriver @@ -64,6 +70,7 @@ public static void setUp() throws Exception { url = PhoenixRuntime.JDBC_PROTOCOL + JDBC_PROTOCOL_SEPARATOR + zkQuorum + JDBC_PROTOCOL_SEPARATOR + "uniqueConn=A"; DriverManager.registerDriver(PhoenixDriver.INSTANCE); + DriverManager.registerDriver(new PhoenixTestDriver()); } private String getUniqueUrl() { @@ -73,6 +80,7 @@ private String getUniqueUrl() { //Have to shutdown our special delayed region server @AfterClass public static void tearDown() throws Exception { + ServerMetadataCacheTestImpl.resetCache(); hbaseTestUtil.shutdownMiniCluster(); } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/query/MetaDataCachingIT.java b/phoenix-core/src/it/java/org/apache/phoenix/query/MetaDataCachingIT.java index 390df78b0f2..06cc176fa01 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/query/MetaDataCachingIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/query/MetaDataCachingIT.java @@ -25,6 +25,7 @@ import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; import org.apache.phoenix.util.ReadOnlyProps; import org.apache.phoenix.util.RunUntilFailure; +import org.apache.phoenix.util.ValidateLastDDLTimestampUtil; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Test; @@ -51,6 +52,9 @@ public class MetaDataCachingIT extends BaseTest { private static final Logger LOGGER = LoggerFactory.getLogger(MetaDataCachingIT.class); private final Random RAND = new Random(11); + private boolean isLastDDLTimestampValidationEnabled + = ValidateLastDDLTimestampUtil.getValidateLastDdlTimestampEnabled(config); + @BeforeClass public static synchronized void doSetup() throws Exception { Map props = Maps.newHashMapWithExpectedSize(1); @@ -143,7 +147,7 @@ public void testSystemTablesAreInCache() throws Exception { @Test public void testGlobalClientCacheMetrics() throws Exception { int numThreads = 5; - int numTables = 1; + int numTables = 1; // test with only 1 table because we pick tables randomly in the workload int numMaxDML = 2; GlobalClientMetrics.GLOBAL_CLIENT_METADATA_CACHE_MISS_COUNTER.getMetric().reset(); @@ -152,12 +156,19 @@ public void testGlobalClientCacheMetrics() throws Exception { simulateWorkload("testGlobalClientCacheMetrics", numTables, numThreads, numMaxDML); // only 1 miss when the table is created + int numExpectedMisses = 1; + if (isLastDDLTimestampValidationEnabled) { + // if we are validating last_ddl_timestamps, + // region server will see 2 more misses when trying to update its cache + numExpectedMisses += 2; + } + assertEquals("Incorrect number of client metadata cache misses", - 1, GlobalClientMetrics.GLOBAL_CLIENT_METADATA_CACHE_MISS_COUNTER.getMetric().getValue()); + numExpectedMisses, GlobalClientMetrics.GLOBAL_CLIENT_METADATA_CACHE_MISS_COUNTER.getMetric().getValue()); // (2 hits per upsert + 1 hit per select) per thread - assertEquals("Incorrect number of client metadata cache hits", - 3*numMaxDML*numThreads, GlobalClientMetrics.GLOBAL_CLIENT_METADATA_CACHE_HIT_COUNTER.getMetric().getValue()); + assertTrue("number of total metadata cache hits (server+client) should be more than or equal to client cache hits", + 3*numMaxDML*numThreads <= GlobalClientMetrics.GLOBAL_CLIENT_METADATA_CACHE_HIT_COUNTER.getMetric().getValue()); } /* diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java index b1f063f57bd..59459e81fea 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheConnectionLevelPropIT.java @@ -22,11 +22,13 @@ import org.apache.phoenix.query.ConnectionQueryServices; import org.apache.phoenix.query.QueryConstants; import org.apache.phoenix.query.QueryServices; +import org.apache.phoenix.query.QueryServicesOptions; import org.apache.phoenix.schema.PName; import org.apache.phoenix.schema.types.PVarchar; import org.apache.phoenix.util.SchemaUtil; import static org.apache.phoenix.util.TestUtil.DEFAULT_SCHEMA_NAME; +import org.apache.phoenix.util.ValidateLastDDLTimestampUtil; import org.junit.AfterClass; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -61,6 +63,8 @@ public class UpdateCacheConnectionLevelPropIT extends ParallelStatsDisabledIT { private static Connection conn1; private static Connection conn2; private static ConnectionQueryServices spyForConn2; + private boolean isLastDDLTimestampValidationEnabled + = ValidateLastDDLTimestampUtil.getValidateLastDdlTimestampEnabled(config); @AfterClass public static synchronized void freeResources() { @@ -135,6 +139,12 @@ public void testWithNoConnAndTableLevelUCF() throws Exception { // both connection and table level properties are not set int numExecutions = 2; int numExpectedGetTableCalls = 4; // 2 for SELECTs, and 2 for UPSERTs + + // there will be no getTable calls if we are validating last_ddl_timestamps + // and schema has not changed. + if (isLastDDLTimestampValidationEnabled) { + numExpectedGetTableCalls = 0; + } setUpTableAndConnections(fullTableName, null, null); verifyExpectedGetTableCalls(fullTableName, numExecutions, numExpectedGetTableCalls); } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java index 70a9ff759a8..02e90c124dd 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java @@ -48,6 +48,7 @@ import org.apache.phoenix.query.ConnectionQueryServices; import org.apache.phoenix.query.QueryConstants; import org.apache.phoenix.query.QueryServices; +import org.apache.phoenix.query.QueryServicesOptions; import org.apache.phoenix.schema.MetaDataClient; import org.apache.phoenix.schema.PName; import org.apache.phoenix.schema.types.PVarchar; @@ -56,6 +57,7 @@ import org.apache.phoenix.util.PropertiesUtil; import org.apache.phoenix.util.SchemaUtil; import org.apache.phoenix.util.TestUtil; +import org.apache.phoenix.util.ValidateLastDDLTimestampUtil; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -69,6 +71,8 @@ @Category(ParallelStatsDisabledTest.class) public class UpdateCacheIT extends ParallelStatsDisabledIT { + private boolean isLastDDLTimestampValidationEnabled + = ValidateLastDDLTimestampUtil.getValidateLastDdlTimestampEnabled(config); private static final Logger LOGGER = LoggerFactory.getLogger(UpdateCacheIT.class); @@ -87,7 +91,13 @@ public void testUpdateCacheForTxnTable() throws Exception { String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + tableName; Connection conn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)); conn.createStatement().execute("create table " + fullTableName + TestUtil.TEST_TABLE_SCHEMA + "TRANSACTIONAL=true,TRANSACTION_PROVIDER='" + provider + "'"); - helpTestUpdateCache(fullTableName, new int[] {1, 3}, false); + int[] expectedRPCs = new int[] {1, 3}; + if (isLastDDLTimestampValidationEnabled) { + // when validating last_ddl_timestamps, no getTable RPCs will be performed + // since schema has not changed. + expectedRPCs = new int[] {0, 0}; + } + helpTestUpdateCache(fullTableName, expectedRPCs, false); } } @@ -97,7 +107,13 @@ public void testUpdateCacheForNonTxnTable() throws Exception { String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + tableName; Connection conn = DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES)); conn.createStatement().execute("create table " + fullTableName + TestUtil.TEST_TABLE_SCHEMA); - helpTestUpdateCache(fullTableName, new int[] {1, 3}, false); + int[] expectedRPCs = new int[] {1, 3}; + if (isLastDDLTimestampValidationEnabled) { + // when validating last_ddl_timestamps, no getTable RPCs will be performed + // since schema has not changed. + expectedRPCs = new int[] {0, 0}; + } + helpTestUpdateCache(fullTableName, expectedRPCs, false); } @Test @@ -217,7 +233,14 @@ public void testUpdateCacheFreqPropagatedToIndexes() throws Exception { } // The indexes should have got the UPDATE_CACHE_FREQUENCY value of their base table - helpTestUpdateCache(fullTableName, new int[] {0, 0}, false); + int numRPCUpsert = 0; + int numRPCSelect = 0; + if (isLastDDLTimestampValidationEnabled) { + // we created indexes on the table which will bump the last_ddl_timestamp of the table + // hence we will do 1 getTable RPC for the upsert + numRPCUpsert = 1; + } + helpTestUpdateCache(fullTableName, new int[] {numRPCUpsert, numRPCSelect}, false); helpTestUpdateCache(INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + localIndex, new int[] {0}, true); helpTestUpdateCache(INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + globalIndex, @@ -231,7 +254,7 @@ public void testUpdateCacheFreqPropagatedToIndexes() throws Exception { // Even the indexes should now have the modified value of UPDATE_CACHE_FREQUENCY // Note that when we query the base table, during query plan generation, we make 2 getTable // requests (to retrieve the base table) for each index of the base table - helpTestUpdateCache(fullTableName, new int[] {1, 18}, false); + helpTestUpdateCache(fullTableName, new int[] {1, 21}, false); helpTestUpdateCache(INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + localIndex, new int[] {3}, true); helpTestUpdateCache(INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + globalIndex, diff --git a/phoenix-core/src/test/java/org/apache/hadoop/hbase/ipc/PhoenixIndexRpcSchedulerTest.java b/phoenix-core/src/test/java/org/apache/hadoop/hbase/ipc/PhoenixRpcSchedulerTest.java similarity index 75% rename from phoenix-core/src/test/java/org/apache/hadoop/hbase/ipc/PhoenixIndexRpcSchedulerTest.java rename to phoenix-core/src/test/java/org/apache/hadoop/hbase/ipc/PhoenixRpcSchedulerTest.java index 15b83e41b50..aa629c31184 100644 --- a/phoenix-core/src/test/java/org/apache/hadoop/hbase/ipc/PhoenixIndexRpcSchedulerTest.java +++ b/phoenix-core/src/test/java/org/apache/hadoop/hbase/ipc/PhoenixRpcSchedulerTest.java @@ -19,6 +19,7 @@ import static org.apache.hadoop.hbase.ipc.TestProtobufRpcServiceImpl.SERVICE; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.mockito.Mockito.when; import java.net.InetSocketAddress; @@ -38,16 +39,9 @@ import org.apache.phoenix.thirdparty.com.google.common.collect.Lists; -/** - * Test that the rpc scheduler schedules index writes to the index handler queue and sends - * everything else to the standard queues - */ -public class PhoenixIndexRpcSchedulerTest { - +public class PhoenixRpcSchedulerTest { private static final Configuration conf = HBaseConfiguration.create(); private static final InetSocketAddress isa = new InetSocketAddress("localhost", 0); - - private class AbortServer implements Abortable { private boolean aborted = false; @@ -62,25 +56,29 @@ public boolean isAborted() { } } + /** + * Test that the rpc scheduler schedules index writes to the index handler queue and sends + * everything else to the standard queues + */ @Test public void testIndexPriorityWritesToIndexHandler() throws Exception { RpcScheduler mock = Mockito.mock(RpcScheduler.class); PriorityFunction qosFunction = Mockito.mock(PriorityFunction.class); Abortable abortable = new AbortServer(); - PhoenixRpcScheduler scheduler = new PhoenixRpcScheduler(conf, mock, 200, 250, 225, qosFunction,abortable); + PhoenixRpcScheduler scheduler = new PhoenixRpcScheduler(conf, mock, 200, 250, 225, 230, qosFunction,abortable); BalancedQueueRpcExecutor executor = new BalancedQueueRpcExecutor("test-queue", 1, 1,qosFunction,conf,abortable); scheduler.setIndexExecutorForTesting(executor); dispatchCallWithPriority(scheduler, 200); List> queues = executor.getQueues(); assertEquals(1, queues.size()); BlockingQueue queue = queues.get(0); - queue.poll(20, TimeUnit.SECONDS); + assertNotNull(queue.poll(5, TimeUnit.SECONDS)); // try again, this time we tweak the ranges we support - scheduler = new PhoenixRpcScheduler(conf, mock, 101, 110, 105, qosFunction,abortable); + scheduler = new PhoenixRpcScheduler(conf, mock, 101, 110, 105, 115, qosFunction,abortable); scheduler.setIndexExecutorForTesting(executor); dispatchCallWithPriority(scheduler, 101); - queue.poll(20, TimeUnit.SECONDS); + assertNotNull(queue.poll(5, TimeUnit.SECONDS)); Mockito.verify(mock, Mockito.times(2)).init(Mockito.any(Context.class)); scheduler.stop(); @@ -92,7 +90,7 @@ public void testServerSideRPCalls() throws Exception { RpcScheduler mock = Mockito.mock(RpcScheduler.class); PriorityFunction qosFunction = Mockito.mock(PriorityFunction.class); Abortable abortable = new AbortServer(); - PhoenixRpcScheduler scheduler1 = new PhoenixRpcScheduler(conf, mock, 200, 250, 100, qosFunction,abortable); + PhoenixRpcScheduler scheduler1 = new PhoenixRpcScheduler(conf, mock, 200, 250, 100, 300, qosFunction,abortable); RpcExecutor executor1 = scheduler1.getServerSideExecutorForTesting(); for (int c = 0; c < 10; c++) { dispatchCallWithPriority(scheduler1, 100); @@ -103,7 +101,7 @@ public void testServerSideRPCalls() throws Exception { if (queue1.size() > 0) { numDispatches1 += queue1.size(); for (int i = 0; i < queue1.size(); i++) { - queue1.poll(20, TimeUnit.SECONDS); + assertNotNull(queue1.poll(5, TimeUnit.SECONDS)); } } } @@ -111,7 +109,7 @@ public void testServerSideRPCalls() throws Exception { scheduler1.stop(); // try again, with the incorrect executor - PhoenixRpcScheduler scheduler2 = new PhoenixRpcScheduler(conf, mock, 101, 110, 50, qosFunction,abortable); + PhoenixRpcScheduler scheduler2 = new PhoenixRpcScheduler(conf, mock, 101, 110, 50, 25, qosFunction,abortable); RpcExecutor executor2 = scheduler2.getIndexExecutorForTesting(); dispatchCallWithPriority(scheduler2, 50); List> queues2 = executor2.getQueues(); @@ -119,7 +117,7 @@ public void testServerSideRPCalls() throws Exception { for (BlockingQueue queue2 : queues2) { if (queue2.size() > 0) { numDispatches2++; - queue2.poll(20, TimeUnit.SECONDS); + assertNotNull(queue2.poll(5, TimeUnit.SECONDS)); } } assertEquals(0, numDispatches2); @@ -140,12 +138,12 @@ public void testDelegateWhenOutsideRange() throws Exception { PriorityFunction qosFunction = Mockito.mock(PriorityFunction.class); Abortable abortable = new AbortServer(); RpcScheduler mock = Mockito.mock(RpcScheduler.class); - PhoenixRpcScheduler scheduler = new PhoenixRpcScheduler(conf, mock, 200, 250, 225, qosFunction,abortable); + PhoenixRpcScheduler scheduler = new PhoenixRpcScheduler(conf, mock, 200, 250, 225, 275, qosFunction,abortable); dispatchCallWithPriority(scheduler, 100); dispatchCallWithPriority(scheduler, 251); // try again, this time we tweak the ranges we support - scheduler = new PhoenixRpcScheduler(conf, mock, 101, 110, 105, qosFunction,abortable); + scheduler = new PhoenixRpcScheduler(conf, mock, 101, 110, 105, 115, qosFunction,abortable); dispatchCallWithPriority(scheduler, 200); dispatchCallWithPriority(scheduler, 111); @@ -154,6 +152,33 @@ public void testDelegateWhenOutsideRange() throws Exception { scheduler.stop(); } + /** + * Test that the rpc scheduler schedules invalidate metadata cache RPC to + * the invalidate metadata cache executor. + */ + @Test + public void testInvalidateMetadataCacheExecutor() throws Exception { + RpcScheduler mock = Mockito.mock(RpcScheduler.class); + PriorityFunction qosFunction = Mockito.mock(PriorityFunction.class); + Abortable abortable = new AbortServer(); + // Set invalidate metadata cache priority to 230. + int invalidateMetadataCacheCallPriority = 230; + PhoenixRpcScheduler scheduler = new PhoenixRpcScheduler(conf, mock, + 200, 250, 225, invalidateMetadataCacheCallPriority, qosFunction,abortable); + BalancedQueueRpcExecutor executor = new BalancedQueueRpcExecutor("test-queue", + 1, 1, qosFunction, conf, abortable); + scheduler.setInvalidateMetadataCacheExecutorForTesting(executor); + dispatchCallWithPriority(scheduler, invalidateMetadataCacheCallPriority); + List> queues = executor.getQueues(); + assertEquals(1, queues.size()); + BlockingQueue queue = queues.get(0); + assertEquals(1, queue.size()); + assertNotNull(queue.poll(5, TimeUnit.SECONDS)); + Mockito.verify(mock, Mockito.times(1)).init(Mockito.any(RpcScheduler.Context.class)); + scheduler.stop(); + executor.stop(); + } + private void dispatchCallWithPriority(RpcScheduler scheduler, int priority) throws Exception { CallRunner task = Mockito.mock(CallRunner.class); RequestHeader header = RequestHeader.newBuilder().setPriority(priority).build(); diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java index 2f58508443a..c905c066a18 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereOptimizerTest.java @@ -3544,6 +3544,49 @@ public void testWithLargeORs() throws Exception { } } + /** + * Test that tenantId is present in the scan start row key when using an inherited index on a tenant view. + */ + @Test + public void testScanKeyInheritedIndexTenantView() throws Exception { + String baseTableName = generateUniqueName(); + String globalViewName = generateUniqueName(); + String globalViewIndexName = generateUniqueName(); + String tenantViewName = generateUniqueName(); + try (Connection conn = DriverManager.getConnection(getUrl())) { + // create table, view and view index + conn.createStatement().execute("CREATE TABLE " + baseTableName + + " (TENANT_ID CHAR(8) NOT NULL, KP CHAR(3) NOT NULL, PK CHAR(3) NOT NULL, KV CHAR(2), KV2 CHAR(2) " + + "CONSTRAINT PK PRIMARY KEY(TENANT_ID, KP, PK)) MULTI_TENANT=true"); + conn.createStatement().execute("CREATE VIEW " + globalViewName + + " AS SELECT * FROM " + baseTableName + " WHERE KP = '001'"); + conn.createStatement().execute("CREATE INDEX " + globalViewIndexName + " on " + + globalViewName + " (KV) " + " INCLUDE (KV2)"); + //create tenant view + String tenantId = "tenantId"; + Properties tenantProps = new Properties(); + tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId); + try (Connection tenantConn = DriverManager.getConnection(getUrl(), tenantProps)) { + tenantConn.createStatement().execute("CREATE VIEW " + tenantViewName + " AS SELECT * FROM " + globalViewName); + // query on secondary key + String query = "SELECT KV2 FROM " + tenantViewName + " WHERE KV = 'KV'"; + PhoenixConnection pconn = tenantConn.unwrap(PhoenixConnection.class); + PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query); + QueryPlan plan = pstmt.compileQuery(); + plan = tenantConn.unwrap(PhoenixConnection.class).getQueryServices().getOptimizer().optimize(pstmt, plan); + // optimized query plan should use inherited index + assertEquals(tenantViewName + "#" + globalViewIndexName, plan.getContext().getCurrentTable().getTable().getName().getString()); + Scan scan = plan.getContext().getScan(); + PTable viewIndexPTable = tenantConn.unwrap(PhoenixConnection.class).getTable(globalViewIndexName); + // PK of view index [_INDEX_ID, tenant_id, KV, PK] + byte[] startRow = ByteUtil.concat(PLong.INSTANCE.toBytes(viewIndexPTable.getViewIndexId()), + PChar.INSTANCE.toBytes(tenantId), + PChar.INSTANCE.toBytes("KV")); + assertArrayEquals(startRow, scan.getStartRow()); + } + } + } + private void createBaseTable(String baseTable) throws SQLException { try (Connection globalConnection = DriverManager.getConnection(getUrl())) { diff --git a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java index 3fb7bffb5ad..ebea1f1effb 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/hbase/index/write/TestWALRecoveryCaching.java @@ -57,6 +57,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread; import org.apache.hadoop.hbase.wal.WALKey; +import org.apache.phoenix.end2end.ServerMetadataCacheTestImpl; import org.apache.phoenix.hbase.index.IndexTableName; import org.apache.phoenix.hbase.index.IndexTestingUtils; import org.apache.phoenix.hbase.index.Indexer; @@ -269,6 +270,7 @@ public void testWaitsOnIndexRegionToReload() throws Exception { scanner.close(); index.close(); primary.close(); + ServerMetadataCacheTestImpl.resetCache(); util.shutdownMiniCluster(); } diff --git a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionFailureTest.java b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionFailureTest.java index c64252d5304..51d89728bfb 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionFailureTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/ParallelPhoenixConnectionFailureTest.java @@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.phoenix.query.BaseTest; +import org.apache.phoenix.query.QueryServices; import org.apache.phoenix.util.PhoenixRuntime; import org.junit.Test; import org.mockito.Mockito; @@ -51,9 +52,9 @@ public class ParallelPhoenixConnectionFailureTest extends BaseTest { @Test public void testExecuteQueryChainFailure() throws SQLException { HBaseTestingUtility hbaseTestingUtility = new HBaseTestingUtility(); - - PhoenixConnection conn1 = (PhoenixConnection) DriverManager.getConnection(url); - PhoenixConnection conn2 = (PhoenixConnection) DriverManager.getConnection(url); + Properties props = new Properties(); + PhoenixConnection conn1 = (PhoenixConnection) DriverManager.getConnection(url, props); + PhoenixConnection conn2 = (PhoenixConnection) DriverManager.getConnection(url, props); PhoenixConnection connSpy1 = Mockito.spy(conn1); PhoenixConnection connSpy2 = Mockito.spy(conn2); AtomicInteger numStatementsCreatedOnConn1 = new AtomicInteger(); diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java index f8350e49406..6a9eb7b8a33 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java @@ -35,6 +35,7 @@ import java.sql.DriverManager; import java.sql.SQLException; +import java.util.Map; import java.util.Properties; import org.apache.hadoop.hbase.HConstants; @@ -49,6 +50,7 @@ import org.apache.phoenix.schema.PTable; import org.apache.phoenix.schema.PTableKey; import org.apache.phoenix.schema.TableRef; +import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; import org.apache.phoenix.util.PhoenixRuntime; import org.apache.phoenix.util.PropertiesUtil; import org.apache.phoenix.util.ReadOnlyProps; @@ -86,7 +88,8 @@ private static void startServer(String url) throws Exception { // only load the test driver if we are testing locally - for integration tests, we want to // test on a wider scale if (PhoenixEmbeddedDriver.isTestUrl(url)) { - driver = initDriver(ReadOnlyProps.EMPTY_PROPS); + Map props = Maps.newHashMapWithExpectedSize(1); + driver = initDriver(new ReadOnlyProps(props)); assertTrue(DriverManager.getDriver(url) == driver); driver.connect(url, PropertiesUtil.deepCopy(TEST_PROPERTIES)); } diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java index 76175e4dc99..1405ad6355c 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java @@ -17,6 +17,7 @@ */ package org.apache.phoenix.query; +import static org.apache.hadoop.hbase.coprocessor.CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY; import static org.apache.phoenix.hbase.index.write.ParallelWriterIndexCommitter.NUM_CONCURRENT_INDEX_WRITER_THREADS_CONF_KEY; import static org.apache.phoenix.query.QueryConstants.MILLIS_IN_DAY; import static org.apache.phoenix.query.QueryServices.DROP_METADATA_ATTRIB; @@ -140,11 +141,14 @@ import org.apache.hadoop.hbase.regionserver.RSRpcServices; import org.apache.hadoop.hbase.util.Bytes; import org.apache.phoenix.SystemExitRule; +import org.apache.phoenix.compat.hbase.CompatUtil; import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; import org.apache.phoenix.end2end.ParallelStatsDisabledIT; import org.apache.phoenix.end2end.ParallelStatsDisabledTest; import org.apache.phoenix.end2end.ParallelStatsEnabledIT; import org.apache.phoenix.end2end.ParallelStatsEnabledTest; +import org.apache.phoenix.end2end.PhoenixRegionServerEndpointTestImpl; +import org.apache.phoenix.end2end.ServerMetadataCacheTestImpl; import org.apache.phoenix.exception.SQLExceptionCode; import org.apache.phoenix.exception.SQLExceptionInfo; import org.apache.phoenix.hbase.index.IndexRegionObserver; @@ -494,6 +498,8 @@ public static synchronized void tearDownMiniCluster(final int numTables) { LOGGER.error("Exception caught when shutting down mini map reduce cluster", t); } finally { try { + // Clear ServerMetadataCache. + ServerMetadataCacheTestImpl.resetCache(); utility.shutdownMiniCluster(); } catch (Throwable t) { LOGGER.error("Exception caught when shutting down mini cluster", t); @@ -641,9 +647,25 @@ public static Configuration setUpConfigForMiniCluster(Configuration conf, ReadOn if (conf.getLong(QueryServices.PHOENIX_SERVER_PAGE_SIZE_MS, 0) == 0) { conf.setLong(QueryServices.PHOENIX_SERVER_PAGE_SIZE_MS, 0); } + setPhoenixRegionServerEndpoint(conf); return conf; } + /* + Set property hbase.coprocessor.regionserver.classes to include test implementation of + PhoenixRegionServerEndpoint by default, if some other regionserver coprocs + are not already present. + */ + protected static void setPhoenixRegionServerEndpoint(Configuration conf) { + String value = conf.get(REGIONSERVER_COPROCESSOR_CONF_KEY); + if (value == null) { + value = PhoenixRegionServerEndpointTestImpl.class.getName(); + } + else { + value = value + "," + PhoenixRegionServerEndpointTestImpl.class.getName(); + } + conf.set(REGIONSERVER_COPROCESSOR_CONF_KEY, value); + } private static PhoenixTestDriver newTestDriver(ReadOnlyProps props) throws Exception { PhoenixTestDriver newDriver; String driverClassName = props.get(DRIVER_CLASS_NAME_ATTRIB);