From 0c2fec1e34cd7646b6f11dbb164d29dc7bbe89b2 Mon Sep 17 00:00:00 2001 From: tkhurana Date: Thu, 3 Oct 2024 13:57:24 -0700 Subject: [PATCH 1/2] PHOENIX-7170 Conditional TTL --- .../coprocessorclient/TableTTLInfo.java | 33 +- .../phoenix/exception/SQLExceptionCode.java | 6 + .../phoenix/iterate/BaseResultIterators.java | 8 +- .../phoenix/jdbc/PhoenixDatabaseMetaData.java | 2 + .../query/ConnectionQueryServicesImpl.java | 11 +- .../schema/ConditionalTTLExpression.java | 493 +++++++++ .../apache/phoenix/schema/DelegateTable.java | 7 +- .../phoenix/schema/LiteralTTLExpression.java | 107 ++ .../apache/phoenix/schema/MetaDataClient.java | 153 ++- .../org/apache/phoenix/schema/PTable.java | 10 +- .../org/apache/phoenix/schema/PTableImpl.java | 96 +- .../apache/phoenix/schema/TTLExpression.java | 128 +++ .../apache/phoenix/schema/TableProperty.java | 12 +- .../org/apache/phoenix/util/ScanUtil.java | 61 +- .../org/apache/phoenix/util/SchemaUtil.java | 9 +- .../src/main/protobuf/PTable.proto | 16 + .../coprocessor/CompactionScanner.java | 348 ++++--- .../coprocessor/GlobalIndexRegionScanner.java | 32 +- .../IndexRebuildRegionScanner.java | 5 - .../coprocessor/IndexRepairRegionScanner.java | 5 - .../coprocessor/IndexerRegionScanner.java | 29 - .../coprocessor/MetaDataEndpointImpl.java | 42 +- .../phoenix/coprocessor/TTLRegionScanner.java | 32 +- .../mapreduce/index/IndexScrutinyMapper.java | 7 +- .../DefaultPhoenixMultiViewListProvider.java | 2 +- .../apache/phoenix/end2end/BaseViewTTLIT.java | 239 +++-- .../org/apache/phoenix/end2end/CDCBaseIT.java | 3 + .../phoenix/end2end/CDCDefinitionIT.java | 2 +- .../apache/phoenix/end2end/CreateTableIT.java | 13 +- .../end2end/IndexRepairRegionScannerIT.java | 9 +- .../apache/phoenix/end2end/IndexToolIT.java | 12 + .../apache/phoenix/end2end/SetPropertyIT.java | 18 +- .../phoenix/end2end/TTLAsPhoenixTTLIT.java | 369 +++++-- .../org/apache/phoenix/end2end/TTLIT.java | 10 +- .../org/apache/phoenix/end2end/ViewTTLIT.java | 173 ++-- .../ViewTTLWithLongViewIndexEnabledIT.java | 11 +- .../schema/ConditionalTTLExpressionIT.java | 964 ++++++++++++++++++ .../index/VerifySingleIndexRowTest.java | 29 - .../schema/ConditionalTTLExpressionTest.java | 599 +++++++++++ .../phoenix/schema/TTLExpressionTest.java | 83 ++ .../org/apache/phoenix/util/ScanUtilTest.java | 8 +- .../org/apache/phoenix/util/TestUtil.java | 88 +- 42 files changed, 3510 insertions(+), 774 deletions(-) create mode 100644 phoenix-core-client/src/main/java/org/apache/phoenix/schema/ConditionalTTLExpression.java create mode 100644 phoenix-core-client/src/main/java/org/apache/phoenix/schema/LiteralTTLExpression.java create mode 100644 phoenix-core-client/src/main/java/org/apache/phoenix/schema/TTLExpression.java create mode 100644 phoenix-core/src/it/java/org/apache/phoenix/schema/ConditionalTTLExpressionIT.java create mode 100644 phoenix-core/src/test/java/org/apache/phoenix/schema/ConditionalTTLExpressionTest.java create mode 100644 phoenix-core/src/test/java/org/apache/phoenix/schema/TTLExpressionTest.java diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/TableTTLInfo.java b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/TableTTLInfo.java index fa9e50b3fbf..3a57eae3b4e 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/TableTTLInfo.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/coprocessorclient/TableTTLInfo.java @@ -18,11 +18,14 @@ package org.apache.phoenix.coprocessorclient; -import org.apache.hadoop.hbase.util.Bytes; - import java.nio.charset.StandardCharsets; import java.util.Arrays; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.phoenix.schema.TTLExpression; + +import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; + /** * Simple POJO class to hold TTL info */ @@ -31,18 +34,38 @@ public class TableTTLInfo implements Comparable { private final byte[] tenantId; private final byte[] entityName; private final byte[] matchPattern; - private final int ttl; + private final TTLExpression ttl; + @VisibleForTesting public TableTTLInfo(String physicalTableName, String tenantId, String entityName, String matchPattern, int ttl) { super(); this.physicalTableName = physicalTableName.getBytes(StandardCharsets.UTF_8); this.tenantId = tenantId.getBytes(StandardCharsets.UTF_8); this.entityName = entityName.getBytes(StandardCharsets.UTF_8); this.matchPattern = matchPattern.getBytes(StandardCharsets.UTF_8); - this.ttl = ttl; + this.ttl = TTLExpression.create(ttl); } + @VisibleForTesting public TableTTLInfo(byte[] physicalTableName, byte[] tenantId, byte[] entityName, byte[] matchPattern, int ttl) { + super(); + this.physicalTableName = physicalTableName; + this.tenantId = tenantId; + this.matchPattern = matchPattern; + this.entityName = entityName; + this.ttl = TTLExpression.create(ttl); + } + + public TableTTLInfo(String physicalTableName, String tenantId, String entityName, String matchPattern, TTLExpression ttl) { + super(); + this.physicalTableName = physicalTableName.getBytes(StandardCharsets.UTF_8); + this.tenantId = tenantId.getBytes(StandardCharsets.UTF_8); + this.entityName = entityName.getBytes(StandardCharsets.UTF_8); + this.matchPattern = matchPattern.getBytes(StandardCharsets.UTF_8); + this.ttl = ttl; + } + + public TableTTLInfo(byte[] physicalTableName, byte[] tenantId, byte[] entityName, byte[] matchPattern, TTLExpression ttl) { super(); this.physicalTableName = physicalTableName; this.tenantId = tenantId; @@ -51,7 +74,7 @@ public TableTTLInfo(byte[] physicalTableName, byte[] tenantId, byte[] entityName this.ttl = ttl; } - public int getTTL() { + public TTLExpression getTTL() { return ttl; } public byte[] getTenantId() { 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 eb52fa0fba2..a2559c2f2f0 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 @@ -207,7 +207,13 @@ public SQLException newException(SQLExceptionInfo info) { INVALID_JSON_DATA(540, "42916", "Invalid json data."), JSON_FRAGMENT_NOT_ALLOWED_IN_INDEX_EXPRESSION(541, "42917", "Functions returning JSON fragments are not allowed in Index Expression."), + AGGREGATE_EXPRESSION_NOT_ALLOWED_IN_CONDITIONAL_TTL(542, "42918", + "Aggregate expression not allowed in a conditional TTL expression."), + CANNOT_SET_CONDITIONAL_TTL_ON_TABLE_WITH_MULTIPLE_COLUMN_FAMILIES(543, "42919", + "Cannot set conditional TTL on table with multiple column families."), + CANNOT_DROP_COL_REFERENCED_IN_CONDITIONAL_TTL(544, "42920", + "Cannot drop column referenced in conditional TTL expression."), /** * HBase and Phoenix specific implementation defined sub-classes. * Column family related exceptions. diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core-client/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java index 30bc118da91..60f11b557d3 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java @@ -111,7 +111,6 @@ import org.apache.phoenix.schema.stats.GuidePostsInfo; import org.apache.phoenix.schema.stats.GuidePostsKey; import org.apache.phoenix.schema.stats.StatisticsUtil; -import org.apache.phoenix.schema.types.PVarbinaryEncoded; import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.phoenix.util.ByteUtil; import org.apache.phoenix.util.ClientUtil; @@ -202,6 +201,13 @@ private static void initializeScan(QueryPlan plan, Integer perScanLimit, Integer DEFAULT_WILDCARD_QUERY_DYNAMIC_COLS_ATTRIB); PTable table = tableRef.getTable(); + // If the table has Conditional TTL set, then we need to add all the non PK columns + // referenced in the conditional TTL expression to the scan. This can influence the + // filters that are applied to the scan so do this before the filter analysis. + if (table.hasConditionalTTL()) { + ScanUtil.addConditionalTTLColumnsToScan(scan, context.getConnection(), table); + } + Map> familyMap = scan.getFamilyMap(); // Hack for PHOENIX-2067 to force raw scan over all KeyValues to fix their row keys if (context.getConnection().isDescVarLengthRowKeyUpgrade()) { diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java index ff250f40ba9..8deecd1e401 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java @@ -399,6 +399,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData { public static final byte[] TTL_BYTES = Bytes.toBytes(TTL); public static final int TTL_NOT_DEFINED = 0; public static final int DEFAULT_TTL = HConstants.FOREVER; + public static final String FOREVER_TTL = "FOREVER"; + public static final String NONE_TTL = "NONE"; public static final String PHOENIX_TTL = "PHOENIX_TTL"; public static final byte[] PHOENIX_TTL_BYTES = Bytes.toBytes(PHOENIX_TTL); public static final String PHOENIX_TTL_HWM = "PHOENIX_TTL_HWM"; 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 dba599f25cb..d5f0580745f 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 @@ -270,6 +270,8 @@ import org.apache.phoenix.schema.SequenceAllocation; import org.apache.phoenix.schema.SequenceKey; import org.apache.phoenix.schema.SortOrder; +import org.apache.phoenix.schema.TTLExpression; +import org.apache.phoenix.schema.LiteralTTLExpression; import org.apache.phoenix.schema.TableAlreadyExistsException; import org.apache.phoenix.schema.TableNotFoundException; import org.apache.phoenix.schema.TableProperty; @@ -2999,7 +3001,7 @@ private Map separateAndValidateProperties(PTab boolean willBeTransactional = false; boolean isOrWillBeTransactional = isTransactional; Integer newTTL = null; - Integer newPhoenixTTL = null; + TTLExpression newPhoenixTTL = null; Integer newReplicationScope = null; KeepDeletedCells newKeepDeletedCells = null; TransactionFactory.Provider txProvider = null; @@ -3045,14 +3047,17 @@ private Map separateAndValidateProperties(PTab //If Phoenix level TTL is enabled we are using TTL as phoenix //Table level property. if (!isPhoenixTTLEnabled()) { - newTTL = ((Number) propValue).intValue(); + // only literal TTL expression + LiteralTTLExpression ttlExpr = + (LiteralTTLExpression) TableProperty.TTL.getValue(propValue); + newTTL = ttlExpr != null ? ttlExpr.getTTLValue() : null; //Even though TTL is really a HColumnProperty we treat it //specially. We enforce that all CFs have the same TTL. commonFamilyProps.put(propName, propValue); } else { //Setting this here just to check if we need to throw Exception //for Transaction's SET_TTL Feature. - newPhoenixTTL = ((Number) propValue).intValue(); + newPhoenixTTL = (TTLExpression) TableProperty.TTL.getValue(propValue); } } else if (propName.equals(PhoenixDatabaseMetaData.TRANSACTIONAL) && Boolean.TRUE.equals(propValue)) { willBeTransactional = isOrWillBeTransactional = true; diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/ConditionalTTLExpression.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/ConditionalTTLExpression.java new file mode 100644 index 00000000000..16ebc080cb4 --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/ConditionalTTLExpression.java @@ -0,0 +1,493 @@ +/* + * 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.schema; + +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_TTL; +import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN; +import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS; +import static org.apache.phoenix.schema.PTableType.CDC; +import static org.apache.phoenix.schema.PTableType.VIEW; +import static org.apache.phoenix.util.SchemaUtil.isPKColumn; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.IOException; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellUtil; +import org.apache.hadoop.hbase.io.ImmutableBytesWritable; +import org.apache.hadoop.hbase.util.ByteStringer; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.hadoop.io.WritableUtils; +import org.apache.phoenix.compile.ColumnResolver; +import org.apache.phoenix.compile.FromCompiler; +import org.apache.phoenix.compile.IndexStatementRewriter; +import org.apache.phoenix.compile.StatementContext; +import org.apache.phoenix.compile.WhereCompiler.WhereExpressionCompiler; +import org.apache.phoenix.coprocessor.generated.PTableProtos; +import org.apache.phoenix.coprocessor.generated.ServerCachingProtos; +import org.apache.phoenix.exception.SQLExceptionCode; +import org.apache.phoenix.exception.SQLExceptionInfo; +import org.apache.phoenix.expression.Expression; +import org.apache.phoenix.expression.ExpressionType; +import org.apache.phoenix.hbase.index.covered.update.ColumnReference; +import org.apache.phoenix.jdbc.PhoenixConnection; +import org.apache.phoenix.jdbc.PhoenixStatement; +import org.apache.phoenix.parse.ColumnDef; +import org.apache.phoenix.parse.ColumnName; +import org.apache.phoenix.parse.CreateTableStatement; +import org.apache.phoenix.parse.ParseNode; +import org.apache.phoenix.parse.SQLParser; +import org.apache.phoenix.parse.TableName; +import org.apache.phoenix.query.QueryConstants; +import org.apache.phoenix.schema.tuple.MultiKeyValueTuple; +import org.apache.phoenix.schema.types.PBoolean; +import org.apache.phoenix.thirdparty.com.google.common.collect.Lists; +import org.apache.phoenix.thirdparty.com.google.common.collect.Sets; +import org.apache.phoenix.util.CDCUtil; +import org.apache.phoenix.util.EnvironmentEdgeManager; +import org.apache.phoenix.util.SchemaUtil; +import org.apache.phoenix.util.ViewUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ConditionalTTLExpression extends TTLExpression { + private static final Logger LOGGER = LoggerFactory.getLogger(ConditionalTTLExpression.class); + + // expression as passed in the DDL statement and stored in syscat + private final String ttlExpr; + // compiled expression according to the table schema. For indexes the expression is + // first re-written to use index column references and then compiled. + private Expression compiledExpr; + // columns referenced in the ttl expression to be added to scan + private Set conditionExprColumns; + + public ConditionalTTLExpression(String ttlExpr) { + this.ttlExpr = ttlExpr; + } + + private ConditionalTTLExpression(String ttlExpr, + Expression compiledExpression, + Set conditionExprColumns) { + this.ttlExpr = ttlExpr; + this.compiledExpr = compiledExpression; + this.conditionExprColumns = conditionExprColumns; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ConditionalTTLExpression that = (ConditionalTTLExpression) o; + return ttlExpr.equals(that.ttlExpr); + } + + @Override + public int hashCode() { + return Objects.hash(ttlExpr); + } + + @Override + public String getTTLExpression() { + return ttlExpr; + } + + @Override + public String toString() { + return getTTLExpression(); + } + + /** + * The cells of the row (i.e., result) read from HBase store are lexicographically ordered + * for tables using the key part of the cells which includes row, family, qualifier, + * timestamp and type. The cells belong of a column are ordered from the latest to + * the oldest. The method leverages this ordering and groups the cells into their columns + * based on the pair of family name and column qualifier. + */ + private List getLatestRowVersion(List result) { + List latestRowVersion = new ArrayList<>(); + Cell currentColumnCell = null; + long maxDeleteFamilyTS = 0; + for (Cell cell : result) { + if (currentColumnCell == null || + !CellUtil.matchingColumn(cell, currentColumnCell)) { + // found a new column cell which has the latest timestamp + currentColumnCell = cell; + if (currentColumnCell.getType() == Cell.Type.DeleteFamily || + currentColumnCell.getType() == Cell.Type.DeleteFamilyVersion) { + // DeleteFamily will be first in the lexicographically ordering because + // it has no qualifier + maxDeleteFamilyTS = currentColumnCell.getTimestamp(); + // no need to add the DeleteFamily cell since it can't be part of + // an expression + continue; + } + if (currentColumnCell.getTimestamp() > maxDeleteFamilyTS) { + // only add the cell if it is not masked by the DeleteFamily + latestRowVersion.add(currentColumnCell); + } + } + } + return latestRowVersion; + } + + @Override + /** + * @param result row to be evaluated against the conditional ttl expression + * @return DEFAULT_TTL (FOREVER) if the expression evaluates to False else 0 + * if the expression evaluates to true i.e. row is expired + */ + public long getRowTTLForMasking(List result) { + long ttl = DEFAULT_TTL; + if (compiledExpr == null) { + throw new RuntimeException( + String.format("Conditional TTL Expression %s not compiled", this.ttlExpr)); + } + ImmutableBytesWritable ptr = new ImmutableBytesWritable(); + List latestRowVersion = getLatestRowVersion(result); + if (latestRowVersion.isEmpty()) { + return ttl; + } + MultiKeyValueTuple row = new MultiKeyValueTuple(latestRowVersion); + if (compiledExpr.evaluate(row, ptr)) { + Boolean isExpired = (Boolean) PBoolean.INSTANCE.toObject(ptr); + ttl = isExpired ? 0 : DEFAULT_TTL; + } else { + LOGGER.info("Expression evaluation failed for expr {}", ttlExpr); + } + return ttl; + } + + @Override + /** + * During compaction, we first use the DEFAULT_TTL (FOREVER) value for doing all the checks + * related to ttl and maxlookback. The actual evaluation of the Conditional TTL expression + * happens later. + */ + public long getRowTTLForCompaction(List result) { + return DEFAULT_TTL; + } + + public boolean isExpired(List result) { + return getRowTTLForMasking(result) == 0; + } + + @Override + public synchronized void compileTTLExpression(PhoenixConnection connection, + PTable table) throws SQLException { + + Pair> exprAndCols = buildExpression(connection, table); + compiledExpr = exprAndCols.getFirst(); + conditionExprColumns = exprAndCols.getSecond(); + } + + private Pair> buildExpression( + PhoenixConnection connection, PTable table) throws SQLException { + return buildExpression(connection, table, null); + } + + private Pair> buildExpression( + PhoenixConnection connection, PTable table, PTable parent) throws SQLException { + ParseNode ttlCondition = parseExpression(connection, table, parent); + ColumnResolver resolver = FromCompiler.getResolver(new TableRef(table)); + StatementContext context = new StatementContext( + new PhoenixStatement(connection), resolver); + WhereExpressionCompiler expressionCompiler = new WhereExpressionCompiler(context); + Expression expr = ttlCondition.accept(expressionCompiler); + if (expressionCompiler.isAggregate()) { + // Aggregate functions are not allowed in Conditional TTL expressions because we + // evaluate one row at a time + throw new SQLExceptionInfo.Builder( + SQLExceptionCode.AGGREGATE_EXPRESSION_NOT_ALLOWED_IN_CONDITIONAL_TTL) + .build().buildException(); + } + Set exprCols = + Sets.newHashSetWithExpectedSize(context.getWhereConditionColumns().size()); + for (Pair column : context.getWhereConditionColumns()) { + exprCols.add( + new ColumnReference(column.getFirst(), column.getSecond())); + } + return new Pair<>(expr, exprCols); + } + + // Returns the columns referenced in the ttl expression to be added to scan + public synchronized Set getColumnsReferenced( + PhoenixConnection connection, + PTable table) throws SQLException { + if (conditionExprColumns == null) { + compileTTLExpression(connection, table); + } + return conditionExprColumns; + } + + private static byte[] serializeExpression(Expression condTTLExpr) throws IOException { + try (ByteArrayOutputStream stream = new ByteArrayOutputStream()) { + DataOutput output = new DataOutputStream(stream); + WritableUtils.writeVInt(output, ExpressionType.valueOf(condTTLExpr).ordinal()); + condTTLExpr.write(output); + return stream.toByteArray(); + } + } + + private static Expression deSerializeExpression(byte[] serializedExpr) throws IOException { + try (ByteArrayInputStream stream = new ByteArrayInputStream(serializedExpr)) { + DataInput input = new DataInputStream(stream); + int expressionOrdinal = WritableUtils.readVInt(input); + Expression expression = ExpressionType.values()[expressionOrdinal].newInstance(); + expression.readFields(input); + return expression; + } + } + + public static ConditionalTTLExpression createFromProto(PTableProtos.ConditionTTL condition) + throws IOException { + String ttlExpr = condition.getTtlExpression(); + Expression compiledExpression = deSerializeExpression( + condition.getCompiledExpression().toByteArray()); + List exprColumnsList = + condition.getTtlExpressionColumnsList(); + Set conditionExprColumns = new HashSet<>(exprColumnsList.size()); + for (ServerCachingProtos.ColumnReference colRefFromProto : exprColumnsList) { + conditionExprColumns.add(new ColumnReference( + colRefFromProto.getFamily().toByteArray(), + colRefFromProto.getQualifier().toByteArray())); + } + return new ConditionalTTLExpression(ttlExpr, compiledExpression, conditionExprColumns); + } + + @Override + public PTableProtos.TTLExpression toProto(PhoenixConnection connection, + PTable table) throws SQLException, IOException { + // we want to compile the expression every time we pass it as a scan attribute. This is + // needed so that any stateless expressions like CURRENT_TIME() are always evaluated. + // Otherwise we can cache stale values and keep reusing the stale values which can give + // incorrect results. + compileTTLExpression(connection, table); + PTableProtos.TTLExpression.Builder ttl = PTableProtos.TTLExpression.newBuilder(); + PTableProtos.ConditionTTL.Builder condition = PTableProtos.ConditionTTL.newBuilder(); + condition.setTtlExpression(ttlExpr); + condition.setCompiledExpression(ByteStringer.wrap(serializeExpression(compiledExpr))); + for (ColumnReference colRef : conditionExprColumns) { + ServerCachingProtos.ColumnReference.Builder cRefBuilder = + ServerCachingProtos.ColumnReference.newBuilder(); + cRefBuilder.setFamily(ByteStringer.wrap(colRef.getFamily())); + cRefBuilder.setQualifier(ByteStringer.wrap(colRef.getQualifier())); + condition.addTtlExpressionColumns(cRefBuilder.build()); + } + ttl.setCondition(condition.build()); + return ttl.build(); + } + + private ParseNode parseExpression(PhoenixConnection connection, + PTable table, + PTable parent) throws SQLException { + ParseNode ttlCondition = SQLParser.parseCondition(this.ttlExpr); + return table.getType() != PTableType.INDEX ? ttlCondition + : rewriteForIndex(connection, table, parent, ttlCondition); + } + + // Transform the conditional ttl expression to replace column references with + // corresponding index column references + private ParseNode rewriteForIndex(PhoenixConnection connection, + PTable index, + PTable parent, + ParseNode ttlCondition) throws SQLException { + if (parent == null) { + parent = getParent(connection, index); + } + ColumnResolver parentResolver = FromCompiler.getResolver(new TableRef(parent)); + return IndexStatementRewriter.translate(ttlCondition, parentResolver); + } + + private PTable getParent(PhoenixConnection connection, + PTable table) throws SQLException { + return connection.getTable(table.getParentName().getString()); + } + + @Override + /** + * @param create CreateTableStatement (TABLE | VIEW | INDEX) + * @param parent Parent of VIEW or INDEX, null for base tables + * @param tableProps Create table properties + */ + public void validateTTLOnCreate(PhoenixConnection conn, + CreateTableStatement create, + PTable parent, + Map tableProps) throws SQLException { + // Construct a PTable with just enough information to be able to compile the TTL expression + PTable table = createTempPTable(conn, create, parent, tableProps); + validateTTLExpression(conn, table, parent); + } + + @Override + /** + * @param table TABLE | VIEW referenced in ALTER statement + */ + public void validateTTLOnAlter(PhoenixConnection conn, + PTable table) throws SQLException { + // first validate the expression on the entity being changed + validateTTLExpression(conn, table, null); + + for (PTable index : table.getIndexes()) { + try { + if (CDCUtil.isCDCIndex(index)) { + // CDC index doesn't inherit ConditionTTL expression + continue; + } + // verify that the new expression is covered by all the existing indexes + buildExpression(conn, index, table); + } catch (ColumnNotFoundException | ColumnFamilyNotFoundException e) { + throw new SQLException(String.format( + "Conditional TTL expression %s not covered by index %s", ttlExpr, + index.getTableName()), e); + } + } + } + + /** + * We are still in the middle of executing the CreateTable statement, so we don't have + * the PTable yet, but we need one for compiling the conditional TTL expression so let's + * build the PTable object with just enough information to be able to compile the Conditional + * TTL expression statement. + * @param createStmt + * @param parent + * @param tableProps + * @return PTable object + * @throws SQLException + */ + private PTable createTempPTable(PhoenixConnection conn, + CreateTableStatement createStmt, + PTable parent, + Map tableProps) throws SQLException { + final TableName tableNameNode = createStmt.getTableName(); + final PName schemaName = PNameFactory.newName(tableNameNode.getSchemaName()); + final PName tableName = PNameFactory.newName(tableNameNode.getTableName()); + PName fullName = SchemaUtil.getTableName(schemaName, tableName); + final PName tenantId = conn.getTenantId(); + PTableType tableType = createStmt.getTableType(); + String defaultFamily; + if (parent != null) { + defaultFamily = parent.getDefaultFamilyName() == null ? null : + parent.getDefaultFamilyName().getString(); + } else { + defaultFamily = (String) TableProperty.DEFAULT_COLUMN_FAMILY.getValue(tableProps); + } + List allCols = Lists.newArrayList(); + List pkCols = Lists.newArrayList(); + int pos = 0; + for (ColumnDef colDef : createStmt.getColumnDefs()) { + ColumnName columnDefName = colDef.getColumnDefName(); + String columnName = columnDefName.getColumnName(); + PName familyName = null; + boolean isPK = isPKColumn(createStmt.getPrimaryKeyConstraint(), colDef); + if (!isPK) { // PK columns always have null column family + String family = columnDefName.getFamilyName(); + if (family != null) { + familyName = PNameFactory.newName(family); + } else { + familyName = PNameFactory.newName(defaultFamily == null ? + QueryConstants.DEFAULT_COLUMN_FAMILY : defaultFamily); + } + } + PColumn pColumn = new PColumnImpl(PNameFactory.newName(columnName), familyName, + colDef.getDataType(), colDef.getMaxLength(), colDef.getScale(), + colDef.isNull(), pos++, colDef.getSortOrder(), colDef.getArraySize(), null, + false, colDef.getExpression(), colDef.isRowTimestamp(), false, + Bytes.toBytes(columnName), EnvironmentEdgeManager.currentTimeMillis()); + allCols.add(pColumn); + if (isPK) { + pkCols.add(pColumn); + } + } + + PTable table = new PTableImpl.Builder() + .setName(fullName) + .setKey(new PTableKey(tenantId, fullName.getString())) + .setTenantId(tenantId) + .setSchemaName(schemaName) + .setTableName(tableName) + .setParentSchemaName((parent == null) ? null : parent.getSchemaName()) + .setParentTableName((parent == null) ? null : parent.getTableName()) + .setPhysicalNames(Collections.EMPTY_LIST) + .setType(tableType) + .setImmutableStorageScheme(ONE_CELL_PER_COLUMN) + .setQualifierEncodingScheme(NON_ENCODED_QUALIFIERS) + .setDefaultFamilyName(PNameFactory.newName(defaultFamily)) + .setColumns(allCols) + .setPkColumns(pkCols) + .setIndexes(Collections.EMPTY_LIST) + .build(); + + if (parent != null) { + // add derived columns for views + if (table.getType() == VIEW) { + table = ViewUtil.addDerivedColumnsFromParent(conn, table, parent); + } + } + return table; + } + + private void validateTTLExpression(PhoenixConnection conn, + PTable table, + PTable parent) throws SQLException { + + if (table.getType() == CDC) { // no need to validate for CDC type tables + return; + } + + // Conditional TTL is only supported on table with 1 column family + if (table.getColumnFamilies().size() > 1) { + throw new SQLExceptionInfo.Builder( + SQLExceptionCode.CANNOT_SET_CONDITIONAL_TTL_ON_TABLE_WITH_MULTIPLE_COLUMN_FAMILIES) + .build().buildException(); + } + + try { + // verify that all the columns referenced in TTL expression are resolvable + Pair> exprAndCols = + buildExpression(conn, table, parent); + Expression ttlExpression = exprAndCols.getFirst(); + // Conditional TTL expression should evaluate to a boolean value + if (ttlExpression.getDataType() != PBoolean.INSTANCE) { + throw TypeMismatchException.newException(PBoolean.INSTANCE, + ttlExpression.getDataType(), ttlExpression.toString()); + } + } catch (ColumnNotFoundException | ColumnFamilyNotFoundException e) { + throw new SQLException(String.format( + "Conditional TTL expression %s refers columns not in %s", ttlExpr, + table.getTableName()), e); + } + } +} 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 e67e876dcd2..8f1ae8d62c2 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 @@ -381,10 +381,15 @@ public Boolean useStatsForParallelization() { return delegate.hasViewModifiedUseStatsForParallelization(); } - @Override public int getTTL() { + @Override public TTLExpression getTTL() { return delegate.getTTL(); } + @Override + public boolean hasConditionalTTL() { + return delegate.hasConditionalTTL(); + } + @Override public Long getLastDDLTimestamp() { return delegate.getLastDDLTimestamp(); diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/LiteralTTLExpression.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/LiteralTTLExpression.java new file mode 100644 index 00000000000..92209129bb2 --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/LiteralTTLExpression.java @@ -0,0 +1,107 @@ +/* + * 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.schema; + +import java.sql.SQLException; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import org.apache.hadoop.hbase.Cell; +import org.apache.phoenix.coprocessor.generated.PTableProtos; +import org.apache.phoenix.jdbc.PhoenixConnection; +import org.apache.phoenix.parse.CreateTableStatement; +import org.apache.phoenix.thirdparty.com.google.common.base.Preconditions; + +public class LiteralTTLExpression extends TTLExpression { + private final int ttlValue; + + public LiteralTTLExpression(int ttl) { + Preconditions.checkArgument(ttl >= 0); + this.ttlValue = ttl; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + LiteralTTLExpression that = (LiteralTTLExpression) o; + return ttlValue == that.ttlValue; + } + + @Override + public int hashCode() { + return Objects.hash(ttlValue); + } + + @Override + public String getTTLExpression() { + return String.valueOf(ttlValue); + } + + @Override + public String toString() { + return getTTLExpression(); + } + + @Override + public long getRowTTLForMasking(List result) { + return getRowTTLForCompaction(result); + } + + @Override + public long getRowTTLForCompaction(List result) { + return ttlValue; + } + + @Override + public void validateTTLOnCreate(PhoenixConnection conn, + CreateTableStatement create, + PTable parent, + Map tableProps) { + + } + + @Override + public void validateTTLOnAlter(PhoenixConnection connection, PTable table) {} + + @Override + public void compileTTLExpression(PhoenixConnection connection, PTable table) { + } + + public static LiteralTTLExpression createFromProto(PTableProtos.LiteralTTL literal) { + return new LiteralTTLExpression(literal.getTtlValue()); + } + + @Override + public PTableProtos.TTLExpression toProto(PhoenixConnection connection, + PTable table) throws SQLException { + if (this.equals(TTLExpression.TTL_EXPRESSION_NOT_DEFINED)) { + return null; + } + PTableProtos.TTLExpression.Builder ttl = PTableProtos.TTLExpression.newBuilder(); + PTableProtos.LiteralTTL.Builder literal = PTableProtos.LiteralTTL.newBuilder(); + literal.setTtlValue(ttlValue); + ttl.setLiteral(literal.build()); + return ttl.build(); + } + + public int getTTLValue() { + return ttlValue; + } +} 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 bc2cf3a0efa..86baae83fce 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 @@ -17,6 +17,7 @@ */ package org.apache.phoenix.schema; +import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_SET_CONDITIONAL_TTL_ON_TABLE_WITH_MULTIPLE_COLUMN_FAMILIES; import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_TRANSFORM_TRANSACTIONAL_TABLE; import static org.apache.phoenix.exception.SQLExceptionCode.CDC_ALREADY_ENABLED; import static org.apache.phoenix.exception.SQLExceptionCode.ERROR_WRITING_TO_SCHEMA_REGISTRY; @@ -24,7 +25,6 @@ import static org.apache.phoenix.exception.SQLExceptionCode.TABLE_ALREADY_EXIST; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CDC_INCLUDE_TABLE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CDC_STREAM_STATUS_NAME; -import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TTL_NOT_DEFINED; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STREAMING_TOPIC_NAME; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_TASK_TABLE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TTL; @@ -32,6 +32,8 @@ import static org.apache.phoenix.query.QueryConstants.SYSTEM_SCHEMA_NAME; import static org.apache.phoenix.query.QueryServices.INDEX_CREATE_DEFAULT_STATE; import static org.apache.phoenix.schema.PTableType.CDC; +import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_FORVER; +import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_NOT_DEFINED; import static org.apache.phoenix.thirdparty.com.google.common.collect.Sets.newLinkedHashSet; import static org.apache.phoenix.thirdparty.com.google.common.collect.Sets.newLinkedHashSetWithExpectedSize; import static org.apache.phoenix.exception.SQLExceptionCode.INSUFFICIENT_MULTI_TENANT_COLUMNS; @@ -2420,13 +2422,18 @@ private void populateFamilyPropsList(Map familyNames, Map 0", - tableName)) - .build() - .buildException(); - } if (!isViewTTLEnabled() && tableType == VIEW) { throw new SQLExceptionInfo.Builder(SQLExceptionCode. - VIEW_TTL_NOT_ENABLED) - .setSchemaName(schemaName) - .setTableName(tableName) - .build() - .buildException(); + VIEW_TTL_NOT_ENABLED) + .setSchemaName(schemaName) + .setTableName(tableName) + .build() + .buildException(); } if (tableType != TABLE && (tableType != VIEW || viewType != UPDATABLE)) { throw new SQLExceptionInfo.Builder(SQLExceptionCode. - TTL_SUPPORTED_FOR_TABLES_AND_VIEWS_ONLY) + TTL_SUPPORTED_FOR_TABLES_AND_VIEWS_ONLY) + .setSchemaName(schemaName) + .setTableName(tableName) + .build() + .buildException(); + } + ttlFromHierarchy = checkAndGetTTLFromHierarchy(parent, tableName); + if (ttlFromHierarchy != TTL_EXPRESSION_NOT_DEFINED) { + throw new SQLExceptionInfo.Builder(SQLExceptionCode. + TTL_ALREADY_DEFINED_IN_HIERARCHY) .setSchemaName(schemaName) .setTableName(tableName) .build() .buildException(); } - ttlFromHierarchy = checkAndGetTTLFromHierarchy(parent); - if (ttlFromHierarchy != TTL_NOT_DEFINED) { - throw new SQLExceptionInfo.Builder(SQLExceptionCode. - TTL_ALREADY_DEFINED_IN_HIERARCHY) + try { + ttlProp.validateTTLOnCreate(connection, statement, parent, tableProps); + } catch (IllegalArgumentException e) { + throw new SQLExceptionInfo.Builder(SQLExceptionCode.ILLEGAL_DATA) + .setMessage(e.getMessage()) .setSchemaName(schemaName) .setTableName(tableName) .build() .buildException(); } - ttl = ttlProp; } else { - ttlFromHierarchy = checkAndGetTTLFromHierarchy(parent); + ttlFromHierarchy = checkAndGetTTLFromHierarchy(parent, tableName); + if (ttlFromHierarchy != TTL_EXPRESSION_NOT_DEFINED) { + ttlFromHierarchy.validateTTLOnCreate(connection, + statement, + parent, + tableProps); + } } Boolean isChangeDetectionEnabledProp = @@ -3166,7 +3181,7 @@ else if (!SchemaUtil.isSystemTable(Bytes.toBytes(SchemaUtil.getTableName(schemaN } else { // do not allow setting NOT-NULL constraint on non-primary columns. if ( !colDef.isNull() && !isImmutableRows && - ( wasPKDefined || !isPkColumn(pkConstraint, colDef))) { + ( wasPKDefined || !SchemaUtil.isPKColumn(pkConstraint, colDef))) { throw new SQLExceptionInfo.Builder(SQLExceptionCode.KEY_VALUE_NOT_NULL) .setSchemaName(schemaName) .setTableName(tableName) @@ -3175,7 +3190,7 @@ else if (!SchemaUtil.isSystemTable(Bytes.toBytes(SchemaUtil.getTableName(schemaN } ColumnName columnDefName = colDef.getColumnDefName(); String colDefFamily = columnDefName.getFamilyName(); - boolean isPkColumn = isPkColumn(pkConstraint, colDef); + boolean isPkColumn = SchemaUtil.isPKColumn(pkConstraint, colDef); String cqCounterFamily = null; if (!isPkColumn) { if (immutableStorageScheme == SINGLE_CELL_ARRAY_WITH_OFFSETS && encodingScheme != NON_ENCODED_QUALIFIERS) { @@ -3380,7 +3395,7 @@ else if (!SchemaUtil.isSystemTable(Bytes.toBytes(SchemaUtil.getTableName(schemaN .setIndexWhere(statement.getWhereClause() == null ? null : statement.getWhereClause().toString()) .setRowKeyMatcher(rowKeyMatcher) - .setTTL(TTL_NOT_DEFINED) + .setTTL(TTL_EXPRESSION_NOT_DEFINED) .build(); connection.addTable(table, MetaDataProtocol.MIN_TABLE_TIMESTAMP); } @@ -3647,10 +3662,10 @@ public boolean isViewReferenced() { tableUpsert.setString(36, cdcIncludeScopesStr); } - if (ttl == null || ttl == TTL_NOT_DEFINED) { + if (ttl == null || ttl == TTL_EXPRESSION_NOT_DEFINED) { tableUpsert.setNull(37, Types.VARCHAR); } else { - tableUpsert.setString(37, String.valueOf(ttl)); + tableUpsert.setString(37, ttl.getTTLExpression()); } if ((rowKeyMatcher == null) || @@ -3808,7 +3823,7 @@ public boolean isViewReferenced() { : statement.getWhereClause().toString()) .setMaxLookbackAge(maxLookbackAge) .setCDCIncludeScopes(cdcIncludeScopes) - .setTTL(ttl == null || ttl == TTL_NOT_DEFINED ? ttlFromHierarchy : ttl) + .setTTL(ttl == null || ttl == TTL_EXPRESSION_NOT_DEFINED ? ttlFromHierarchy : ttl) .setRowKeyMatcher(rowKeyMatcher) .build(); result = new MetaDataMutationResult(code, result.getMutationTime(), table, true); @@ -3952,10 +3967,6 @@ private void throwsSQLExceptionUtil(String code,String schemaName, String tableN .setSchemaName(schemaName).setTableName(tableName).build().buildException(); } - private static boolean isPkColumn(PrimaryKeyConstraint pkConstraint, ColumnDef colDef) { - return colDef.isPK() || (pkConstraint != null && pkConstraint.contains(colDef.getColumnDefName())); - } - /** * A table can be a parent table to tenant-specific tables if all of the following conditions are true: *

@@ -4315,7 +4326,7 @@ private long incrementTableSeqNum(PTable table, PTableType expectedType, int col Long updateCacheFrequency, Boolean isImmutableRows, Boolean disableWAL, Boolean isMultiTenant, Boolean storeNulls, Long guidePostWidth, Boolean appendOnlySchema, ImmutableStorageScheme immutableStorageScheme, Boolean useStatsForParallelization, - Integer ttl, Boolean isChangeDetectionEnabled, String physicalTableName, String schemaVersion, + TTLExpression ttl, Boolean isChangeDetectionEnabled, String physicalTableName, String schemaVersion, QualifierEncodingScheme columnEncodedBytes, String streamingTopicName, Long maxLookbackAge) throws SQLException { String schemaName = table.getSchemaName().getString(); @@ -4374,7 +4385,7 @@ private long incrementTableSeqNum(PTable table, PTableType expectedType, int col } if (ttl != null) { mutateStringProperty(connection, tenantId, schemaName, tableName, TTL, - ttl == TTL_NOT_DEFINED ? null : String.valueOf(ttl)); + ttl == TTL_EXPRESSION_NOT_DEFINED ? null : ttl.getTTLExpression()); } if (isChangeDetectionEnabled != null) { mutateBooleanProperty(connection, tenantId, schemaName, tableName, CHANGE_DETECTION_ENABLED, isChangeDetectionEnabled); @@ -4551,7 +4562,6 @@ public MutationState addColumn(PTable table, List origColumnDefs, boolean acquiredBaseTableMutex = false; try { connection.setAutoCommit(false); - List columnDefs; if ((table.isAppendOnlySchema() || ifNotExists) && origColumnDefs != null) { // only make the rpc if we are adding new columns @@ -4630,12 +4640,13 @@ public MutationState addColumn(PTable table, List origColumnDefs, if (areWeIntroducingTTLAtThisLevel.booleanValue()) { //As we are introducing TTL for the first time at this level, we need to check //if TTL is already defined up or down in the hierarchy. - Integer ttlAlreadyDefined = TTL_NOT_DEFINED; + TTLExpression ttlAlreadyDefined = TTL_EXPRESSION_NOT_DEFINED; //Check up the hierarchy if (table.getType() != PTableType.TABLE) { - ttlAlreadyDefined = checkAndGetTTLFromHierarchy(PhoenixRuntime.getTableNoCache(connection, table.getParentName().toString())); + ttlAlreadyDefined = checkAndGetTTLFromHierarchy(PhoenixRuntime.getTableNoCache( + connection, table.getParentName().toString()), tableName); } - if (ttlAlreadyDefined != TTL_NOT_DEFINED) { + if (ttlAlreadyDefined != TTL_EXPRESSION_NOT_DEFINED) { throw new SQLExceptionInfo.Builder(SQLExceptionCode. TTL_ALREADY_DEFINED_IN_HIERARCHY) .setSchemaName(schemaName) @@ -4653,9 +4664,6 @@ public MutationState addColumn(PTable table, List origColumnDefs, * byte[], byte[], List, int)} we are already traversing through * allDescendantViews. */ - - - } boolean isTransformNeeded = TransformClient.checkIsTransformNeeded(metaProperties, schemaName, table, tableName, null, tenantIdToUse, connection); @@ -4710,6 +4718,23 @@ public MutationState addColumn(PTable table, List origColumnDefs, if (!colDef.validateDefault(context, null)) { colDef = new ColumnDef(colDef, null); // Remove DEFAULT as it's not necessary } + if (!colDef.isPK() && table.hasConditionalTTL()) { + // Only 1 column family is allowed if the table has conditional TTL + PColumnFamily family = table.getColumnFamilies().get(0); + String tableFamilyName = family.getName().getString(); + String colFamilyName = colDef.getColumnDefName().getFamilyName(); + if (colFamilyName == null) { + colFamilyName = table.getDefaultFamilyName() == null ? + DEFAULT_COLUMN_FAMILY : + table.getDefaultFamilyName().getString(); + } + if (!colFamilyName.equals(tableFamilyName)) { + throw new SQLExceptionInfo.Builder( + CANNOT_SET_CONDITIONAL_TTL_ON_TABLE_WITH_MULTIPLE_COLUMN_FAMILIES) + .setMessage(String.format("Cannot add column %s", colDef)) + .build().buildException(); + } + } String familyName = null; Integer encodedCQ = null; if (!colDef.isPK()) { @@ -5287,6 +5312,20 @@ else if (table.isAppendOnlySchema()) { else if (columnToDrop.isViewReferenced()) { throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_VIEW_REFERENCED_COL) .setColumnName(columnToDrop.getName().getString()).build().buildException(); + } else if (table.hasConditionalTTL()) { + ConditionalTTLExpression ttlExpr = (ConditionalTTLExpression) table.getTTL(); + Set colsReferencedInTTLExpr = + ttlExpr.getColumnsReferenced(connection, table); + ColumnReference colDropRef = new ColumnReference( + columnToDrop.getFamilyName() == null ? + null : columnToDrop.getFamilyName().getBytes(), + columnToDrop.getColumnQualifierBytes()); + if (colsReferencedInTTLExpr.contains(colDropRef)) { + throw new SQLExceptionInfo.Builder( + SQLExceptionCode.CANNOT_DROP_COL_REFERENCED_IN_CONDITIONAL_TTL) + .setColumnName(columnToDrop.getName().getString()) + .build().buildException(); + } } columnsToDrop.add(new ColumnRef(columnRef.getTableRef(), columnToDrop.getPosition())); // check if client is already holding a mutex from previous retry @@ -5961,7 +6000,7 @@ private MetaProperties loadStmtProperties(ListMultimap getDefaultPropertyValues(); /** - * @return The TTL duration associated with the entity when Phoenix level TTL is enabled. + * @return The TTL expression associated with the entity when Phoenix level TTL is enabled. + * The expression can be a Literal value or a boolean Condition. */ - int getTTL(); + TTLExpression getTTL(); + + /** + * @return Returns true if table has conditional TTL set + */ + boolean hasConditionalTTL(); /** * @return the last timestamp at which this entity had its data shape created or modified (e 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 8caa0f72e63..7d9f72bfaa7 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 @@ -32,7 +32,6 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MAX_LOOKBACK_AGE; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MULTI_TENANT; -import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TTL_NOT_DEFINED; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PHYSICAL_TABLE_NAME; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTIONAL; @@ -49,6 +48,7 @@ import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_UPDATE_CACHE_FREQUENCY; import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_USE_STATS_FOR_PARALLELIZATION; import static org.apache.phoenix.schema.SaltingUtil.SALTING_COLUMN; +import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_NOT_DEFINED; import static org.apache.phoenix.schema.TableProperty.DEFAULT_COLUMN_FAMILY; import static org.apache.phoenix.schema.types.PDataType.TRUE_BYTES; @@ -122,7 +122,6 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -130,72 +129,9 @@ import java.util.Set; import javax.annotation.Nonnull; -import javax.annotation.Nonnull; - -import org.apache.phoenix.schema.types.PVarbinary; -import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.hadoop.hbase.Cell; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.Durability; -import org.apache.hadoop.hbase.client.Mutation; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.hbase.util.ByteStringer; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Pair; -import org.apache.phoenix.compile.ExpressionCompiler; -import org.apache.phoenix.compile.FromCompiler; -import org.apache.phoenix.compile.QueryPlan; -import org.apache.phoenix.compile.StatementContext; -import org.apache.phoenix.coprocessor.generated.DynamicColumnMetaDataProtos; -import org.apache.phoenix.coprocessor.generated.PTableProtos; -import org.apache.phoenix.exception.DataExceedsCapacityException; -import org.apache.phoenix.expression.Expression; -import org.apache.phoenix.expression.LiteralExpression; -import org.apache.phoenix.expression.SingleCellConstructorExpression; -import org.apache.phoenix.hbase.index.covered.update.ColumnReference; -import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr; -import org.apache.phoenix.hbase.index.util.KeyValueBuilder; -import org.apache.phoenix.jdbc.PhoenixConnection; -import org.apache.phoenix.jdbc.PhoenixPreparedStatement; -import org.apache.phoenix.jdbc.PhoenixStatement; -import org.apache.phoenix.index.IndexMaintainer; -import org.apache.phoenix.parse.ParseNode; -import org.apache.phoenix.parse.SQLParser; -import org.apache.phoenix.protobuf.ProtobufUtil; -import org.apache.phoenix.query.QueryConstants; -import org.apache.phoenix.schema.RowKeySchema.RowKeySchemaBuilder; -import org.apache.phoenix.schema.transform.TransformMaintainer; -import org.apache.phoenix.schema.types.PBinary; -import org.apache.phoenix.schema.types.PChar; -import org.apache.phoenix.schema.types.PDataType; -import org.apache.phoenix.schema.types.PDouble; -import org.apache.phoenix.schema.types.PFloat; import org.apache.phoenix.schema.types.PVarbinaryEncoded; -import org.apache.phoenix.schema.types.PVarchar; -import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; -import org.apache.phoenix.thirdparty.com.google.common.base.Objects; -import org.apache.phoenix.thirdparty.com.google.common.base.Preconditions; -import org.apache.phoenix.thirdparty.com.google.common.base.Strings; -import org.apache.phoenix.thirdparty.com.google.common.collect.ArrayListMultimap; -import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableList; -import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableMap; -import org.apache.phoenix.thirdparty.com.google.common.collect.ImmutableSortedMap; -import org.apache.phoenix.thirdparty.com.google.common.collect.ListMultimap; -import org.apache.phoenix.thirdparty.com.google.common.collect.Lists; -import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; -import org.apache.phoenix.thirdparty.com.google.common.collect.Sets; -import org.apache.phoenix.transaction.TransactionFactory; -import org.apache.phoenix.util.ByteUtil; import org.apache.phoenix.util.CDCUtil; -import org.apache.phoenix.util.EncodedColumnsUtil; -import org.apache.phoenix.util.MetaDataUtil; -import org.apache.phoenix.util.PhoenixRuntime; -import org.apache.phoenix.util.SchemaUtil; -import org.apache.phoenix.util.SizedUtil; -import org.apache.phoenix.util.TrustedByteArrayOutputStream; /** * @@ -271,7 +207,7 @@ public class PTableImpl implements PTable { private final QualifierEncodingScheme qualifierEncodingScheme; private final EncodedCQCounter encodedCQCounter; private final Boolean useStatsForParallelization; - private final int ttl; + private final TTLExpression ttl; private final BitSet viewModifiedPropSet; private final Long lastDDLTimestamp; private final boolean isChangeDetectionEnabled; @@ -353,7 +289,7 @@ public static class Builder { private String indexWhere; private Long maxLookbackAge; private Map ancestorLastDDLTimestampMap = new HashMap<>(); - private int ttl; + private TTLExpression ttl = TTL_EXPRESSION_NOT_DEFINED; private byte[] rowKeyMatcher; // Used to denote which properties a view has explicitly modified @@ -689,8 +625,10 @@ public Builder setViewModifiedUseStatsForParallelization(boolean modified) { return this; } - public Builder setTTL(int ttl) { - propertyValues.put(TTL, String.valueOf(ttl)); + public Builder setTTL(TTLExpression ttl) { + if (ttl != null) { + propertyValues.put(TTL, ttl.getTTLExpression()); + } this.ttl = ttl; return this; } @@ -2140,10 +2078,10 @@ public static PTable createFromProto(PTableProtos.PTable table) { cdcIncludeScopesStr = table.getCDCIncludeScopes(); } - Integer ttl = TTL_NOT_DEFINED; + TTLExpression ttl = TTL_EXPRESSION_NOT_DEFINED; if (table.hasTtl()) { - String ttlStr = (String) PVarchar.INSTANCE.toObject(table.getTtl().toByteArray()); - ttl = Integer.parseInt(ttlStr); + String ttlExpr = (String) PVarchar.INSTANCE.toObject(table.getTtl().toByteArray()); + ttl = TTLExpression.create(ttlExpr); } byte[] rowKeyMatcher = null; @@ -2357,9 +2295,10 @@ public static PTableProtos.PTable toProto(PTable table) { builder.setCDCIncludeScopes(CDCUtil.makeChangeScopeStringFromEnums( table.getCDCIncludeScopes() != null ? table.getCDCIncludeScopes() : Collections.EMPTY_SET)); - - builder.setTtl(ByteStringer.wrap(PVarchar.INSTANCE.toBytes(String.valueOf(table.getTTL())))); - + if (table.getTTL() != null) { + builder.setTtl(ByteStringer.wrap(PVarchar.INSTANCE.toBytes( + table.getTTL().getTTLExpression()))); + } if (table.getRowKeyMatcher() != null) { builder.setRowKeyMatcher(ByteStringer.wrap(table.getRowKeyMatcher())); } @@ -2459,10 +2398,15 @@ public Boolean useStatsForParallelization() { } @Override - public int getTTL() { + public TTLExpression getTTL() { return ttl; } + @Override + public boolean hasConditionalTTL() { + return ttl instanceof ConditionalTTLExpression; + } + @Override public boolean hasViewModifiedUpdateCacheFrequency() { return viewModifiedPropSet.get(VIEW_MODIFIED_UPDATE_CACHE_FREQUENCY_BIT_SET_POS); } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TTLExpression.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TTLExpression.java new file mode 100644 index 00000000000..7512a9d2b17 --- /dev/null +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TTLExpression.java @@ -0,0 +1,128 @@ +/* + * 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.schema; + +import java.io.IOException; +import java.sql.SQLException; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.HConstants; +import org.apache.phoenix.coprocessor.generated.PTableProtos; +import org.apache.phoenix.jdbc.PhoenixConnection; +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; +import org.apache.phoenix.parse.CreateTableStatement; + +public abstract class TTLExpression { + + public static final TTLExpression TTL_EXPRESSION_FORVER = + new LiteralTTLExpression(HConstants.FOREVER); + public static final TTLExpression TTL_EXPRESSION_NOT_DEFINED = + new LiteralTTLExpression(PhoenixDatabaseMetaData.TTL_NOT_DEFINED); + + public static TTLExpression create(String ttlExpr) { + if (PhoenixDatabaseMetaData.NONE_TTL.equalsIgnoreCase(ttlExpr)) { + return TTL_EXPRESSION_NOT_DEFINED; + } else if (PhoenixDatabaseMetaData.FOREVER_TTL.equalsIgnoreCase(ttlExpr)) { + return TTL_EXPRESSION_FORVER; + } else { + try { + int ttlValue = Integer.parseInt(ttlExpr); + return create(ttlValue); + } catch (NumberFormatException e) { + return new ConditionalTTLExpression(ttlExpr); + } + } + } + + public static TTLExpression create (int ttlValue) { + if (ttlValue == PhoenixDatabaseMetaData.TTL_NOT_DEFINED) { + return TTL_EXPRESSION_NOT_DEFINED; + } else if (ttlValue == HConstants.FOREVER) { + return TTL_EXPRESSION_FORVER; + } else { + return new LiteralTTLExpression(ttlValue); + } + } + + public static TTLExpression create(byte[] phoenixTTL) throws IOException { + return createFromProto(PTableProtos.TTLExpression.parseFrom(phoenixTTL)); + } + + public static TTLExpression createFromProto( + PTableProtos.TTLExpression ttlExpressionProto) throws IOException { + if (ttlExpressionProto.hasLiteral()) { + return LiteralTTLExpression.createFromProto(ttlExpressionProto.getLiteral()); + } + if (ttlExpressionProto.hasCondition()) { + return ConditionalTTLExpression.createFromProto(ttlExpressionProto.getCondition()); + } + throw new RuntimeException("Unxexpected! Shouldn't reach here"); + } + + /** + * Serialize the TTL expression as a protobuf byte[] + * @param connection + * @param table + * @return protobuf for the TTL expression + * @throws SQLException + */ + public byte[] getTTLForScanAttribute(PhoenixConnection connection, + PTable table) throws SQLException { + try { + PTableProtos.TTLExpression proto = toProto(connection, table); + return proto != null ? proto.toByteArray() : null; + } catch (IOException e) { + throw new SQLException( + String.format("Error serializing %s as scan attribute", this), e); + } + } + + abstract public String getTTLExpression(); + + /** + * Returns the TTL value used for masking in TTLRegionScanner + * @param result Input row + * @return ttl value in seconds + */ + abstract public long getRowTTLForMasking(List result); + + /** + * Returns the TTL value used during compaction in CompactionScanner + * @param result Input row + * @return ttl value in seconds + */ + abstract public long getRowTTLForCompaction(List result); + + abstract public String toString(); + + abstract public void validateTTLOnCreate(PhoenixConnection conn, + CreateTableStatement create, + PTable parent, + Map tableProps) throws SQLException; + + abstract public void validateTTLOnAlter(PhoenixConnection connection, + PTable table) throws SQLException; + + abstract public void compileTTLExpression(PhoenixConnection connection, + PTable table) throws SQLException; + + abstract public PTableProtos.TTLExpression toProto( + PhoenixConnection connection, PTable table) throws SQLException, IOException; +} diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TableProperty.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TableProperty.java index 7f42c421246..b182de70ff6 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TableProperty.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TableProperty.java @@ -254,23 +254,19 @@ public Object getPTableValue(PTable table) { * special values :- * NONE or 0L => Not Defined. * FOREVER => HConstants.LATEST_TIMESTAMP - * + * Value can also be a boolean condition * @param value * @return */ @Override public Object getValue(Object value) { if (value instanceof String) { - String strValue = (String) value; - if ("FOREVER".equalsIgnoreCase(strValue)) { - return HConstants.FOREVER; - } else if ("NONE".equalsIgnoreCase(strValue)) { - return TTL_NOT_DEFINED; - } + return TTLExpression.create((String)value); } else if (value != null) { //Not converting to milli-seconds for better understanding at compaction and masking //stage. As HBase Descriptor level gives this value in seconds. - return ((Number) value).intValue(); + int ttlValue = ((Number) value).intValue(); + return TTLExpression.create(ttlValue); } return value; } diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/util/ScanUtil.java b/phoenix-core-client/src/main/java/org/apache/phoenix/util/ScanUtil.java index 7a97c6a7ed0..3d44c015906 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/util/ScanUtil.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/util/ScanUtil.java @@ -40,6 +40,7 @@ import java.util.List; import java.util.Map; import java.util.NavigableSet; +import java.util.Set; import java.util.TreeMap; import org.apache.hadoop.conf.Configuration; @@ -80,6 +81,7 @@ import org.apache.phoenix.filter.MultiEncodedCQKeyValueComparisonFilter; import org.apache.phoenix.filter.PagingFilter; import org.apache.phoenix.filter.SkipScanFilter; +import org.apache.phoenix.hbase.index.covered.update.ColumnReference; import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr; import org.apache.phoenix.hbase.index.util.VersionUtil; import org.apache.phoenix.index.CDCTableInfo; @@ -91,6 +93,7 @@ import org.apache.phoenix.query.QueryConstants; import org.apache.phoenix.query.QueryServices; import org.apache.phoenix.query.QueryServicesOptions; +import org.apache.phoenix.schema.ConditionalTTLExpression; import org.apache.phoenix.schema.IllegalDataException; import org.apache.phoenix.schema.PColumn; import org.apache.phoenix.schema.PName; @@ -101,6 +104,8 @@ import org.apache.phoenix.schema.PTableType; import org.apache.phoenix.schema.RowKeySchema; import org.apache.phoenix.schema.SortOrder; +import org.apache.phoenix.schema.TTLExpression; +import org.apache.phoenix.schema.LiteralTTLExpression; import org.apache.phoenix.schema.TableNotFoundException; import org.apache.phoenix.schema.ValueSchema.Field; import org.apache.phoenix.schema.transform.SystemTransformRecord; @@ -109,7 +114,6 @@ import org.apache.phoenix.schema.tuple.ResultTuple; import org.apache.phoenix.schema.tuple.Tuple; import org.apache.phoenix.schema.types.PDataType; -import org.apache.phoenix.schema.types.PInteger; import org.apache.phoenix.schema.types.PVarbinary; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -1147,12 +1151,25 @@ public static boolean getStatsForParallelizationProp(PhoenixConnection conn, PTa } - public static int getTTL(Scan scan) { + public static int getTTL(Scan scan) throws IOException { byte[] phoenixTTL = scan.getAttribute(BaseScannerRegionObserverConstants.TTL); if (phoenixTTL == null) { return DEFAULT_TTL; } - return Bytes.readAsInt(phoenixTTL, 0, phoenixTTL.length); + TTLExpression ttlExpression = TTLExpression.create(phoenixTTL); + if (ttlExpression instanceof LiteralTTLExpression) { + LiteralTTLExpression literal = (LiteralTTLExpression)ttlExpression; + return literal.getTTLValue(); + } + return DEFAULT_TTL; + } + + public static TTLExpression getTTLExpression(Scan scan) throws IOException { + byte[] phoenixTTL = scan.getAttribute(BaseScannerRegionObserverConstants.TTL); + if (phoenixTTL == null) { + return TTLExpression.TTL_EXPRESSION_FORVER; + } + return TTLExpression.create(phoenixTTL); } public static boolean isPhoenixTableTTLEnabled(Configuration conf) { @@ -1193,7 +1210,7 @@ public static long getMaxTimestamp(List cellList) { return maxTs; } - public static boolean isTTLExpired(Cell cell, Scan scan, long nowTS) { + public static boolean isTTLExpired(Cell cell, Scan scan, long nowTS) throws IOException { long ts = cell.getTimestamp(); int ttl = ScanUtil.getTTL(scan); return ts + ttl < nowTS; @@ -1424,7 +1441,9 @@ public static void setScanAttributesForPhoenixTTL(Scan scan, PTable table, return; } } - if (dataTable.getTTL() != 0) { + TTLExpression ttlExpr = table.getTTL(); + byte[] ttlForScan = ttlExpr.getTTLForScanAttribute(phoenixConnection, table); + if (ttlForScan != null) { byte[] emptyColumnFamilyName = SchemaUtil.getEmptyColumnFamily(table); byte[] emptyColumnName = table.getEncodingScheme() == PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS ? @@ -1434,8 +1453,7 @@ public static void setScanAttributesForPhoenixTTL(Scan scan, PTable table, Bytes.toBytes(tableName)); scan.setAttribute(BaseScannerRegionObserverConstants.EMPTY_COLUMN_FAMILY_NAME, emptyColumnFamilyName); scan.setAttribute(BaseScannerRegionObserverConstants.EMPTY_COLUMN_QUALIFIER_NAME, emptyColumnName); - scan.setAttribute(BaseScannerRegionObserverConstants.TTL, - Bytes.toBytes(Integer.valueOf(dataTable.getTTL()))); + scan.setAttribute(BaseScannerRegionObserverConstants.TTL, ttlForScan); if (!ScanUtil.isDeleteTTLExpiredRows(scan)) { scan.setAttribute(BaseScannerRegionObserverConstants.MASK_PHOENIX_TTL_EXPIRED, PDataType.TRUE_BYTES); } @@ -1451,6 +1469,35 @@ public static void setScanAttributesForPhoenixTTL(Scan scan, PTable table, } } + public static void addConditionalTTLColumnsToScan(Scan scan, + PhoenixConnection connection, + PTable table) throws SQLException { + //If entity is a view and phoenix.view.ttl.enabled is false then don't + // set TTL scan attribute. + if ((table.getType() == PTableType.VIEW) && + !connection.getQueryServices().getConfiguration().getBoolean( + QueryServices.PHOENIX_VIEW_TTL_ENABLED, + QueryServicesOptions.DEFAULT_PHOENIX_VIEW_TTL_ENABLED)) { + return; + } + + // If Phoenix level TTL is not enabled OR is a system table then return. + if (!isPhoenixTableTTLEnabled(connection.getQueryServices().getConfiguration())) { + return; + } + + if (!table.hasConditionalTTL()) { + return; + } + + ConditionalTTLExpression ttlExpr = (ConditionalTTLExpression) table.getTTL(); + Set colsReferenced = ttlExpr.getColumnsReferenced(connection, table); + for (ColumnReference colref : colsReferenced) { + // TODO Single Cell + scan.addColumn(colref.getFamily(), colref.getQualifier()); + } + } + public static void setScanAttributesForClient(Scan scan, PTable table, StatementContext context) throws SQLException { PhoenixConnection phoenixConnection = context.getConnection(); diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core-client/src/main/java/org/apache/phoenix/util/SchemaUtil.java index 61b22d594ac..2e0fd1b3236 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/util/SchemaUtil.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/util/SchemaUtil.java @@ -64,8 +64,10 @@ import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; +import org.apache.phoenix.parse.ColumnDef; import org.apache.phoenix.parse.ColumnParseNode; import org.apache.phoenix.parse.LiteralParseNode; +import org.apache.phoenix.parse.PrimaryKeyConstraint; import org.apache.phoenix.query.KeyRange; import org.apache.phoenix.query.QueryConstants; import org.apache.phoenix.query.QueryServices; @@ -158,7 +160,12 @@ private SchemaUtil() { public static boolean isPKColumn(PColumn column) { return column.getFamilyName() == null; } - + + public static boolean isPKColumn(PrimaryKeyConstraint pkConstraint, ColumnDef colDef) { + return colDef.isPK() || + (pkConstraint != null && pkConstraint.contains(colDef.getColumnDefName())); + } + /** * Imperfect estimate of row size given a PTable * TODO: keep row count in stats table and use total size / row count instead diff --git a/phoenix-core-client/src/main/protobuf/PTable.proto b/phoenix-core-client/src/main/protobuf/PTable.proto index 6533fdfcccf..bc9aa45a14c 100644 --- a/phoenix-core-client/src/main/protobuf/PTable.proto +++ b/phoenix-core-client/src/main/protobuf/PTable.proto @@ -23,6 +23,7 @@ option java_generate_equals_and_hash = true; option optimize_for = SPEED; import "PGuidePosts.proto"; +import "ServerCachingService.proto"; enum PTableType { SYSTEM = 0; @@ -129,3 +130,18 @@ message EncodedCQCounter { required string colFamily = 1; required int32 counter = 2; } + +message LiteralTTL { + required int32 ttlValue = 1; +} + +message ConditionTTL { + required string ttlExpression = 1; + required bytes compiledExpression = 2; + repeated ColumnReference ttlExpressionColumns = 30; +} + +message TTLExpression { + optional LiteralTTL literal = 1; + optional ConditionTTL condition = 2; +} diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/CompactionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/CompactionScanner.java index 241ea00f36b..f31154e2271 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/CompactionScanner.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/CompactionScanner.java @@ -17,20 +17,37 @@ */ package org.apache.phoenix.coprocessor; +import static org.apache.phoenix.coprocessor.CompactionScanner.MatcherType.GLOBAL_INDEXES; +import static org.apache.phoenix.coprocessor.CompactionScanner.MatcherType.GLOBAL_VIEWS; +import static org.apache.phoenix.coprocessor.CompactionScanner.MatcherType.TENANT_INDEXES; +import static org.apache.phoenix.coprocessor.CompactionScanner.MatcherType.TENANT_VIEWS; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAMESPACE_BYTES; +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES; +import static org.apache.phoenix.query.QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX; +import static org.apache.phoenix.query.QueryServices.PHOENIX_VIEW_TTL_TENANT_VIEWS_PER_SCAN_LIMIT; +import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_PHOENIX_VIEW_TTL_TENANT_VIEWS_PER_SCAN_LIMIT; +import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_FORVER; +import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_NOT_DEFINED; +import static org.apache.phoenix.util.ByteUtil.EMPTY_BYTE_ARRAY; + import java.io.IOException; import java.sql.Connection; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Types; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; -import java.util.Iterator; import java.util.HashSet; +import java.util.Iterator; import java.util.LinkedList; +import java.util.List; +import java.util.Map; import java.util.Properties; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; @@ -51,7 +68,10 @@ import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.phoenix.coprocessorclient.RowKeyMatcher; import org.apache.phoenix.coprocessorclient.TableInfo; +import org.apache.phoenix.coprocessorclient.TableTTLInfo; +import org.apache.phoenix.coprocessorclient.TableTTLInfoCache; import org.apache.phoenix.exception.SQLExceptionCode; import org.apache.phoenix.exception.SQLExceptionInfo; import org.apache.phoenix.expression.RowKeyColumnExpression; @@ -60,10 +80,16 @@ import org.apache.phoenix.jdbc.PhoenixPreparedStatement; import org.apache.phoenix.query.QueryServices; import org.apache.phoenix.query.QueryServicesOptions; +import org.apache.phoenix.schema.ConditionalTTLExpression; import org.apache.phoenix.schema.IllegalDataException; +import org.apache.phoenix.schema.PColumn; import org.apache.phoenix.schema.PName; import org.apache.phoenix.schema.PNameFactory; +import org.apache.phoenix.schema.PTable; +import org.apache.phoenix.schema.PTableType; +import org.apache.phoenix.schema.RowKeyValueAccessor; import org.apache.phoenix.schema.SortOrder; +import org.apache.phoenix.schema.TTLExpression; import org.apache.phoenix.schema.types.PDataType; import org.apache.phoenix.schema.types.PLong; import org.apache.phoenix.schema.types.PSmallint; @@ -71,14 +97,10 @@ import org.apache.phoenix.thirdparty.com.google.common.collect.Lists; import org.apache.phoenix.util.ByteUtil; import org.apache.phoenix.util.CDCUtil; -import org.apache.phoenix.util.PhoenixRuntime; -import org.apache.phoenix.coprocessorclient.RowKeyMatcher; -import org.apache.phoenix.coprocessorclient.TableTTLInfoCache; -import org.apache.phoenix.coprocessorclient.TableTTLInfo; -import org.apache.phoenix.schema.PTableType; -import org.apache.phoenix.schema.RowKeyValueAccessor; +import org.apache.phoenix.util.ClientUtil; import org.apache.phoenix.util.EnvironmentEdgeManager; import org.apache.phoenix.util.MetaDataUtil; +import org.apache.phoenix.util.PhoenixRuntime; import org.apache.phoenix.util.QueryUtil; import org.apache.phoenix.util.ScanUtil; import org.apache.phoenix.util.SchemaUtil; @@ -86,27 +108,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.phoenix.coprocessor.CompactionScanner.MatcherType.GLOBAL_INDEXES; -import static org.apache.phoenix.coprocessor.CompactionScanner.MatcherType.GLOBAL_VIEWS; -import static org.apache.phoenix.coprocessor.CompactionScanner.MatcherType.TENANT_INDEXES; -import static org.apache.phoenix.coprocessor.CompactionScanner.MatcherType.TENANT_VIEWS; -import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_TTL; -import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAMESPACE_BYTES; -import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES; -import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TTL_NOT_DEFINED; -import static org.apache.phoenix.query.QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX; -import static org.apache.phoenix.query.QueryServices.PHOENIX_VIEW_TTL_TENANT_VIEWS_PER_SCAN_LIMIT; -import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_PHOENIX_VIEW_TTL_TENANT_VIEWS_PER_SCAN_LIMIT; -import static org.apache.phoenix.util.ByteUtil.EMPTY_BYTE_ARRAY; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import org.apache.phoenix.schema.PColumn; -import org.apache.phoenix.schema.PTable; - /** * The store scanner that implements compaction for Phoenix. Phoenix coproc overrides the scan * options so that HBase store scanner retains all cells during compaction and flushes. Then this @@ -215,7 +216,8 @@ public CompactionScanner(RegionCoprocessorEnvironment env, LOGGER.info("Starting CompactionScanner for table " + tableName + " store " + columnFamilyName + (this.major ? " major " : " not major ") + "compaction ttl " - + ttlTracker.getRowContext().getTTL() + "ms " + "max lookback " + this.maxLookbackInMillis + "ms"); + + ttlTracker.getDefaultTTL() + " " + "max lookback " + + this.maxLookbackInMillis + "ms"); LOGGER.info(String.format("CompactionScanner params:- (" + "physical-data-tablename = %s, compaction-tablename = %s, region = %s, " + "start-key = %s, end-key = %s, " + @@ -230,7 +232,6 @@ public CompactionScanner(RegionCoprocessorEnvironment env, this.minVersion, this.maxVersion, this.keepDeletedCells.name(), this.familyCount, this.localIndex, this.emptyCFStore, compactionTime, maxLookbackWindowStart, maxLookbackInMillis, this.major)); - } @VisibleForTesting @@ -269,11 +270,6 @@ private TTLTracker createTTLTrackerFor(RegionCoprocessorEnvironment env, PHOENIX_VIEW_TTL_TENANT_VIEWS_PER_SCAN_LIMIT, DEFAULT_PHOENIX_VIEW_TTL_TENANT_VIEWS_PER_SCAN_LIMIT); } - // If VIEW TTL is not enabled then return TTL tracker for base HBase tables. - // since TTL can be set only at the table level. - if (!isViewTTLEnabled) { - return new NonPartitionedTableTTLTracker(baseTable, store); - } long currentTime = EnvironmentEdgeManager.currentTimeMillis(); String compactionTableName = env.getRegion().getRegionInfo().getTable().getNameAsString(); @@ -293,6 +289,12 @@ private TTLTracker createTTLTrackerFor(RegionCoprocessorEnvironment env, try (PhoenixConnection serverConnection = QueryUtil.getConnectionOnServer(new Properties(), env.getConfiguration()).unwrap(PhoenixConnection.class)) { + // If VIEW TTL is not enabled then return TTL tracker for base HBase tables. + // since TTL can be set only at the table level. + if (!isViewTTLEnabled) { + return new NonPartitionedTableTTLTracker(serverConnection, baseTable, store); + } + byte[] childLinkTableNameBytes = SchemaUtil.isNamespaceMappingEnabled( PTableType.SYSTEM, env.getConfiguration()) ? SYSTEM_CHILD_LINK_NAMESPACE_BYTES : @@ -307,9 +309,9 @@ private TTLTracker createTTLTrackerFor(RegionCoprocessorEnvironment env, currentTime); return isPartitioned ? - new PartitionedTableTTLTracker(baseTable, isSalted, isSharedIndex, - isLongViewIndexEnabled, viewTTLTenantViewsPerScanLimit) : - new NonPartitionedTableTTLTracker(baseTable, store); + new PartitionedTableTTLTracker(serverConnection, baseTable, isSalted, + isSharedIndex, isLongViewIndexEnabled, viewTTLTenantViewsPerScanLimit) + : new NonPartitionedTableTTLTracker(serverConnection, baseTable, store); } catch (SQLException e) { throw new IOException(e); @@ -356,8 +358,9 @@ static class CellTimeComparator implements Comparator { return false; } } - /* - private void printRow(List result, String title, boolean sort) { + + + private void printRow(List result, String title, boolean sort, boolean output) { List row; if (sort) { row = new ArrayList<>(result); @@ -370,31 +373,59 @@ private void printRow(List result, String title, boolean sort) { + "compaction time: " + compactionTime); System.out.println("Max lookback window start time: " + maxLookbackWindowStart); System.out.println("Max lookback in ms: " + maxLookbackInMillis); - System.out.println("TTL in ms: " + ttlInMillis); - boolean maxLookbackLine = false; - boolean ttlLine = false; + if (output) { + RowContext rowContext = phoenixLevelRowCompactor.rowContext; + System.out.println("TTL in ms: " + rowContext.ttl); + System.out.println("TTL window start time: " + rowContext.ttlWindowStart); + System.out.println("Max lookback window start time: " + + rowContext.maxLookbackWindowStartForRow); + } for (Cell cell : row) { - if (!maxLookbackLine && cell.getTimestamp() < maxLookbackWindowStart) { - System.out.println("-----> Max lookback window start time: " + maxLookbackWindowStart); - maxLookbackLine = true; - } else if (!ttlLine && cell.getTimestamp() < ttlWindowStart) { - System.out.println("-----> TTL window start time: " + ttlWindowStart); - ttlLine = true; - } System.out.println(cell); } } + + /** + * In case of conditional ttl, we run through the compaction algorithm with TTL set to FOREVER. + * The result of that should basically include all the versions within the maxlookback + * window and the last version visible through the maxlookback window. We evaluate the + * conditional TTL expression on that result. */ + private void postProcessForConditionalTTL(List result) { + RowContext rowContext = phoenixLevelRowCompactor.rowContext; + for (Cell cell : result) { + // If there are any cells within the maxlookback window retain everything. If we do a + // partial purge of cells outside the maxlookback window then next time when compaction + // runs we might not be able to evaluate the conditional ttl expression because cells + // of some columns which are referenced in the expression might not be available + // anymore and then the expression evaluation will fail all the time. As a result, + // we will end up retaining the row forever. + if (cell.getTimestamp() >= rowContext.getMaxLookbackWindowStart()) { + return; + } + } + ConditionalTTLExpression ttlExpr = (ConditionalTTLExpression) rowContext.ttlExprForRow; + if (ttlExpr.isExpired(result)) { + // If the row is expired, purge the row + result.clear(); + } + } @Override public boolean next(List result) throws IOException { boolean hasMore = storeScanner.next(result); inputCellCount += result.size(); if (!result.isEmpty()) { - // printRow(result, "Input for " + tableName + " " + columnFamilyName, true); // This is for debugging + // This is for debugging + // printRow(result, "Input for " + tableName + " " + columnFamilyName, true, false); phoenixLevelRowCompactor.compact(result, false); + // do post-processing for conditional TTL + if (phoenixLevelRowCompactor.rowContext.hasConditionalTTL()) { + postProcessForConditionalTTL(result); + } outputCellCount += result.size(); - // printRow(result, "Output for " + tableName + " " + columnFamilyName, true); // This is for debugging + // This is for debugging + // printRow(result, "Output for " + tableName + " " + columnFamilyName, true, true); } return hasMore; } @@ -584,7 +615,7 @@ private RowKeyMatcher initializeMatcher(MatcherType type) throws SQLException { if (tableList != null && !tableList.isEmpty()) { tableList.forEach(m -> { - if (m.getTTL() != TTL_NOT_DEFINED) { + if (m.getTTL() != TTL_EXPRESSION_NOT_DEFINED) { // add the ttlInfo to the cache. // each new/unique ttlInfo object added returns a unique tableId. int tableId = -1; @@ -664,7 +695,7 @@ private void refreshMatcher(MatcherType type) throws SQLException { if (tableList != null && !tableList.isEmpty()) { tableList.forEach(m -> { - if (m.getTTL() != TTL_NOT_DEFINED) { + if (m.getTTL() != TTL_EXPRESSION_NOT_DEFINED) { // add the ttlInfo to the cache. // each new/unique ttlInfo object added returns a unique tableId. int tableId = -1; @@ -1013,8 +1044,9 @@ private void getTTLInfo(String physicalTableName, String schem = viewTTLRS.getString("TABLE_SCHEM"); String tName = viewTTLRS.getString("TABLE_NAME"); String viewTTLStr = viewTTLRS.getString("TTL"); - int viewTTL = viewTTLStr == null || viewTTLStr.isEmpty() ? - TTL_NOT_DEFINED : Integer.valueOf(viewTTLStr); + TTLExpression viewTTL = viewTTLStr == null || viewTTLStr.isEmpty() + ? TTL_EXPRESSION_NOT_DEFINED + : TTLExpression.create(viewTTLStr); byte[] rowKeyMatcher = viewTTLRS.getBytes("ROW_KEY_MATCHER"); byte[] tenantIdBytes = @@ -1031,9 +1063,7 @@ private void getTTLInfo(String physicalTableName, tableConnection = QueryUtil.getConnectionOnServer(tenantProps, configuration)) { - PTable - pTable = - PhoenixRuntime.getTableNoCache( + PTable pTable = PhoenixRuntime.getTableNoCache( tableConnection, fullTableName); for (PTable index : pTable.getIndexes()) { // Handling the case when it is a table level index. @@ -1049,6 +1079,12 @@ private void getTTLInfo(String physicalTableName, viewIndexIdBytes = PLong.INSTANCE.toBytes(index.getViewIndexId()); } + TTLExpression indexTTL = index.getTTL(); + if (indexTTL instanceof ConditionalTTLExpression) { + indexTTL.compileTTLExpression( + tableConnection.unwrap(PhoenixConnection.class), + index); + } tableTTLInfoList.add( new TableTTLInfo(pTable.getPhysicalName().getBytes(), tenantIdBytes, index.getTableName().getBytes(), @@ -1057,6 +1093,17 @@ private void getTTLInfo(String physicalTableName, } } else { + if (viewTTL instanceof ConditionalTTLExpression) { + try (Connection tableConnection = + QueryUtil.getConnectionOnServer(tenantProps, + configuration)) { + PTable pTable = PhoenixRuntime.getTableNoCache( + tableConnection, fullTableName); + viewTTL.compileTTLExpression( + tableConnection.unwrap(PhoenixConnection.class), + pTable); + } + } tableTTLInfoList.add( new TableTTLInfo(physicalTableName.getBytes(), tenantIdBytes, fullTableName.getBytes(), @@ -1176,12 +1223,11 @@ public int getNumTablesInCache() { * For Flushes and Minor compaction we do not need to track the TTL. */ private interface TTLTracker { - // Set the TTL for the given row in the row-context being tracked. - void setTTL(Cell firstCell) throws IOException; - // get the row context for the current row. - RowContext getRowContext(); - // set the row context for the current row. - void setRowContext(RowContext rowContext); + // get TTL for the row + TTLExpression getTTLExpressionForRow(List result) throws IOException; + + // get the default TTL (TTL of the base table) + TTLExpression getDefaultTTL(); } /** @@ -1190,87 +1236,65 @@ private interface TTLTracker { */ private class TableTTLTrackerForFlushesAndMinor implements TTLTracker { - private long ttl; - private RowContext rowContext; + private TTLExpression ttlExpr; public TableTTLTrackerForFlushesAndMinor(String tableName) { - ttl = DEFAULT_TTL; + ttlExpr = TTL_EXPRESSION_FORVER; LOGGER.info(String.format( "TableTTLTrackerForFlushesAndMinor params:- " + - "(table-name=%s, ttl=%d)", - tableName, ttl*1000)); + "(table-name=%s, ttl=%s)", + tableName, ttlExpr)); } @Override - public void setTTL(Cell firstCell) { - if (this.rowContext == null) { - this.rowContext = new RowContext(); - } - this.rowContext.setTTL(ttl); - - } - - @Override - public RowContext getRowContext() { - if (this.rowContext == null) { - this.rowContext = new RowContext(); - this.rowContext.setTTL(ttl); - } - return rowContext; + public TTLExpression getTTLExpressionForRow(List result) throws IOException { + return ttlExpr; } @Override - public void setRowContext(RowContext rowContext) { - this.rowContext = rowContext; - this.rowContext.setTTL(ttl); + public TTLExpression getDefaultTTL() { + return ttlExpr; } } private class NonPartitionedTableTTLTracker implements TTLTracker { - private long ttl; - private RowContext rowContext; + private TTLExpression ttlExpr; public NonPartitionedTableTTLTracker( + PhoenixConnection pConn, PTable pTable, - Store store) { + Store store) throws IOException { boolean isSystemTable = pTable.getType() == PTableType.SYSTEM; if (isSystemTable) { ColumnFamilyDescriptor cfd = store.getColumnFamilyDescriptor(); - ttl = cfd.getTimeToLive(); + ttlExpr = TTLExpression.create(cfd.getTimeToLive()); } else { - ttl = pTable.getTTL() != TTL_NOT_DEFINED ? pTable.getTTL() : DEFAULT_TTL; + ttlExpr = pTable.getTTL() != TTL_EXPRESSION_NOT_DEFINED + ? pTable.getTTL() : TTL_EXPRESSION_FORVER; + } + try { + ttlExpr.compileTTLExpression(pConn, pTable); + } catch (SQLException e) { + throw ClientUtil.createIOException( + String.format("Error compiling ttl expression %s", ttlExpr), e); } LOGGER.info(String.format( "NonPartitionedTableTTLTracker params:- " + - "(physical-name=%s, ttl=%d, isSystemTable=%s)", - pTable.getName().toString(), ttl*1000, isSystemTable)); + "(physical-name=%s, ttl=%s, isSystemTable=%s)", + pTable.getName().toString(), ttlExpr, isSystemTable)); } @Override - public void setTTL(Cell firstCell) { - if (this.rowContext == null) { - this.rowContext = new RowContext(); - } - this.rowContext.setTTL(ttl); - + public TTLExpression getTTLExpressionForRow(List result) throws IOException { + return ttlExpr; } @Override - public RowContext getRowContext() { - if (this.rowContext == null) { - this.rowContext = new RowContext(); - this.rowContext.setTTL(ttl); - } - return rowContext; - } - - @Override - public void setRowContext(RowContext rowContext) { - this.rowContext = rowContext; - this.rowContext.setTTL(ttl); + public TTLExpression getDefaultTTL() { + return ttlExpr; } } @@ -1279,9 +1303,7 @@ private class PartitionedTableTTLTracker implements TTLTracker { PartitionedTableTTLTracker.class); // Default or Table-Level TTL - private long ttl; - private RowContext rowContext; - + private TTLExpression ttlExpr; private boolean isSharedIndex = false; private boolean isMultiTenant = false; private boolean isSalted = false; @@ -1290,6 +1312,7 @@ private class PartitionedTableTTLTracker implements TTLTracker { private PartitionedTableRowKeyMatcher tableRowKeyMatcher; public PartitionedTableTTLTracker( + PhoenixConnection pConn, PTable table, boolean isSalted, boolean isSharedIndex, @@ -1302,21 +1325,28 @@ public PartitionedTableTTLTracker( this.tableRowKeyMatcher = new PartitionedTableRowKeyMatcher(table, isSalted, isSharedIndex, isLongViewIndexEnabled, viewTTLTenantViewsPerScanLimit); - this.ttl = table.getTTL() != TTL_NOT_DEFINED ? table.getTTL() : DEFAULT_TTL; + this.ttlExpr = table.getTTL() != TTL_EXPRESSION_NOT_DEFINED + ? table.getTTL() : TTL_EXPRESSION_FORVER; + try { + this.ttlExpr.compileTTLExpression(pConn, table); + } catch (SQLException e) { + throw ClientUtil.createIOException( + String.format("Error compiling ttl expression %s", this.ttlExpr), e); + } this.isSharedIndex = isSharedIndex || localIndex; this.isLongViewIndexEnabled = isLongViewIndexEnabled; this.isSalted = isSalted; this.isMultiTenant = table.isMultiTenant(); - this.startingPKPosition = getStartingPKPosition();; + this.startingPKPosition = getStartingPKPosition(); LOGGER.info(String.format( "PartitionedTableTTLTracker params:- " + "region-name = %s, table-name = %s, " + "multi-tenant = %s, shared-index = %s, salted = %s, " + - "default-ttl = %d, startingPKPosition = %d", + "default-ttl = %s, startingPKPosition = %d", region.getRegionInfo().getEncodedName(), region.getRegionInfo().getTable().getNameAsString(), this.isMultiTenant, - this.isSharedIndex, this.isSalted, this.ttl, this.startingPKPosition)); + this.isSharedIndex, this.isSalted, this.ttlExpr, this.startingPKPosition)); } catch (SQLException e) { LOGGER.error(String.format("Failed to read from catalog: " + e.getMessage())); @@ -1391,15 +1421,16 @@ private int getStartingPKPosition() { } @Override - public void setTTL(Cell firstCell) throws IOException { - + public TTLExpression getTTLExpressionForRow(List result) throws IOException { boolean matched = false; TableTTLInfo tableTTLInfo = null; List pkPositions = null; - long rowTTLInSecs = ttl; + TTLExpression defaultTTLExpr = ttlExpr; + TTLExpression rowTTLExpr = defaultTTLExpr; long matchedOffset = -1; int pkPosition = startingPKPosition; MatcherType matchedType = null; + Cell firstCell = result.get(0); try { // pkPositions holds the byte offsets for the PKs of the base table // for the current row @@ -1451,28 +1482,29 @@ public void setTTL(Cell firstCell) throws IOException { } matched = tableTTLInfo != null; matchedOffset = matched ? offset : -1; - rowTTLInSecs = matched ? tableTTLInfo.getTTL() : ttl; /* in secs */ - if (this.rowContext == null) { - this.rowContext = new RowContext(); + if (matched) { + rowTTLExpr = tableTTLInfo.getTTL(); + } else { + rowTTLExpr = defaultTTLExpr; /* in secs */ } - this.rowContext.setTTL(rowTTLInSecs); + return rowTTLExpr; } catch (SQLException e) { LOGGER.error(String.format("Exception when visiting table: " + e.getMessage())); throw new IOException(e); } finally { if (LOGGER.isTraceEnabled()) { - LOGGER.trace(String.format("visiting row-key = %s, region = %s, " + - "table-ttl-info=%s, " + - "matched = %s, matched-type = %s, match-pattern = %s, " + - "ttl = %d, matched-offset = %d, " + - "pk-pos = %d, pk-pos-list = %s", + LOGGER.trace(String.format("visiting row-key = %s, region = %s, " + + "table-ttl-info=%s, " + + "matched = %s, matched-type = %s, match-pattern = %s, " + + "ttlExpr = %s, matched-offset = %d, " + + "pk-pos = %d, pk-pos-list = %s", CellUtil.getCellKeyAsString(firstCell), CompactionScanner.this.store.getRegionInfo().getEncodedName(), matched ? tableTTLInfo : "NULL", matched, matchedType, matched ? Bytes.toStringBinary(tableTTLInfo.getMatchPattern()) : "NULL", - rowTTLInSecs, + rowTTLExpr, matchedOffset, pkPosition, pkPositions != null ? pkPositions.stream() @@ -1480,22 +1512,11 @@ public void setTTL(Cell firstCell) throws IOException { .collect(Collectors.joining(",")) : "")); } } - } @Override - public RowContext getRowContext() { - if (this.rowContext == null) { - this.rowContext = new RowContext(); - this.rowContext.setTTL(ttl); - } - return rowContext; - } - - @Override - public void setRowContext(RowContext rowContext) { - this.rowContext = rowContext; - this.rowContext.setTTL(ttl); + public TTLExpression getDefaultTTL() { + return ttlExpr; } } @@ -1747,6 +1768,7 @@ class RowContext { long maxTimestamp; long minTimestamp; long ttl; + TTLExpression ttlExprForRow; long ttlWindowStart; long maxLookbackWindowStartForRow; @@ -1757,16 +1779,25 @@ private void init() { version = 0; } - public void setTTL(long ttlInSecs) { - this.ttl = ttlInSecs*1000; + private void setTTL(long ttlInSecs) { + this.ttl = Math.max(ttlInSecs * 1000, maxLookbackInMillis + 1); this.ttlWindowStart = ttlInSecs == HConstants.FOREVER ? 1 : compactionTime - ttl ; this.maxLookbackWindowStartForRow = Math.max(ttlWindowStart, maxLookbackWindowStart); if (LOGGER.isTraceEnabled()) { LOGGER.trace(String.format("RowContext:- (ttlWindowStart=%d, maxLookbackWindowStart=%d)", ttlWindowStart, maxLookbackWindowStart)); } + } + public void setTTL(TTLExpression ttlExpr, List result) { + ttlExprForRow = ttlExpr; + setTTL(ttlExprForRow.getRowTTLForCompaction(result)); } + + public boolean hasConditionalTTL() { + return ttlExprForRow != null && ttlExprForRow instanceof ConditionalTTLExpression; + } + public long getTTL() { return ttl; } @@ -2103,9 +2134,10 @@ else if (cell.getType() == Cell.Type.DeleteFamilyVersion) { } private void formCompactionRowVersions(LinkedList> columns, - List result) { + List result) throws IOException { rowContext.init(); - rowTracker.setRowContext(rowContext); + TTLExpression ttlExprForRow = rowTracker.getTTLExpressionForRow(result); + rowContext.setTTL(ttlExprForRow, result); while (!columns.isEmpty()) { formNextCompactionRowVersion(columns, rowContext, result); // Remove the columns that are empty @@ -2150,7 +2182,7 @@ private void formColumns(List result, LinkedList> columns * Compacts a single row at the HBase level. The result parameter is the input row and * modified to be the output of the compaction. */ - private void compact(List result) { + private void compact(List result) throws IOException { if (result.isEmpty()) { return; } @@ -2201,7 +2233,7 @@ private void getLastRowVersionInMaxLookbackWindow(List result, Cell currentColumnCell = null; boolean isEmptyColumn = false; for (Cell cell : result) { - long maxLookbackWindowStart = rowTracker.getRowContext().getMaxLookbackWindowStart(); + long maxLookbackWindowStart = rowContext.getMaxLookbackWindowStart(); if (cell.getTimestamp() > maxLookbackWindowStart) { retainedCells.add(cell); continue; @@ -2307,8 +2339,9 @@ private void retainCellsOfLastRowVersion(List lastRow, if (lastRow.isEmpty()) { return; } + // init doesn't change ttl rowContext.init(); - rowTracker.setRowContext(rowContext); + // ttl has already been evaluated long ttl = rowContext.getTTL(); rowContext.getNextRowVersionTimestamps(lastRow, storeColumnFamily); Cell firstCell = lastRow.get(0); @@ -2365,7 +2398,7 @@ else if (! major) { */ private boolean retainCellsForCDCIndex(List result, List retainedCells) { for (Cell cell : result) { - if (cell.getTimestamp() >= rowTracker.getRowContext().getMaxLookbackWindowStart()) { + if (cell.getTimestamp() >= rowContext.getMaxLookbackWindowStart()) { retainedCells.add(cell); } } @@ -2395,7 +2428,7 @@ private boolean retainCellsForMaxLookback(List result, boolean regionLevel retainCellsOfLastRowVersion(lastRowVersion, emptyColumn, retainedCells); return true; } - long ttl = rowTracker.getRowContext().getTTL(); + long ttl = rowContext.getTTL(); long maxTimestamp = 0; long minTimestamp = Long.MAX_VALUE; long ts; @@ -2489,6 +2522,7 @@ private void removeDuplicates(List input, List output) { previousCell = cell; } } + /** * Compacts a single row at the Phoenix level. The result parameter is the input row and * modified to be the output of the compaction process. @@ -2498,7 +2532,8 @@ private void compact(List result, boolean regionLevel) throws IOException return; } phoenixResult.clear(); - rowTracker.setTTL(result.get(0)); + TTLExpression ttlExprForRow = rowTracker.getTTLExpressionForRow(result); + rowContext.setTTL(ttlExprForRow, result); // For multi-CF case, always do region level scan for empty CF store during major compaction else // we could end-up removing some empty cells which are needed to close the gap b/w empty CF cell and // non-empty CF cell to prevent partial row expiry. This can happen when last row version of non-empty @@ -2513,6 +2548,7 @@ else if (!retainCellsForMaxLookback(result, regionLevel, phoenixResult)) { phoenixResult.clear(); compactRegionLevel(result, phoenixResult); } + if (maxVersion == 1 && (!major || (minVersion == 0 && keepDeletedCells == KeepDeletedCells.FALSE))) { diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java index 52ca74c1d54..cf80d4d2b8a 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/GlobalIndexRegionScanner.java @@ -433,12 +433,6 @@ public void close() throws IOException { } } - @VisibleForTesting - public int setIndexTableTTL(int ttl) { - indexTableTTL = ttl; - return 0; - } - @VisibleForTesting public int setIndexMaintainer(IndexMaintainer indexMaintainer) { this.indexMaintainer = indexMaintainer; @@ -857,15 +851,6 @@ public boolean verifySingleIndexRow(byte[] indexRowKey, List actualMut while (expectedIndex < expectedSize && actualIndex > actualIndexMutationMa ClientUtil.throwIOException(region.getRegionInfo().getRegionNameAsString(), t); } } - List expiredIndexRows = new ArrayList<>(); - // Check if any expected rows from index(which we didn't get) are already expired due to TTL - // TODO: metrics for expired rows - long currentTime = EnvironmentEdgeManager.currentTimeMillis(); - for (Map.Entry> entry: expectedIndexMutationMap.entrySet()) { - List mutationList = entry.getValue(); - if (isTimestampBeforeTTL(indexTableTTL, currentTime, getTimestamp(mutationList.get(mutationList.size() - 1)))) { - verificationPhaseResult.setExpiredIndexRowCount(verificationPhaseResult.getExpiredIndexRowCount() + 1); - expiredIndexRows.add(entry.getKey()); - } - } - // Remove the expired rows from indexMutationMap - for (byte[] indexKey : expiredIndexRows) { - expectedIndexMutationMap.remove(indexKey); - } // Count and log missing rows for (Map.Entry> entry: expectedIndexMutationMap.entrySet()) { byte[] indexKey = entry.getKey(); @@ -1020,7 +990,7 @@ protected void verifyIndexRows(Map> actualIndexMutationMa if (mutation instanceof Delete) { continue; } - currentTime = EnvironmentEdgeManager.currentTimeMillis(); + long currentTime = EnvironmentEdgeManager.currentTimeMillis(); String errorMsg; IndexVerificationOutputRepository.IndexVerificationErrorType errorType; if (isTimestampBeyondMaxLookBack(maxLookBackInMills, currentTime, getTimestamp(mutation))){ diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/IndexRebuildRegionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/IndexRebuildRegionScanner.java index 4fd416d5054..3470108aa2e 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/IndexRebuildRegionScanner.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/IndexRebuildRegionScanner.java @@ -91,11 +91,6 @@ public IndexRebuildRegionScanner(final RegionScanner innerScanner, super(innerScanner, region, scan, env, ungroupedAggregateRegionObserver); indexHTable = hTableFactory.getTable(new ImmutableBytesPtr(indexMaintainer.getIndexTableName())); - if (BaseScannerRegionObserver.isPhoenixTableTTLEnabled(env.getConfiguration())) { - indexTableTTL = ScanUtil.getTTL(scan); - } else { - indexTableTTL = indexHTable.getDescriptor().getColumnFamilies()[0].getTimeToLive(); - } indexRowKeyforReadRepair = scan.getAttribute(BaseScannerRegionObserverConstants.INDEX_ROW_KEY); if (indexRowKeyforReadRepair != null) { setReturnCodeForSingleRowRebuild(); diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/IndexRepairRegionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/IndexRepairRegionScanner.java index f82088d217b..43ce7faa076 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/IndexRepairRegionScanner.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/IndexRepairRegionScanner.java @@ -89,11 +89,6 @@ public IndexRepairRegionScanner(final RegionScanner innerScanner, byte[] dataTableName = scan.getAttribute(PHYSICAL_DATA_TABLE_NAME); dataHTable = hTableFactory.getTable(new ImmutableBytesPtr(dataTableName)); - if (BaseScannerRegionObserver.isPhoenixTableTTLEnabled(env.getConfiguration())) { - indexTableTTL = ScanUtil.getTTL(scan); - } else { - indexTableTTL = indexHTable.getDescriptor().getColumnFamilies()[0].getTimeToLive(); - } try (org.apache.hadoop.hbase.client.Connection connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(env.getConfiguration())) { regionEndKeys = connection.getRegionLocator(dataHTable.getName()).getEndKeys(); diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/IndexerRegionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/IndexerRegionScanner.java index 105d21f1ba2..c64612ac04f 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/IndexerRegionScanner.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/IndexerRegionScanner.java @@ -27,7 +27,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -99,11 +98,6 @@ public class IndexerRegionScanner extends GlobalIndexRegionScanner { UngroupedAggregateRegionObserver ungroupedAggregateRegionObserver) throws IOException { super(innerScanner, region, scan, env, ungroupedAggregateRegionObserver); indexHTable = hTableFactory.getTable(new ImmutableBytesPtr(indexMaintainer.getIndexTableName())); - if (BaseScannerRegionObserver.isPhoenixTableTTLEnabled(env.getConfiguration())) { - indexTableTTL = ScanUtil.getTTL(scan); - } else { - indexTableTTL = indexHTable.getDescriptor().getColumnFamilies()[0].getTimeToLive(); - } pool = new WaitForCompletionTaskRunner(ThreadPoolManager.getExecutor( new ThreadPoolBuilder("IndexVerify", env.getConfiguration()).setMaxThread(NUM_CONCURRENT_INDEX_VERIFY_THREADS_CONF_KEY, @@ -174,16 +168,6 @@ valueGetter, new ImmutableBytesWritable(dataRow.getRow()), ts, null, null, byte[] qualifier = CellUtil.cloneQualifier(expectedCell); Cell actualCell = indexRow.getColumnLatestCell(family, qualifier); if (actualCell == null) { - // Check if cell expired as per the current server's time and data table ttl - // Index table should have the same ttl as the data table, hence we might not - // get a value back from index if it has already expired between our rebuild and - // verify - - // or if cell timestamp is beyond maxlookback - if (isTimestampBeforeTTL(indexTableTTL, currentTime, expectedCell.getTimestamp())) { - continue; - } - return false; } if (actualCell.getTimestamp() < ts) { @@ -233,19 +217,6 @@ private void verifyIndexRows(List keys, Map perTaskDataKe } catch (Throwable t) { ClientUtil.throwIOException(indexHTable.getName().toString(), t); } - // Check if any expected rows from index(which we didn't get) are already expired due to TTL - if (!perTaskDataKeyToDataPutMap.isEmpty()) { - Iterator> itr = perTaskDataKeyToDataPutMap.entrySet().iterator(); - long currentTime = EnvironmentEdgeManager.currentTime(); - while(itr.hasNext()) { - Entry entry = itr.next(); - long ts = IndexUtil.getMaxTimestamp(entry.getValue()); - if (isTimestampBeforeTTL(indexTableTTL, currentTime, ts)) { - itr.remove(); - verificationPhaseResult.setExpiredIndexRowCount(verificationPhaseResult.getExpiredIndexRowCount()+1); - } - } - } // Check if any expected rows from index(which we didn't get) are beyond max look back and have been compacted away if (!perTaskDataKeyToDataPutMap.isEmpty()) { for (Entry entry : perTaskDataKeyToDataPutMap.entrySet()) { 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 2cb00501d84..6c0c7a7782e 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 @@ -58,7 +58,6 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ORDINAL_POSITION_BYTES; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PARENT_TENANT_ID_BYTES; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES; -import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TTL_NOT_DEFINED; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PHYSICAL_TABLE_NAME_BYTES; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PK_NAME_BYTES; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.RETURN_TYPE_BYTES; @@ -96,6 +95,8 @@ import static org.apache.phoenix.schema.PTableType.CDC; import static org.apache.phoenix.schema.PTableType.INDEX; import static org.apache.phoenix.schema.PTableType.VIEW; +import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_FORVER; +import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_NOT_DEFINED; import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB; import static org.apache.phoenix.util.SchemaUtil.*; import static org.apache.phoenix.util.ViewUtil.findAllDescendantViews; @@ -242,6 +243,7 @@ import org.apache.phoenix.schema.SequenceKey; import org.apache.phoenix.schema.SequenceNotFoundException; import org.apache.phoenix.schema.SortOrder; +import org.apache.phoenix.schema.TTLExpression; import org.apache.phoenix.schema.TableNotFoundException; import org.apache.phoenix.schema.export.SchemaRegistryRepository; import org.apache.phoenix.schema.export.SchemaRegistryRepositoryFactory; @@ -1480,17 +1482,17 @@ private PTable getTableFromCells(List tableCellList, List> allC maxLookbackAge = scanMaxLookbackAgeFromParent(viewKey, clientTimeStamp); } Cell ttlKv = tableKeyValues[TTL_INDEX]; - int ttl = TTL_NOT_DEFINED; + TTLExpression ttl = TTL_EXPRESSION_NOT_DEFINED; if (ttlKv != null) { String ttlStr = (String) PVarchar.INSTANCE.toObject( ttlKv.getValueArray(), ttlKv.getValueOffset(), ttlKv.getValueLength()); - ttl = Integer.parseInt(ttlStr); + ttl = TTLExpression.create(ttlStr); } ttl = ttlKv != null ? ttl : oldTable != null - ? oldTable.getTTL() : TTL_NOT_DEFINED; - if (tableType == VIEW && viewType != MAPPED && ttl == TTL_NOT_DEFINED) { + ? oldTable.getTTL() : TTL_EXPRESSION_NOT_DEFINED; + if (tableType == VIEW && viewType != MAPPED && ttl == TTL_EXPRESSION_NOT_DEFINED) { //Scan SysCat to get TTL from Parent View/Table byte[] viewKey = SchemaUtil.getTableKey(tenantId == null ? null : tenantId.getBytes(), schemaName == null ? null : schemaName.getBytes(), tableNameBytes); @@ -1677,13 +1679,18 @@ private PTable getTableFromCells(List tableCellList, List> allC builder.setMaxLookbackAge(maxLookbackAge != null ? maxLookbackAge : (oldTable != null ? oldTable.getMaxLookbackAge() : null)); - if(tableType == INDEX && !isThisAViewIndex && ttl == TTL_NOT_DEFINED) { + if (tableType == INDEX && !isThisAViewIndex && ttl == TTL_EXPRESSION_NOT_DEFINED) { //If this is an index on Table get TTL from Table byte[] tableKey = getTableKey(tenantId == null ? null : tenantId.getBytes(), parentSchemaName == null ? null : parentSchemaName.getBytes(), parentTableName.getBytes()); ttl = getTTLForTable(tableKey, clientTimeStamp); } + if (tableType == INDEX + && CDCUtil.isCDCIndex(tableName.getString()) + && ttl != TTL_EXPRESSION_NOT_DEFINED) { + ttl = TTL_EXPRESSION_FORVER; + } builder.setTTL(ttl); builder.setEncodedCQCounter(cqCounter); @@ -1783,7 +1790,10 @@ private byte[] getParentTableKeyFromChildRowKeyMetaData(byte[][] rowKeyMetaData) * @throws SQLException */ - private int getTTLFromHierarchy(byte[] viewKey, long clientTimeStamp, boolean checkForMappedView) throws IOException, SQLException { + private TTLExpression getTTLFromHierarchy( + byte[] viewKey, + long clientTimeStamp, + boolean checkForMappedView) throws IOException, SQLException { Scan scan = MetaDataUtil.newTableRowsScan(viewKey, MIN_TABLE_TIMESTAMP, clientTimeStamp); Table sysCat = ServerUtil.getHTableForCoprocessorScan(this.env, SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, @@ -1795,12 +1805,12 @@ private int getTTLFromHierarchy(byte[] viewKey, long clientTimeStamp, boolean ch do { if (result == null) { - return TTL_NOT_DEFINED; + return TTL_EXPRESSION_NOT_DEFINED; } //return TTL_NOT_DEFINED for Index on a Mapped View. if (checkForMappedView && checkIfViewIsMappedView(result)) { - return TTL_NOT_DEFINED; + return TTL_EXPRESSION_NOT_DEFINED; } byte[] linkTypeBytes = result.getValue(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES); @@ -1810,7 +1820,7 @@ private int getTTLFromHierarchy(byte[] viewKey, long clientTimeStamp, boolean ch if (result.getValue(TABLE_FAMILY_BYTES, TTL_BYTES) != null) { String ttlStr = (String) PVarchar.INSTANCE.toObject( result.getValue(DEFAULT_COLUMN_FAMILY_BYTES, TTL_BYTES)); - return Integer.parseInt(ttlStr); + return TTLExpression.create(ttlStr); } else if (linkTypeBytes != null ) { String parentSchema =SchemaUtil.getSchemaNameFromFullName( rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]); @@ -1859,7 +1869,7 @@ private boolean checkIfViewIsMappedView(Result result) { * @return TTL defined for a given table if it is null then return TTL_NOT_DEFINED(0) * @throws IOException */ - private int getTTLForTable(byte[] tableKey, long clientTimeStamp) throws IOException { + private TTLExpression getTTLForTable(byte[] tableKey, long clientTimeStamp) throws IOException { Scan scan = MetaDataUtil.newTableRowsScan(tableKey, MIN_TABLE_TIMESTAMP, clientTimeStamp); Table sysCat = ServerUtil.getHTableForCoprocessorScan(this.env, SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, @@ -1868,16 +1878,16 @@ private int getTTLForTable(byte[] tableKey, long clientTimeStamp) throws IOExcep Result result = scanner.next(); do { if (result == null) { - return TTL_NOT_DEFINED; + return TTL_EXPRESSION_NOT_DEFINED; } if (result.getValue(TABLE_FAMILY_BYTES, TTL_BYTES) != null) { String ttlStr = (String) PVarchar.INSTANCE.toObject( result.getValue(DEFAULT_COLUMN_FAMILY_BYTES, TTL_BYTES)); - return Integer.parseInt(ttlStr); + return TTLExpression.create(ttlStr); } result = scanner.next(); } while (result != null); - return TTL_NOT_DEFINED; + return TTL_EXPRESSION_NOT_DEFINED; } private Long getViewIndexId(Cell[] tableKeyValues, PDataType viewIndexIdType) { @@ -3857,8 +3867,8 @@ private boolean validateTTLAttributeSettingForEntity( Cell cell = cells.get(0); String newTTLStr = (String) PVarchar.INSTANCE.toObject(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); - int newTTL = Integer.parseInt(newTTLStr); - return newTTL != TTL_NOT_DEFINED; + TTLExpression newTTL = TTLExpression.create(newTTLStr); + return newTTL != TTL_EXPRESSION_NOT_DEFINED; } } } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/TTLRegionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/TTLRegionScanner.java index 2543128ece8..a17b5ee8d4f 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/TTLRegionScanner.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/TTLRegionScanner.java @@ -27,14 +27,14 @@ import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.filter.PageFilter; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScannerContext; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.phoenix.query.QueryServices; -import org.apache.phoenix.query.QueryServicesOptions; +import org.apache.phoenix.schema.TTLExpression; import org.apache.phoenix.util.EnvironmentEdgeManager; import org.apache.phoenix.util.ScanUtil; import org.slf4j.Logger; @@ -44,6 +44,7 @@ import static org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants.EMPTY_COLUMN_QUALIFIER_NAME; import static org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants.IS_PHOENIX_TTL_SCAN_TABLE_SYSTEM; import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.isPhoenixTableTTLEnabled; +import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_FORVER; /** * TTLRegionScanner masks expired rows using the empty column cell timestamp @@ -62,33 +63,35 @@ public class TTLRegionScanner extends BaseRegionScanner { byte[] emptyCQ; byte[] emptyCF; private boolean initialized = false; + private TTLExpression ttlExpression; + long currentTime; public TTLRegionScanner(final RegionCoprocessorEnvironment env, final Scan scan, - final RegionScanner s) { + final RegionScanner s) throws IOException { super(s); this.env = env; this.scan = scan; this.pageSizeMs = ScanUtil.getPageSizeMsForRegionScanner(scan); emptyCQ = scan.getAttribute(EMPTY_COLUMN_QUALIFIER_NAME); emptyCF = scan.getAttribute(EMPTY_COLUMN_FAMILY_NAME); - long currentTime = scan.getTimeRange().getMax() == HConstants.LATEST_TIMESTAMP ? - EnvironmentEdgeManager.currentTimeMillis() : scan.getTimeRange().getMax(); + currentTime = scan.getTimeRange().getMax() == HConstants.LATEST_TIMESTAMP + ? EnvironmentEdgeManager.currentTimeMillis() : scan.getTimeRange().getMax(); byte[] isSystemTable = scan.getAttribute(IS_PHOENIX_TTL_SCAN_TABLE_SYSTEM); if (isPhoenixTableTTLEnabled(env.getConfiguration()) && (isSystemTable == null || !Bytes.toBoolean(isSystemTable))) { - ttl = ScanUtil.getTTL(this.scan); + ttlExpression = ScanUtil.getTTLExpression(this.scan); } else { - ttl = env.getRegion().getTableDescriptor().getColumnFamilies()[0].getTimeToLive(); + ColumnFamilyDescriptor cfd = + env.getRegion().getTableDescriptor().getColumnFamilies()[0]; + ttlExpression = TTLExpression.create(cfd.getTimeToLive()); } // Regardless if the Phoenix Table TTL feature is disabled cluster wide or the client is // an older client and does not supply the empty column parameters, the masking should not // be done here. We also disable masking when TTL is HConstants.FOREVER. - isMaskingEnabled = emptyCF != null && emptyCQ != null && ttl != HConstants.FOREVER + isMaskingEnabled = emptyCF != null && emptyCQ != null + && ttlExpression != TTL_EXPRESSION_FORVER && (isPhoenixTableTTLEnabled(env.getConfiguration()) && (isSystemTable == null || !Bytes.toBoolean(isSystemTable))); - - ttlWindowStart = ttl == HConstants.FOREVER ? 1 : currentTime - ttl * 1000; - ttl *= 1000; } private void init() throws IOException { @@ -102,11 +105,18 @@ private void init() throws IOException { } } + private void setTTLContextForRow(List result) { + ttl = ttlExpression.getRowTTLForMasking(result); + ttlWindowStart = ttl == HConstants.FOREVER ? 1 : currentTime - ttl * 1000; + ttl *= 1000; + } + private boolean isExpired(List result) throws IOException { long maxTimestamp = 0; long minTimestamp = Long.MAX_VALUE; long ts; boolean found = false; + setTTLContextForRow(result); for (Cell c : result) { ts = c.getTimestamp(); if (!found && ScanUtil.isEmptyColumn(c, emptyCF, emptyCQ)) { diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyMapper.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyMapper.java index 78161a9c644..925edc9ac53 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyMapper.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyMapper.java @@ -35,6 +35,7 @@ import org.apache.phoenix.query.QueryServices; import org.apache.phoenix.query.QueryServicesOptions; +import org.apache.phoenix.schema.LiteralTTLExpression; import org.apache.phoenix.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.commons.codec.binary.Hex; import org.apache.hadoop.conf.Configuration; @@ -69,7 +70,7 @@ import org.apache.phoenix.thirdparty.com.google.common.base.Joiner; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_TTL; -import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TTL_NOT_DEFINED; +import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_NOT_DEFINED; /** * Mapper that reads from the data table and checks the rows against the index table @@ -344,8 +345,8 @@ private int getTableTTL(Configuration configuration) throws SQLException, IOExce SchemaUtil.isNamespaceMappingEnabled(null, cqsi.getProps())); if (configuration.getBoolean(QueryServices.PHOENIX_TABLE_TTL_ENABLED, QueryServicesOptions.DEFAULT_PHOENIX_TABLE_TTL_ENABLED)) { - return pSourceTable.getTTL() == TTL_NOT_DEFINED ? DEFAULT_TTL - : pSourceTable.getTTL(); + return pSourceTable.getTTL() == TTL_EXPRESSION_NOT_DEFINED ? DEFAULT_TTL + : ((LiteralTTLExpression) pSourceTable.getTTL()).getTTLValue(); // TODO } else { TableDescriptor tableDesc; try (Admin admin = cqsi.getAdmin()) { diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/DefaultPhoenixMultiViewListProvider.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/DefaultPhoenixMultiViewListProvider.java index 31ba3f19f4f..9c4f6093a8a 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/DefaultPhoenixMultiViewListProvider.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/DefaultPhoenixMultiViewListProvider.java @@ -139,7 +139,7 @@ private boolean isParentHasTTL(PhoenixConnection connection, PTable parentTable = connection.getTable(null, pTable.getParentName().toString()); System.out.println("Parent Table"); if (parentTable.getType() == PTableType.VIEW && - parentTable.getTTL() > 0) { + parentTable.getTTL() != null) { /* if the current view parent already has a TTL value, we want to skip the current view cleanup job because we want to run the cleanup job for at the GlobalView level instead of running multi-jobs at diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewTTLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewTTLIT.java index b206fba04ca..a9e37dfc98e 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewTTLIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewTTLIT.java @@ -18,13 +18,43 @@ package org.apache.phoenix.end2end; +import static java.util.Arrays.asList; +import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.COLUMN_TYPES; +import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.MAX_ROWS; +import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_COLUMNS; +import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_INCLUDE_COLUMNS; +import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_INDEX_COLUMNS; +import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_PK_COLUMNS; +import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_PK_TYPES; +import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_NOT_DEFINED; +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Types; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; + import org.apache.hadoop.hbase.CompareOperator; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.client.Admin; -import org.apache.hadoop.hbase.client.CompactionState; import org.apache.hadoop.hbase.client.ConnectionFactory; -import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.ResultScanner; import org.apache.hadoop.hbase.client.Scan; @@ -38,8 +68,6 @@ import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; import org.apache.phoenix.query.PhoenixTestBuilder; -import org.apache.phoenix.query.QueryConstants; -import org.apache.phoenix.query.QueryServices; import org.apache.phoenix.query.PhoenixTestBuilder.BasicDataReader; import org.apache.phoenix.query.PhoenixTestBuilder.BasicDataWriter; import org.apache.phoenix.query.PhoenixTestBuilder.DataReader; @@ -54,11 +82,12 @@ import org.apache.phoenix.query.PhoenixTestBuilder.SchemaBuilder.TableOptions; import org.apache.phoenix.query.PhoenixTestBuilder.SchemaBuilder.TenantViewIndexOptions; import org.apache.phoenix.query.PhoenixTestBuilder.SchemaBuilder.TenantViewOptions; - +import org.apache.phoenix.query.QueryConstants; +import org.apache.phoenix.query.QueryServices; import org.apache.phoenix.schema.PName; import org.apache.phoenix.schema.PTable; - import org.apache.phoenix.schema.PTableKey; +import org.apache.phoenix.schema.TTLExpression; import org.apache.phoenix.schema.types.PDataType; import org.apache.phoenix.thirdparty.com.google.common.base.Joiner; import org.apache.phoenix.thirdparty.com.google.common.collect.Lists; @@ -70,39 +99,13 @@ import org.apache.phoenix.util.TestUtil; import org.junit.After; import org.junit.Before; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.PreparedStatement; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.sql.Types; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Random; -import java.util.Set; - -import static java.util.Arrays.asList; -import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.COLUMN_TYPES; -import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.MAX_ROWS; -import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_COLUMNS; -import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_INCLUDE_COLUMNS; -import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_INDEX_COLUMNS; -import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_PK_COLUMNS; -import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_PK_TYPES; -import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; +@RunWith(Parameterized.class) public abstract class BaseViewTTLIT extends ParallelStatsDisabledIT { static final Logger LOGGER = LoggerFactory.getLogger(ViewTTLIT.class); static final int VIEW_TTL_10_SECS = 10; @@ -112,7 +115,7 @@ public abstract class BaseViewTTLIT extends ParallelStatsDisabledIT { static final String ID_FMT = "00A0y000%07d"; static final String ZID_FMT = "00B0y000%07d"; static final String ALTER_TTL_SQL - = "ALTER VIEW \"%s\".\"%s\" set TTL=%s"; + = "ALTER VIEW \"%s\".\"%s\" set TTL='%s'"; static final String ALTER_SQL_WITH_NO_TTL = "ALTER VIEW \"%s\".\"%s\" ADD IF NOT EXISTS %s CHAR(10)"; @@ -134,10 +137,23 @@ public abstract class BaseViewTTLIT extends ParallelStatsDisabledIT { ManualEnvironmentEdge injectEdge; + private boolean useCondExpression; + + public BaseViewTTLIT(boolean useCondExpression) { + this.useCondExpression = useCondExpression; + } + protected static void setUpTestDriver(ReadOnlyProps props) throws Exception { setUpTestDriver(props, props); } + @Parameterized.Parameters(name = "useCondExpression={0}") + public static synchronized Collection data() { + return Arrays.asList(new Boolean[][]{ + {false}, {true} + }); + } + @Before public void beforeTest(){ EnvironmentEdgeManager.reset(); @@ -164,6 +180,17 @@ public void testResetServerCache() { } } + /** + * + * @param ttl in seconds + * @return + */ + protected String getTTLExpression(int ttl) { + return useCondExpression ? String.format( + "TO_NUMBER(CURRENT_TIME()) - TO_NUMBER(PHOENIX_ROW_TIMESTAMP()) >= %d", ttl*1000) + : String.valueOf(ttl); + } + private void clearCache(boolean globalFixNeeded, boolean tenantFixNeeded, List allTenants) throws SQLException { @@ -249,7 +276,7 @@ void assertViewHeaderRowsHavePhoenixTTLRelatedCells(String schemaName, } void assertSyscatHavePhoenixTTLRelatedColumns(String tenantId, String schemaName, - String tableName, String tableType, long ttlValueExpected) throws SQLException { + String tableName, String tableType, String ttlExpected) throws SQLException { try (Connection connection = DriverManager.getConnection(getUrl())) { Statement stmt = connection.createStatement(); @@ -261,10 +288,11 @@ void assertSyscatHavePhoenixTTLRelatedColumns(String tenantId, String schemaName stmt.execute(sql); ResultSet rs = stmt.getResultSet(); String ttlStr = rs.next() ? rs.getString(1) : null; - long actualTTLValueReturned = ttlStr != null ? Integer.valueOf(ttlStr): 0; - + TTLExpression actual = ttlStr != null ? + TTLExpression.create(ttlStr): TTL_EXPRESSION_NOT_DEFINED; + TTLExpression expected = TTLExpression.create(ttlExpected); assertEquals(String.format("Expected rows do not match for schema = %s, table = %s", - schemaName, tableName), ttlValueExpected, actualTTLValueReturned); + schemaName, tableName), expected, actual); } } @@ -289,7 +317,8 @@ protected SchemaBuilder createLevel2TenantViewWithGlobalLevelTTL( GlobalViewOptions globalViewOptions = GlobalViewOptions.withDefaults(); // View TTL is set to 300s => 300000 ms - globalViewOptions.setTableProps("TTL=300"); + int viewTTL = 300; + globalViewOptions.setTableProps(String.format("TTL='%s'", getTTLExpression(viewTTL))); GlobalViewIndexOptions globalViewIndexOptions = GlobalViewIndexOptions.withDefaults(); @@ -341,7 +370,9 @@ protected SchemaBuilder createLevel2TenantViewWithTenantLevelTTL( TenantViewOptions tenantViewWithOverrideOptions = TenantViewOptions.withDefaults(); // View TTL is set to 300s => 300000 ms - tenantViewWithOverrideOptions.setTableProps("TTL=300"); + int viewTTL = 300; + tenantViewWithOverrideOptions.setTableProps(String.format("TTL='%s'", + getTTLExpression(viewTTL))); if (tenantViewOptions != null) { tenantViewWithOverrideOptions = tenantViewOptions; } @@ -402,7 +433,7 @@ void upsertDataAndRunValidations(long viewTTL, int numRowsToUpsert, void validateExpiredRowsAreNotReturnedUsingCounts(long viewTTL, DataReader dataReader, SchemaBuilder schemaBuilder) throws IOException, SQLException { - + EnvironmentEdgeManager.reset(); String tenantConnectUrl = getUrl() + ';' + TENANT_ID_ATTRIB + '=' + schemaBuilder.getDataOptions().getTenantId(); @@ -417,9 +448,11 @@ void validateExpiredRowsAreNotReturnedUsingCounts(long viewTTL, DataReader dataR } // Verify after TTL expiration - long scnTimestamp = EnvironmentEdgeManager.currentTimeMillis(); + long scnTimestamp = EnvironmentEdgeManager.currentTimeMillis() + (2 * viewTTL * 1000); Properties props = new Properties(); - props.setProperty("CurrentSCN", Long.toString(scnTimestamp + (2 * viewTTL * 1000))); + props.setProperty("CurrentSCN", Long.toString(scnTimestamp)); + injectEdge.setValue(scnTimestamp); + EnvironmentEdgeManager.injectEdge(injectEdge); try (Connection readConnection = DriverManager.getConnection(tenantConnectUrl, props)) { dataReader.setConnection(readConnection); @@ -429,12 +462,13 @@ void validateExpiredRowsAreNotReturnedUsingCounts(long viewTTL, DataReader dataR assertEquals("Expired rows should not be fetched", 0, fetchedData.rowKeySet().size()); } + EnvironmentEdgeManager.reset(); } void validateExpiredRowsAreNotReturnedUsingData(long viewTTL, org.apache.phoenix.thirdparty.com.google.common.collect.Table upsertedData, DataReader dataReader, SchemaBuilder schemaBuilder) throws SQLException { - + EnvironmentEdgeManager.reset(); String tenantConnectUrl = getUrl() + ';' + TENANT_ID_ATTRIB + '=' + schemaBuilder.getDataOptions().getTenantId(); @@ -443,6 +477,8 @@ void validateExpiredRowsAreNotReturnedUsingData(long viewTTL, long scnTimestamp = EnvironmentEdgeManager.currentTimeMillis() + 1; props.setProperty("CurrentSCN", Long.toString(scnTimestamp)); props.setProperty(QueryServices.COLLECT_REQUEST_LEVEL_METRICS, String.valueOf(true)); + injectEdge.setValue(scnTimestamp); + EnvironmentEdgeManager.injectEdge(injectEdge); try (Connection readConnection = DriverManager.getConnection(tenantConnectUrl, props)) { dataReader.setConnection(readConnection); @@ -457,6 +493,7 @@ void validateExpiredRowsAreNotReturnedUsingData(long viewTTL, // Verify after TTL expiration props.setProperty("CurrentSCN", Long.toString(scnTimestamp + (2 * viewTTL * 1000))); + injectEdge.setValue(scnTimestamp + (2 * viewTTL * 1000)); try (Connection readConnection = DriverManager.getConnection(tenantConnectUrl, props)) { dataReader.setConnection(readConnection); @@ -466,12 +503,12 @@ void validateExpiredRowsAreNotReturnedUsingData(long viewTTL, assertNotNull("Fetched data should not be null", fetchedData); assertEquals("Expired rows should not be fetched", 0, fetchedData.rowKeySet().size()); } - + EnvironmentEdgeManager.reset(); } void validateRowsAreNotMaskedUsingCounts(long probeTimestamp, DataReader dataReader, SchemaBuilder schemaBuilder) throws SQLException { - + EnvironmentEdgeManager.reset(); String tenantConnectUrl = getUrl() + ';' + TENANT_ID_ATTRIB + '=' + schemaBuilder.getDataOptions() .getTenantId(); @@ -480,6 +517,8 @@ void validateRowsAreNotMaskedUsingCounts(long probeTimestamp, DataReader dataRea long scnTimestamp = EnvironmentEdgeManager.currentTimeMillis() + 1; Properties props = new Properties(); props.setProperty("CurrentSCN", Long.toString(scnTimestamp )); + injectEdge.setValue(scnTimestamp); + EnvironmentEdgeManager.injectEdge(injectEdge); try (Connection readConnection = DriverManager.getConnection(tenantConnectUrl, props)) { dataReader.setConnection(readConnection); @@ -493,6 +532,7 @@ void validateRowsAreNotMaskedUsingCounts(long probeTimestamp, DataReader dataRea // Verify rows exists (not masked) at probed timestamp props.setProperty("CurrentSCN", Long.toString(probeTimestamp)); + injectEdge.setValue(probeTimestamp); try (Connection readConnection = DriverManager.getConnection(tenantConnectUrl, props)) { dataReader.setConnection(readConnection); @@ -503,6 +543,7 @@ void validateRowsAreNotMaskedUsingCounts(long probeTimestamp, DataReader dataRea assertTrue("Rows should exists before ttl expiration (probe-timestamp)", fetchedData.rowKeySet().size() > 0); } + EnvironmentEdgeManager.reset(); } static void verifyRowsBeforeTTLExpiration( @@ -534,6 +575,8 @@ org.apache.phoenix.thirdparty.com.google.common.collect.Table getTableAndGlobalAndTenantColumnFamilyOptions() { testCaseWhenAllCFMatchAndSame.setTenantViewCFs(Lists.newArrayList("A", "A", "A")); testCases.add(testCaseWhenAllCFMatchAndSame); - OtherOptions - testCaseWhenAllCFMatch = new OtherOptions(); - testCaseWhenAllCFMatch.setTestName("testCaseWhenAllCFMatch"); - testCaseWhenAllCFMatch.setTableCFs(Lists.newArrayList(null, "A", "B")); - testCaseWhenAllCFMatch.setGlobalViewCFs(Lists.newArrayList(null, "A", "B")); - testCaseWhenAllCFMatch.setTenantViewCFs(Lists.newArrayList(null, "A", "B")); - testCases.add(testCaseWhenAllCFMatch); - - OtherOptions - testCaseWhenTableCFsAreDiff = new OtherOptions(); - testCaseWhenTableCFsAreDiff.setTestName("testCaseWhenTableCFsAreDiff"); - testCaseWhenTableCFsAreDiff.setTableCFs(Lists.newArrayList(null, "A", "B")); - testCaseWhenTableCFsAreDiff.setGlobalViewCFs(Lists.newArrayList("A", "A", "B")); - testCaseWhenTableCFsAreDiff.setTenantViewCFs(Lists.newArrayList("A", "A", "B")); - testCases.add(testCaseWhenTableCFsAreDiff); - - OtherOptions - testCaseWhenGlobalAndTenantCFsAreDiff = new OtherOptions(); - testCaseWhenGlobalAndTenantCFsAreDiff.setTestName("testCaseWhenGlobalAndTenantCFsAreDiff"); - testCaseWhenGlobalAndTenantCFsAreDiff.setTableCFs(Lists.newArrayList(null, "A", "B")); - testCaseWhenGlobalAndTenantCFsAreDiff.setGlobalViewCFs(Lists.newArrayList("A", "A", "A")); - testCaseWhenGlobalAndTenantCFsAreDiff.setTenantViewCFs(Lists.newArrayList("B", "B", "B")); - testCases.add(testCaseWhenGlobalAndTenantCFsAreDiff); + // conditional TTL expressions not supported for multiple column families + if (useCondExpression == false) { + OtherOptions + testCaseWhenAllCFMatch = new OtherOptions(); + testCaseWhenAllCFMatch.setTestName("testCaseWhenAllCFMatch"); + testCaseWhenAllCFMatch.setTableCFs(Lists.newArrayList(null, "A", "B")); + testCaseWhenAllCFMatch.setGlobalViewCFs(Lists.newArrayList(null, "A", "B")); + testCaseWhenAllCFMatch.setTenantViewCFs(Lists.newArrayList(null, "A", "B")); + testCases.add(testCaseWhenAllCFMatch); + + OtherOptions + testCaseWhenTableCFsAreDiff = new OtherOptions(); + testCaseWhenTableCFsAreDiff.setTestName("testCaseWhenTableCFsAreDiff"); + testCaseWhenTableCFsAreDiff.setTableCFs(Lists.newArrayList(null, "A", "B")); + testCaseWhenTableCFsAreDiff.setGlobalViewCFs(Lists.newArrayList("A", "A", "B")); + testCaseWhenTableCFsAreDiff.setTenantViewCFs(Lists.newArrayList("A", "A", "B")); + testCases.add(testCaseWhenTableCFsAreDiff); + + OtherOptions + testCaseWhenGlobalAndTenantCFsAreDiff = new OtherOptions(); + testCaseWhenGlobalAndTenantCFsAreDiff.setTestName("testCaseWhenGlobalAndTenantCFsAreDiff"); + testCaseWhenGlobalAndTenantCFsAreDiff.setTableCFs(Lists.newArrayList(null, "A", "B")); + testCaseWhenGlobalAndTenantCFsAreDiff.setGlobalViewCFs(Lists.newArrayList("A", "A", "A")); + testCaseWhenGlobalAndTenantCFsAreDiff.setTenantViewCFs(Lists.newArrayList("B", "B", "B")); + testCases.add(testCaseWhenGlobalAndTenantCFsAreDiff); + } return testCases; } @@ -713,7 +759,8 @@ protected void testMajorCompactWithSaltedIndexedTenantView() throws Exception { if (isMultiTenant) { TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults(); - tenantViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + tenantViewOptions.setTableProps(String.format("TTL='%s'", + getTTLExpression(viewTTL))); schemaBuilder .withTableOptions(tableOptions) .withTenantViewOptions(tenantViewOptions) @@ -729,7 +776,8 @@ protected void testMajorCompactWithSaltedIndexedTenantView() throws Exception { globalViewOptions.setGlobalViewColumnTypes(Lists.newArrayList(COLUMN_TYPES)); globalViewOptions.setGlobalViewPKColumns(Lists.newArrayList(TENANT_VIEW_PK_COLUMNS)); globalViewOptions.setGlobalViewPKColumnTypes(Lists.newArrayList(TENANT_VIEW_PK_TYPES)); - globalViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + globalViewOptions.setTableProps(String.format("TTL='%s'", + getTTLExpression(viewTTL))); GlobalViewIndexOptions globalViewIndexOptions = new GlobalViewIndexOptions(); @@ -863,7 +911,7 @@ protected void testMajorCompactWithOnlyTenantView() throws Exception { TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults(); - tenantViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + tenantViewOptions.setTableProps(String.format("TTL='%s'", getTTLExpression(viewTTL))); // Define the test schema. final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl()); @@ -950,7 +998,7 @@ protected void testMajorCompactFromMultipleGlobalIndexes() throws Exception { GlobalViewOptions globalViewOptions = GlobalViewOptions.withDefaults(); - globalViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + globalViewOptions.setTableProps(String.format("TTL='%s'", getTTLExpression(viewTTL))); GlobalViewIndexOptions globalViewIndexOptions = GlobalViewIndexOptions.withDefaults(); @@ -1125,7 +1173,7 @@ protected void testMajorCompactFromMultipleTenantIndexes() throws Exception { tenantViewOptions = new TenantViewOptions(); tenantViewOptions.setTenantViewColumns(Lists.newArrayList(TENANT_VIEW_COLUMNS)); tenantViewOptions.setTenantViewColumnTypes(Lists.newArrayList(COLUMN_TYPES)); - tenantViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + tenantViewOptions.setTableProps(String.format("TTL='%s'", getTTLExpression(viewTTL))); TenantViewIndexOptions tenantViewIndexOptions = TenantViewIndexOptions.withDefaults(); @@ -1298,7 +1346,8 @@ protected void testMajorCompactWithSimpleIndexedBaseTables() throws Exception { int viewTTL = VIEW_TTL_10_SECS; TableOptions tableOptions = TableOptions.withDefaults(); - String tableProps = "COLUMN_ENCODED_BYTES=0,DEFAULT_COLUMN_FAMILY='0',TTL=10"; + String tableProps = String.format("COLUMN_ENCODED_BYTES=0,DEFAULT_COLUMN_FAMILY='0'," + + "TTL='%s'", getTTLExpression(viewTTL)); tableOptions.setTableProps(tableProps); tableOptions.getTablePKColumns().add("ZID"); tableOptions.getTablePKColumnTypes().add("CHAR(15)"); @@ -1484,7 +1533,8 @@ protected void testMajorCompactWithSaltedIndexedBaseTables() throws Exception { int viewTTL = VIEW_TTL_10_SECS; TableOptions tableOptions = TableOptions.withDefaults(); - String tableProps = "COLUMN_ENCODED_BYTES=0,DEFAULT_COLUMN_FAMILY='0',TTL=10"; + String tableProps = String.format("COLUMN_ENCODED_BYTES=0,DEFAULT_COLUMN_FAMILY='0'," + + "TTL='%s'", getTTLExpression(viewTTL)); tableOptions.setTableProps(tableProps); tableOptions.setSaltBuckets(1); tableOptions.getTablePKColumns().add("ZID"); @@ -1680,7 +1730,7 @@ protected void testMajorCompactWithVariousViewsAndOptions() throws Exception { GlobalViewOptions globalViewOptions = GlobalViewOptions.withDefaults(); - globalViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + globalViewOptions.setTableProps(String.format("TTL='%s'", getTTLExpression(viewTTL))); GlobalViewIndexOptions globalViewIndexOptions = GlobalViewIndexOptions.withDefaults(); @@ -1858,7 +1908,8 @@ protected void testMajorCompactWhenTTLSetForSomeTenants() throws Exception { for (int tenant : tenantSet) { // Set TTL only when tenant in hasTTLSet if (hasTTLSet.contains(tenant)) { - tenantViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + tenantViewOptions.setTableProps(String.format("TTL='%s'", + getTTLExpression(viewTTL))); } // build schema for tenant schemaBuilder.getDataOptions().setTenantId(null); @@ -2129,9 +2180,11 @@ protected void testMajorCompactWithVariousTenantIdTypesAndRegions(PDataType tena // Set TTL only when hasGlobalTTLSet OR hasTenantTTLSet // View TTL is set to 10s => 10000 ms if (hasGlobalTTLSet.contains(globalView)) { - globalViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + globalViewOptions.setTableProps(String.format("TTL='%s'", + getTTLExpression(viewTTL))); } else if (hasTenantTTLSet.contains(tenant)) { - tenantViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + tenantViewOptions.setTableProps(String.format("TTL='%s'", + getTTLExpression(viewTTL))); } if (schemaBuilder.getDataOptions() != null) { @@ -2139,13 +2192,18 @@ protected void testMajorCompactWithVariousTenantIdTypesAndRegions(PDataType tena switch (tenantType.getSqlType()) { case Types.VARCHAR: case Types.CHAR: + // ensure a new tenantid is generated for every iteration of the test schemaBuilder.getDataOptions().setTenantId(dataOptions.getNextTenantId()); break; case Types.INTEGER: - schemaBuilder.getDataOptions().setTenantId(Integer.toString(tenant*100000)); + // ensure a new tenantid is generated for every iteration of the test + schemaBuilder.getDataOptions().setTenantId(Integer.toString( + dataOptions.getNextTenantNumber()*100000)); break; case Types.BIGINT: - schemaBuilder.getDataOptions().setTenantId(Long.toString(tenant*10000000000l)); + // ensure a new tenantid is generated for every iteration of the test + schemaBuilder.getDataOptions().setTenantId(Long.toString( + dataOptions.getNextTenantNumber()*10000000000l)); break; } } @@ -2316,10 +2374,7 @@ protected void testMajorCompactWithVariousTenantIdTypesAndRegions(PDataType tena validateRowsAreNotMaskedUsingCounts(scnTimestamp, dataReader, schemaBuilder); } } - } - - } PTable table = schemaBuilder.getBaseTable(); @@ -2387,8 +2442,8 @@ protected void testTenantViewsWIthOverlappingRowPrefixes() throws Exception { dataOptions.setGlobalViewName(dataOptions.getGlobalViewName() + "_1"); dataOptions.setTenantViewName("Z01"); dataOptions.setKeyPrefix("00D0t0002000001"); - // View TTL is set to 300s => 300000 ms - globalViewOptions.setTableProps("TTL=10"); + globalViewOptions.setTableProps( + String.format("TTL='%s'", getTTLExpression(viewTTL))); } else { dataOptions.setGlobalViewName(dataOptions.getGlobalViewName() + "_2"); dataOptions.setTenantViewName("Z02"); @@ -2556,9 +2611,11 @@ protected void testMajorCompactWithGlobalAndTenantViewHierarchy() throws Excepti // Set TTL only when hasGlobalTTLSet OR hasTenantTTLSet // View TTL is set to 10s => 10000 ms if (hasGlobalTTLSet.contains(globalView)) { - globalViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + globalViewOptions.setTableProps( + String.format("TTL='%s'", getTTLExpression(viewTTL))); } else if (hasTenantTTLSet.contains(tenant)) { - tenantViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + tenantViewOptions.setTableProps( + String.format("TTL='%s'", getTTLExpression(viewTTL))); } // build schema for tenant diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CDCBaseIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CDCBaseIT.java index 2d9aa6738a5..46332469c8f 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CDCBaseIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CDCBaseIT.java @@ -33,6 +33,7 @@ import org.apache.phoenix.hbase.index.IndexRegionObserver; import org.apache.phoenix.query.QueryServicesOptions; import org.apache.phoenix.schema.PTable; +import org.apache.phoenix.schema.TTLExpression; import org.apache.phoenix.schema.TableProperty; import org.apache.phoenix.schema.types.PBinaryBase; import org.apache.phoenix.schema.types.PChar; @@ -219,6 +220,8 @@ protected void assertPTable(String cdcName, Set expInclud CDCUtil.getCDCIndexName(cdcName)); assertEquals(cdcTable.getPhysicalName().getString(), tableName == datatableName ? indexFullName : getViewIndexPhysicalName(datatableName)); + PTable cdcIndexTable = PhoenixRuntime.getTable(conn, indexFullName); + assertEquals(cdcIndexTable.getTTL(), TTLExpression.TTL_EXPRESSION_FORVER); } protected void assertSaltBuckets(Connection conn, String tableName, Integer nbuckets) diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CDCDefinitionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CDCDefinitionIT.java index 4a7dfa0e8a7..2847237c915 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CDCDefinitionIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CDCDefinitionIT.java @@ -66,7 +66,7 @@ public void testCreate() throws Exception { String datatableName = tableName; conn.createStatement().execute( "CREATE TABLE " + tableName + " ( k INTEGER PRIMARY KEY," + " v1 INTEGER," - + " v2 DATE)"); + + " v2 DATE) TTL=100"); if (forView) { String viewName = generateUniqueName(); conn.createStatement().execute( 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 93a63dd2335..7ca41f5338f 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 @@ -62,7 +62,6 @@ import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.util.Bytes; -import org.apache.phoenix.coprocessor.BaseScannerRegionObserver; import org.apache.phoenix.exception.SQLExceptionCode; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; @@ -81,13 +80,13 @@ import org.apache.phoenix.schema.PTableKey; import org.apache.phoenix.schema.PTableType; import org.apache.phoenix.schema.SchemaNotFoundException; +import org.apache.phoenix.schema.LiteralTTLExpression; import org.apache.phoenix.schema.TableAlreadyExistsException; import org.apache.phoenix.schema.TableNotFoundException; import org.apache.phoenix.util.ByteUtil; import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; import org.apache.phoenix.util.EnvironmentEdgeManager; import org.apache.phoenix.util.IndexUtil; -import org.apache.phoenix.util.PhoenixRuntime; import org.apache.phoenix.util.PropertiesUtil; import org.apache.phoenix.util.QueryUtil; import org.apache.phoenix.util.ReadOnlyProps; @@ -479,7 +478,7 @@ public void testCreateTableColumnFamilyHBaseAttribs1() throws Exception { assertEquals(1, columnFamilies.length); assertEquals(86400, columnFamilies[0].getTimeToLive()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable - assertEquals(86400, conn.unwrap(PhoenixConnection.class).getTable( + assertEquals(new LiteralTTLExpression(86400), conn.unwrap(PhoenixConnection.class).getTable( new PTableKey(null, tableName)).getTTL()); } @@ -538,7 +537,7 @@ public void testCreateTableColumnFamilyHBaseAttribs2() throws Exception { assertEquals(86400, columnFamilies[1].getTimeToLive()); assertEquals("C", columnFamilies[1].getNameAsString()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable - assertEquals(86400, conn.unwrap(PhoenixConnection.class).getTable( + assertEquals(new LiteralTTLExpression(86400), conn.unwrap(PhoenixConnection.class).getTable( new PTableKey(null, tableName)).getTTL()); } @@ -567,7 +566,7 @@ public void testCreateTableColumnFamilyHBaseAttribs3() throws Exception { assertEquals("B", columnFamilies[1].getNameAsString()); assertEquals(86400, columnFamilies[1].getTimeToLive()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable - assertEquals(86400, conn.unwrap(PhoenixConnection.class).getTable( + assertEquals(new LiteralTTLExpression(86400), conn.unwrap(PhoenixConnection.class).getTable( new PTableKey(null, tableName)).getTTL()); } @@ -648,7 +647,7 @@ public void testCreateTableColumnFamilyHBaseAttribs6() throws Exception { assertEquals("a", columnFamilies[0].getNameAsString()); assertEquals(10000, columnFamilies[0].getTimeToLive()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable - assertEquals(10000, conn.unwrap(PhoenixConnection.class).getTable( + assertEquals(new LiteralTTLExpression(10000), conn.unwrap(PhoenixConnection.class).getTable( new PTableKey(null, tableName)).getTTL()); } @@ -673,7 +672,7 @@ public void testCreateTableColumnFamilyHBaseAttribs7() throws Exception { assertEquals("a", columnFamilies[0].getNameAsString()); assertEquals(10000, columnFamilies[0].getTimeToLive()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable - assertEquals(10000, conn.unwrap(PhoenixConnection.class).getTable( + assertEquals(new LiteralTTLExpression(10000), conn.unwrap(PhoenixConnection.class).getTable( new PTableKey(null, tableName)).getTTL()); } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexRepairRegionScannerIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexRepairRegionScannerIT.java index dc37668731a..1a0b6ec57d1 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexRepairRegionScannerIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexRepairRegionScannerIT.java @@ -219,13 +219,6 @@ private void truncateIndexToolTables() throws IOException { getUtility().getAdmin().truncateTable(TableName.valueOf(RESULT_TABLE_NAME), true); } - private void dumpIndexToolMRJobCounters(IndexTool indexTool) throws IOException { - CounterGroup mrJobCounters = IndexToolIT.getMRJobCounters(indexTool); - for (Counter counter : mrJobCounters) { - LOGGER.info(String.format("%s=%d", counter.getName(), counter.getValue())); - } - } - private void assertExtraCounters(IndexTool indexTool, long extraVerified, long extraUnverified, boolean isBefore) throws IOException { CounterGroup mrJobCounters = IndexToolIT.getMRJobCounters(indexTool); @@ -260,7 +253,7 @@ private void assertDisableLogging(Connection conn, int expectedExtraRows, int ex try { assertExtraCounters(tool, expectedExtraRows, 0, true); } catch (AssertionError e) { - dumpIndexToolMRJobCounters(tool); + IndexToolIT.dumpMRJobCounters(tool); throw e; } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java index 3291457164f..7214ac69915 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java @@ -69,6 +69,7 @@ import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.CounterGroup; import org.apache.hadoop.mapreduce.Job; import org.apache.phoenix.compile.ExplainPlan; @@ -838,6 +839,17 @@ public static CounterGroup getMRJobCounters(IndexTool indexTool) throws IOExcept return indexTool.getJob().getCounters().getGroup(PhoenixIndexToolJobCounters.class.getName()); } + public static void dumpMRJobCounters(IndexTool indexTool) throws IOException { + CounterGroup mrJobCounters = IndexToolIT.getMRJobCounters(indexTool); + dumpMRJobCounters(mrJobCounters); + } + + public static void dumpMRJobCounters(CounterGroup mrJobCounters) { + for (Counter cntr : mrJobCounters) { + LOGGER.info(String.format("%s=%d", cntr.getName(), cntr.getValue())); + } + } + private static List getArgList (boolean useSnapshot, String schemaName, String dataTable, String indxTable, String tenantId, IndexTool.IndexVerifyType verifyType, Long startTime, diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SetPropertyIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SetPropertyIT.java index 2a79254b07d..0a487a5df55 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SetPropertyIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SetPropertyIT.java @@ -38,13 +38,13 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; -import org.apache.phoenix.coprocessor.BaseScannerRegionObserver; import org.apache.phoenix.exception.SQLExceptionCode; import org.apache.phoenix.jdbc.PhoenixConnection; -import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; import org.apache.phoenix.query.QueryServices; import org.apache.phoenix.schema.PTable; import org.apache.phoenix.schema.PTableKey; +import org.apache.phoenix.schema.TTLExpression; +import org.apache.phoenix.schema.LiteralTTLExpression; import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; import org.apache.phoenix.util.PropertiesUtil; import org.apache.phoenix.util.ReadOnlyProps; @@ -430,7 +430,7 @@ public void testSettingPropertiesWhenTableHasDefaultColFamilySpecified() throws assertEquals(ColumnFamilyDescriptorBuilder.DEFAULT_BLOCKSIZE, columnFamilies[1].getBlocksize()); assertEquals(Boolean.toString(false), tableDesc.getValue(TableDescriptorBuilder.COMPACTION_ENABLED)); //Check if Alter Table TTL also changes TTL stored in SYSCAT with phoenix.table.ttl disabled - assertEquals(1000, conn.unwrap(PhoenixConnection.class).getTable( + assertEquals(new LiteralTTLExpression(1000), conn.unwrap(PhoenixConnection.class).getTable( new PTableKey(null, dataTableFullName)).getTTL()); } } @@ -793,7 +793,7 @@ public void testSetTTLForTableWithOnlyPKCols() throws Exception { assertEquals("XYZ", columnFamilies[0].getNameAsString()); assertEquals(86400, columnFamilies[0].getTimeToLive()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable - assertEquals(86400, conn.unwrap(PhoenixConnection.class).getTable( + assertEquals(new LiteralTTLExpression(86400), conn.unwrap(PhoenixConnection.class).getTable( new PTableKey(null, dataTableFullName)).getTTL()); } ddl = "ALTER TABLE " + dataTableFullName + " SET TTL=30"; @@ -806,7 +806,7 @@ public void testSetTTLForTableWithOnlyPKCols() throws Exception { assertEquals(30, columnFamilies[0].getTimeToLive()); assertEquals("XYZ", columnFamilies[0].getNameAsString()); //Check if Alter Table TTL also changes TTL stored in SYSCAT with phoenix.table.ttl disabled - assertEquals(30, conn.unwrap(PhoenixConnection.class).getTable( + assertEquals(new LiteralTTLExpression(30), conn.unwrap(PhoenixConnection.class).getTable( new PTableKey(null, dataTableFullName)).getTTL()); } } finally { @@ -834,7 +834,7 @@ public void testSetTTLForTableWithForeverAndNoneValue() throws Exception { assertEquals("XYZ", columnFamilies[0].getNameAsString()); assertEquals(HConstants.FOREVER, columnFamilies[0].getTimeToLive()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable - assertEquals(PhoenixDatabaseMetaData.TTL_NOT_DEFINED, conn.unwrap(PhoenixConnection.class).getTable( + assertEquals(TTLExpression.TTL_EXPRESSION_NOT_DEFINED, conn.unwrap(PhoenixConnection.class).getTable( new PTableKey(null, dataTableFullName)).getTTL()); } ddl = "ALTER TABLE " + dataTableFullName + " SET TTL=FOREVER"; @@ -847,7 +847,7 @@ public void testSetTTLForTableWithForeverAndNoneValue() throws Exception { assertEquals(HConstants.FOREVER, columnFamilies[0].getTimeToLive()); assertEquals("XYZ", columnFamilies[0].getNameAsString()); //Check if Alter Table TTL also changes TTL stored in SYSCAT with phoenix.table.ttl disabled - assertEquals(HConstants.FOREVER, conn.unwrap(PhoenixConnection.class).getTable( + assertEquals(TTLExpression.TTL_EXPRESSION_FORVER, conn.unwrap(PhoenixConnection.class).getTable( new PTableKey(null, dataTableFullName)).getTTL()); } } finally { @@ -996,7 +996,7 @@ public void testTTLAssignmentForNewEmptyCF() throws Exception { assertEquals(false, columnFamilies[1].isInMemory()); assertEquals(86400, columnFamilies[1].getTimeToLive()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable - assertEquals(86400, conn.unwrap(PhoenixConnection.class).getTable( + assertEquals(new LiteralTTLExpression(86400), conn.unwrap(PhoenixConnection.class).getTable( new PTableKey(null, dataTableFullName)).getTTL()); } @@ -1014,7 +1014,7 @@ public void testTTLAssignmentForNewEmptyCF() throws Exception { assertEquals(false, columnFamilies[1].isInMemory()); assertEquals(1000, columnFamilies[1].getTimeToLive()); //Check if Alter Table TTL also changes TTL stored in SYSCAT with phoenix.table.ttl disabled - assertEquals(1000, conn.unwrap(PhoenixConnection.class).getTable( + assertEquals(new LiteralTTLExpression(1000), conn.unwrap(PhoenixConnection.class).getTable( new PTableKey(null, dataTableFullName)).getTTL()); } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLAsPhoenixTTLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLAsPhoenixTTLIT.java index ac835947e0f..56ce1b80799 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLAsPhoenixTTLIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLAsPhoenixTTLIT.java @@ -24,38 +24,89 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; import org.apache.hadoop.hbase.util.Bytes; import org.apache.phoenix.jdbc.PhoenixConnection; -import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; +import org.apache.phoenix.schema.ColumnNotFoundException; +import org.apache.phoenix.schema.ColumnFamilyNotFoundException; import org.apache.phoenix.schema.PName; import org.apache.phoenix.schema.PTable; import org.apache.phoenix.schema.PTableKey; +import org.apache.phoenix.schema.TTLExpression; import org.apache.phoenix.util.ByteUtil; import org.apache.phoenix.util.PhoenixRuntime; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.sql.Connection; import java.sql.DriverManager; import java.sql.SQLException; -import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Properties; import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_SET_OR_ALTER_PROPERTY_FOR_INDEX; import static org.apache.phoenix.exception.SQLExceptionCode.TTL_ALREADY_DEFINED_IN_HIERARCHY; import static org.apache.phoenix.exception.SQLExceptionCode.TTL_SUPPORTED_FOR_TABLES_AND_VIEWS_ONLY; -import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TTL_NOT_DEFINED; +import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_NOT_DEFINED; 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.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @Category(ParallelStatsDisabledTest.class) +@RunWith(Parameterized.class) public class TTLAsPhoenixTTLIT extends ParallelStatsDisabledIT{ + private static final String DDL_TEMPLATE = "CREATE TABLE IF NOT EXISTS %s " + + "(" + + " ID INTEGER NOT NULL," + + " COL1 INTEGER NOT NULL," + + " COL2 bigint NOT NULL," + + " CREATED_DATE DATE," + + " CREATION_TIME TIME," + + " CONSTRAINT NAME_PK PRIMARY KEY (ID, COL1, COL2)" + + ")"; + + private static final String DEFAULT_DDL_OPTIONS = "MULTI_TENANT=true"; + private static final int DEFAULT_TTL_FOR_TEST = 86400; private static final int DEFAULT_TTL_FOR_CHILD = 10000; private static final int DEFAULT_TTL_FOR_ALTER = 7000; + private static final String DEFAULT_TTL_EXPRESSION = "CURRENT_TIME() - cREATION_TIME > 500"; // case-insensitive comparison + private static final String DEFAULT_TTL_EXPRESSION_FOR_ALTER = + "CURRENT_TIME() - PHOENIX_ROW_TIMESTAMP() > 100"; + + private boolean useExpression; + private TTLExpression defaultTTL; + private String defaultTTLDDLOption; + private TTLExpression alterTTL; + private String alterTTLDDLOption; + + public TTLAsPhoenixTTLIT(boolean useExpression) { + this.useExpression = useExpression; + this.defaultTTL = useExpression ? + TTLExpression.create(DEFAULT_TTL_EXPRESSION) : + TTLExpression.create(DEFAULT_TTL_FOR_TEST); + this.defaultTTLDDLOption = useExpression ? + String.format("'%s'", DEFAULT_TTL_EXPRESSION) : + String.valueOf(DEFAULT_TTL_FOR_TEST); + this.alterTTL = useExpression ? + TTLExpression.create(DEFAULT_TTL_EXPRESSION_FOR_ALTER) : + TTLExpression.create(DEFAULT_TTL_FOR_ALTER); + this.alterTTLDDLOption = useExpression ? + String.format("'%s'", DEFAULT_TTL_EXPRESSION_FOR_ALTER) : + String.valueOf(DEFAULT_TTL_FOR_ALTER); + } + + @Parameterized.Parameters(name = "useExpression={0}") + public static synchronized Collection data() { + return Arrays.asList(new Boolean[][]{ + {false}, {true} + }); + } /** * test TTL is being set as PhoenixTTL when PhoenixTTL is enabled. @@ -65,13 +116,42 @@ public void testCreateTableWithTTL() throws Exception { try (Connection conn = DriverManager.getConnection(getUrl());) { PTable table = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, createTableWithOrWithOutTTLAsItsProperty(conn, true))); - assertEquals("TTL is not set correctly at Phoenix level", DEFAULT_TTL_FOR_TEST, - table.getTTL()); + assertTTLValue(table, defaultTTL); assertTrue("RowKeyMatcher should be Null", (Bytes.compareTo(HConstants.EMPTY_BYTE_ARRAY, table.getRowKeyMatcher()) == 0)); } } + @Test + public void testCreateTableWithNoTTL() throws Exception { + try (Connection conn = DriverManager.getConnection(getUrl());) { + PTable table = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, + createTableWithOrWithOutTTLAsItsProperty(conn, false))); + assertTTLValue(table, TTL_EXPRESSION_NOT_DEFINED); + assertFalse(table.hasConditionalTTL()); + } + } + + @Test + public void testSwitchingTTLFromCondToValue() throws Exception { + try (Connection conn = DriverManager.getConnection(getUrl())) { + String tableName = createTableWithOrWithOutTTLAsItsProperty(conn, true); + PTable table = conn.unwrap(PhoenixConnection.class). + getTable(new PTableKey(null, tableName)); + assertTTLValue(table, defaultTTL); + assertTrue(table.hasConditionalTTL() == useExpression); + // Switch from cond ttl to value or vice versa + String alterTTL = useExpression ? String.valueOf(DEFAULT_TTL_FOR_ALTER) : + String.format("'%s'", DEFAULT_TTL_EXPRESSION_FOR_ALTER); + String alterDDL = "ALTER TABLE " + tableName + " SET TTL = " + alterTTL; + conn.createStatement().execute(alterDDL); + TTLExpression expected = useExpression ? + TTLExpression.create(DEFAULT_TTL_FOR_ALTER) : + TTLExpression.create(DEFAULT_TTL_EXPRESSION_FOR_ALTER); + assertTTLValue(conn.unwrap(PhoenixConnection.class), expected, tableName); + } + } + /** * Tests that when: 1) DDL has both pk as well as key value columns 2) Key value columns have * * both default and explicit column family names 3) TTL specifier doesn't have column family @@ -79,70 +159,74 @@ public void testCreateTableWithTTL() throws Exception { */ @Test public void testCreateTableWithTTLWithDifferentColumnFamilies() throws Exception { + if (useExpression) { + return; // multiple column families not supported for Conditional TTL + } String tableName = generateUniqueName(); String ddl = "create table IF NOT EXISTS " + tableName + " (" + " id char(1) NOT NULL," + " col1 integer NOT NULL," + " b.col2 bigint," + " col3 bigint, " + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1)" - + " ) TTL=86400"; + + " ) TTL=" + DEFAULT_TTL_FOR_TEST; Connection conn = DriverManager.getConnection(getUrl()); conn.createStatement().execute(ddl); - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), DEFAULT_TTL_FOR_TEST, tableName); + TTLExpression expected = TTLExpression.create(DEFAULT_TTL_FOR_TEST); + assertTTLValue(conn.unwrap(PhoenixConnection.class), expected, tableName); //Setting TTL should not be stored as CF Descriptor properties when //phoenix.table.ttl.enabled is true Admin admin = driver.getConnectionQueryServices(getUrl(), new Properties()).getAdmin(); ColumnFamilyDescriptor[] columnFamilies = admin.getDescriptor(TableName.valueOf(tableName)).getColumnFamilies(); assertEquals(ColumnFamilyDescriptorBuilder.DEFAULT_TTL, columnFamilies[0].getTimeToLive()); - } @Test public void testCreateAndAlterTableDDLWithForeverAndNoneTTLValues() throws Exception { + if (useExpression) { + return; + } String tableName = generateUniqueName(); String ddl = "create table IF NOT EXISTS " + tableName + " (" + " id char(1) NOT NULL," + " col1 integer NOT NULL," + " b.col2 bigint," + " col3 bigint, " + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1)" + " ) TTL=FOREVER"; - Connection conn = DriverManager.getConnection(getUrl()); - conn.createStatement().execute(ddl); - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), - HConstants.FOREVER, tableName); - - ddl = "ALTER TABLE " + tableName - + " SET TTL=NONE"; - conn.createStatement().execute(ddl); - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), - PhoenixDatabaseMetaData.TTL_NOT_DEFINED, tableName); - //Setting TTL should not be stored as CF Descriptor properties when - //phoenix.table.ttl.enabled is true - Admin admin = driver.getConnectionQueryServices(getUrl(), new Properties()).getAdmin(); - ColumnFamilyDescriptor[] columnFamilies = - admin.getDescriptor(TableName.valueOf(tableName)).getColumnFamilies(); - assertEquals(ColumnFamilyDescriptorBuilder.DEFAULT_TTL, columnFamilies[0].getTimeToLive()); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + assertTTLValue(conn.unwrap(PhoenixConnection.class), + TTLExpression.TTL_EXPRESSION_FORVER, tableName); - tableName = generateUniqueName(); - ddl = - "create table IF NOT EXISTS " + tableName + " (" + " id char(1) NOT NULL," - + " col1 integer NOT NULL," + " b.col2 bigint," + " col3 bigint, " - + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1)" - + " ) TTL=NONE"; - conn.createStatement().execute(ddl); - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), - PhoenixDatabaseMetaData.TTL_NOT_DEFINED, tableName); + ddl = "ALTER TABLE " + tableName + " SET TTL=NONE"; + conn.createStatement().execute(ddl); + assertTTLValue(conn.unwrap(PhoenixConnection.class), + TTL_EXPRESSION_NOT_DEFINED, tableName); + //Setting TTL should not be stored as CF Descriptor properties when + //phoenix.table.ttl.enabled is true + Admin admin = driver.getConnectionQueryServices(getUrl(), new Properties()).getAdmin(); + ColumnFamilyDescriptor[] columnFamilies = + admin.getDescriptor(TableName.valueOf(tableName)).getColumnFamilies(); + assertEquals(ColumnFamilyDescriptorBuilder.DEFAULT_TTL, columnFamilies[0].getTimeToLive()); - ddl = "ALTER TABLE " + tableName - + " SET TTL=FOREVER"; - conn.createStatement().execute(ddl); - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), - HConstants.FOREVER, tableName); - //Setting TTL should not be stored as CF Descriptor properties when - //phoenix.table.ttl.enabled is true - columnFamilies = - admin.getDescriptor(TableName.valueOf(tableName)).getColumnFamilies(); - assertEquals(ColumnFamilyDescriptorBuilder.DEFAULT_TTL, columnFamilies[0].getTimeToLive()); + tableName = generateUniqueName(); + ddl = + "create table IF NOT EXISTS " + tableName + " (" + " id char(1) NOT NULL," + + " col1 integer NOT NULL," + " b.col2 bigint," + " col3 bigint, " + + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1)" + + " ) TTL=NONE"; + conn.createStatement().execute(ddl); + assertTTLValue(conn.unwrap(PhoenixConnection.class), + TTL_EXPRESSION_NOT_DEFINED, tableName); + ddl = "ALTER TABLE " + tableName + " SET TTL=FOREVER"; + conn.createStatement().execute(ddl); + assertTTLValue(conn.unwrap(PhoenixConnection.class), + TTLExpression.TTL_EXPRESSION_FORVER, tableName); + //Setting TTL should not be stored as CF Descriptor properties when + //phoenix.table.ttl.enabled is true + columnFamilies = + admin.getDescriptor(TableName.valueOf(tableName)).getColumnFamilies(); + assertEquals(ColumnFamilyDescriptorBuilder.DEFAULT_TTL, columnFamilies[0].getTimeToLive()); + } } @Test @@ -151,12 +235,11 @@ public void testSettingTTLAsAlterTableCommand() throws Exception { PhoenixConnection pConn = conn.unwrap(PhoenixConnection.class);){ String tableName = createTableWithOrWithOutTTLAsItsProperty(conn, false); //Checking Default TTL in case of PhoenixTTLEnabled - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), PhoenixDatabaseMetaData.TTL_NOT_DEFINED, tableName); - String ddl = "ALTER TABLE " + tableName - + " SET TTL = " + DEFAULT_TTL_FOR_ALTER; + assertTTLValue(conn.unwrap(PhoenixConnection.class), TTL_EXPRESSION_NOT_DEFINED, tableName); + + String ddl = "ALTER TABLE " + tableName + " SET TTL = " + this.alterTTLDDLOption; conn.createStatement().execute(ddl); - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), - DEFAULT_TTL_FOR_ALTER, tableName); + assertTTLValue(conn.unwrap(PhoenixConnection.class), this.alterTTL, tableName); //Asserting TTL should not be stored as CF Descriptor properties when //phoenix.table.ttl.enabled is true Admin admin = driver.getConnectionQueryServices(getUrl(), new Properties()).getAdmin(); @@ -168,7 +251,7 @@ public void testSettingTTLAsAlterTableCommand() throws Exception { @Test public void testSettingTTLForIndexes() throws Exception { - try (Connection conn = DriverManager.getConnection(getUrl())){ + try (Connection conn = DriverManager.getConnection(getUrl())) { String tableName = createTableWithOrWithOutTTLAsItsProperty(conn, true); //By default, Indexes should set TTL what Base Table has @@ -176,7 +259,7 @@ public void testSettingTTLForIndexes() throws Exception { createIndexOnTableOrViewProvidedWithTTL(conn, tableName, PTable.IndexType.GLOBAL, false); List indexes = PhoenixRuntime.getTable(conn, tableName).getIndexes(); for (PTable index : indexes) { - assertTTLValueOfIndex(DEFAULT_TTL_FOR_TEST, index);; + assertTTLValue(index, defaultTTL); } tableName = createTableWithOrWithOutTTLAsItsProperty(conn, false); @@ -185,15 +268,18 @@ public void testSettingTTLForIndexes() throws Exception { String globalIndexName = createIndexOnTableOrViewProvidedWithTTL(conn, tableName, PTable.IndexType.GLOBAL, false); indexes = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, tableName)).getIndexes(); for (PTable index : indexes) { - assertTTLValueOfIndex(PhoenixDatabaseMetaData.TTL_NOT_DEFINED, index); + assertTTLValue(index, TTL_EXPRESSION_NOT_DEFINED); assertTrue(Bytes.compareTo( index.getRowKeyMatcher(), HConstants.EMPTY_BYTE_ARRAY) == 0 ); } //Test setting TTL as index property not allowed while creating them or setting them explicitly. + String ttl = (useExpression ? + String.format("'%s'",DEFAULT_TTL_EXPRESSION) : + String.valueOf(1000)); try { - conn.createStatement().execute("ALTER TABLE " + localIndexName + " SET TTL = 1000"); + conn.createStatement().execute("ALTER TABLE " + localIndexName + " SET TTL = " + ttl); fail(); } catch (SQLException sqe) { assertEquals("Should fail with cannot set or alter property for index", @@ -201,7 +287,7 @@ public void testSettingTTLForIndexes() throws Exception { } try { - conn.createStatement().execute("ALTER TABLE " + globalIndexName + " SET TTL = 1000"); + conn.createStatement().execute("ALTER TABLE " + globalIndexName + " SET TTL = " + ttl); fail(); } catch (SQLException sqe) { assertEquals("Should fail with cannot set or alter property for index", @@ -223,7 +309,60 @@ public void testSettingTTLForIndexes() throws Exception { assertEquals("Should fail with cannot set or alter property for index", CANNOT_SET_OR_ALTER_PROPERTY_FOR_INDEX.getErrorCode(), sqe.getErrorCode()); } + } + } + @Test + public void testConditionalTTLDDL() throws Exception { + if (!useExpression) { + return; + } + try (Connection conn = DriverManager.getConnection(getUrl())) { + String tableName = generateUniqueName(); + String ddl = "CREATE TABLE %s (ID1 VARCHAR NOT NULL, ID2 INTEGER NOT NULL, COL1 VARCHAR, COL2 DATE " + + "CONSTRAINT PK PRIMARY KEY(ID1, ID2)) TTL = '%s'"; + try { + conn.createStatement().execute(String.format(ddl, tableName, "ID2 = 12 OR UNKNOWN_COLUMN = 67")); + fail("Should have thrown ColumnNotFoundException"); + } catch (SQLException e) { + assertTrue(e.getCause() instanceof ColumnNotFoundException); + } + String ttl = "ID2 = 34 AND COL2 > CURRENT_DATE() + 1000"; + conn.createStatement().execute(String.format(ddl, tableName, ttl)); + TTLExpression expected = TTLExpression.create(ttl); + assertTTLValue(conn.unwrap(PhoenixConnection.class), expected, tableName); + + conn.createStatement().execute(String.format("ALTER TABLE %s SET TTL=NONE", tableName)); + assertTTLValue(conn.unwrap(PhoenixConnection.class), TTL_EXPRESSION_NOT_DEFINED, tableName); + + try { + conn.createStatement().execute(String.format("ALTER TABLE %s SET TTL='%s'", + tableName, "UNKNOWN_COLUMN=67")); + fail("Alter table should have thrown ColumnNotFoundException"); + } catch (SQLException e) { + assertTrue(e.getCause() instanceof ColumnNotFoundException); + } + + String viewName = generateUniqueName(); + ddl = "CREATE VIEW %s ( VINT SMALLINT) AS SELECT * FROM %s TTL='%s'"; + ttl = "F.ID2 = 124"; + try { + conn.createStatement().execute(String.format(ddl, viewName, tableName, ttl)); + fail("Should have thrown ColumnFamilyNotFoundException"); + } catch (SQLException e) { + assertTrue(e.getCause() instanceof ColumnFamilyNotFoundException); + } + + ttl = "COL2 > CURRENT_DATE() + 200 AND VINT > 123"; + conn.createStatement().execute(String.format(ddl, viewName, tableName, ttl)); + expected = TTLExpression.create(ttl); + assertTTLValue(conn.unwrap(PhoenixConnection.class), expected, viewName); + + ttl = "COL2 > CURRENT_DATE() + 500 AND VINT > 123"; + conn.createStatement().execute(String.format("ALTER VIEW %s SET TTL='%s'", + viewName, ttl)); + expected = TTLExpression.create(ttl); + assertTTLValue(conn.unwrap(PhoenixConnection.class), expected, viewName); } } @@ -242,8 +381,7 @@ public void testSettingTTLForViewsOnTableWithTTL() throws Exception { Connection tenantConn1 = DriverManager.getConnection(getUrl(), props1); String tableName = createTableWithOrWithOutTTLAsItsProperty(conn, true); - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), DEFAULT_TTL_FOR_TEST, - tableName); + assertTTLValue(conn.unwrap(PhoenixConnection.class), defaultTTL, tableName); //Setting TTL on views is not allowed if Table already has TTL try { @@ -265,17 +403,14 @@ public void testSettingTTLForViewsOnTableWithTTL() throws Exception { //View should have gotten TTL from parent table. String viewName = createUpdatableViewOnTableWithTTL(conn, tableName, false); - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), - DEFAULT_TTL_FOR_TEST, viewName); + assertTTLValue(conn.unwrap(PhoenixConnection.class), defaultTTL, viewName); //Child View's PTable gets TTL from parent View's PTable which gets from Table. String childView = createViewOnViewWithTTL(tenantConn, viewName, false); - assertTTLValueOfTableOrView(tenantConn.unwrap(PhoenixConnection.class), - DEFAULT_TTL_FOR_TEST, childView); + assertTTLValue(tenantConn.unwrap(PhoenixConnection.class), defaultTTL, childView); String childView1 = createViewOnViewWithTTL(tenantConn1, viewName, false); - assertTTLValueOfTableOrView(tenantConn1.unwrap(PhoenixConnection.class), - DEFAULT_TTL_FOR_TEST, childView1); + assertTTLValue(tenantConn1.unwrap(PhoenixConnection.class), defaultTTL, childView1); createIndexOnTableOrViewProvidedWithTTL(conn, viewName, PTable.IndexType.GLOBAL, false); @@ -286,7 +421,7 @@ public void testSettingTTLForViewsOnTableWithTTL() throws Exception { conn.unwrap(PhoenixConnection.class), viewName).getIndexes(); for (PTable index : indexes) { - assertTTLValueOfIndex(DEFAULT_TTL_FOR_TEST, index); + assertTTLValue(index, defaultTTL); } createIndexOnTableOrViewProvidedWithTTL(conn, tableName, PTable.IndexType.GLOBAL, false); @@ -294,9 +429,8 @@ public void testSettingTTLForViewsOnTableWithTTL() throws Exception { conn.unwrap(PhoenixConnection.class), tableName).getIndexes(); for (PTable index : tIndexes) { - assertTTLValueOfIndex(DEFAULT_TTL_FOR_TEST, index); + assertTTLValue(index, defaultTTL); } - } } @@ -310,8 +444,7 @@ public void testAlteringTTLToNONEAndThenSettingAtAnotherLevel() throws Exception Connection tenantConn = DriverManager.getConnection(getUrl(), props); String tableName = createTableWithOrWithOutTTLAsItsProperty(conn, true); - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), DEFAULT_TTL_FOR_TEST, - tableName); + assertTTLValue(conn.unwrap(PhoenixConnection.class), defaultTTL, tableName); //Setting TTL on views is not allowed if Table already has TTL try { @@ -325,11 +458,14 @@ public void testAlteringTTLToNONEAndThenSettingAtAnotherLevel() throws Exception String ddl = "ALTER TABLE " + tableName + " SET TTL=NONE"; conn.createStatement().execute(ddl); - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), TTL_NOT_DEFINED, - tableName); + assertTTLValue(conn.unwrap(PhoenixConnection.class), + TTL_EXPRESSION_NOT_DEFINED, tableName); String viewName = createUpdatableViewOnTableWithTTL(conn, tableName, true); - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), DEFAULT_TTL_FOR_CHILD, viewName); + TTLExpression expectedChildTTl = useExpression ? + TTLExpression.create(DEFAULT_TTL_EXPRESSION) : + TTLExpression.create(DEFAULT_TTL_FOR_CHILD); + assertTTLValue(conn.unwrap(PhoenixConnection.class), expectedChildTTl, viewName); try { createViewOnViewWithTTL(tenantConn, viewName, true); @@ -339,8 +475,11 @@ public void testAlteringTTLToNONEAndThenSettingAtAnotherLevel() throws Exception TTL_ALREADY_DEFINED_IN_HIERARCHY.getErrorCode(), sqe.getErrorCode()); } + String ttlAlter = (useExpression ? + String.format("'%s'", DEFAULT_TTL_EXPRESSION_FOR_ALTER) : + String.valueOf(DEFAULT_TTL_FOR_ALTER)); try { - ddl = "ALTER TABLE " + tableName + " SET TTL=" + DEFAULT_TTL_FOR_ALTER; + ddl = "ALTER TABLE " + tableName + " SET TTL=" + ttlAlter; conn.createStatement().execute(ddl); } catch (SQLException sqe) { assertEquals("Should fail with TTL already defined in hierarchy", @@ -351,19 +490,20 @@ public void testAlteringTTLToNONEAndThenSettingAtAnotherLevel() throws Exception conn.createStatement().execute(ddl); String childView = createViewOnViewWithTTL(tenantConn, viewName, true); - assertTTLValueOfTableOrView(tenantConn.unwrap(PhoenixConnection.class), - DEFAULT_TTL_FOR_CHILD, childView); + assertTTLValue(tenantConn.unwrap(PhoenixConnection.class), expectedChildTTl, childView); ddl = "ALTER VIEW " + childView + " SET TTL=NONE"; tenantConn.createStatement().execute(ddl); - assertTTLValueOfTableOrView(tenantConn.unwrap(PhoenixConnection.class), - TTL_NOT_DEFINED, childView); + assertTTLValue(tenantConn.unwrap(PhoenixConnection.class), + TTL_EXPRESSION_NOT_DEFINED, childView); - ddl = "ALTER VIEW " + viewName + " SET TTL=" + DEFAULT_TTL_FOR_ALTER; + ddl = "ALTER VIEW " + viewName + " SET TTL=" + ttlAlter; conn.createStatement().execute(ddl); - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), DEFAULT_TTL_FOR_ALTER, viewName); - + TTLExpression expectedAlterTTl = useExpression ? + TTLExpression.create(DEFAULT_TTL_EXPRESSION_FOR_ALTER) : + TTLExpression.create(DEFAULT_TTL_FOR_ALTER); + assertTTLValue(conn.unwrap(PhoenixConnection.class), expectedAlterTTl, viewName); } } @@ -382,24 +522,23 @@ public void testAlteringTTLAtOneLevelAndCheckingAtAnotherLevel() throws Exceptio Connection tenantConn1 = DriverManager.getConnection(getUrl(), props1); String tableName = createTableWithOrWithOutTTLAsItsProperty(conn, true); - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), DEFAULT_TTL_FOR_TEST, - tableName); + assertTTLValue(conn.unwrap(PhoenixConnection.class), defaultTTL, tableName); //View should have gotten TTL from parent table. String viewName = createUpdatableViewOnTableWithTTL(conn, tableName, false); - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), - DEFAULT_TTL_FOR_TEST, viewName); + assertTTLValue(conn.unwrap(PhoenixConnection.class), defaultTTL, viewName); //Child View's PTable gets TTL from parent View's PTable which gets from Table. String childView = createViewOnViewWithTTL(tenantConn, viewName, false); - assertTTLValueOfTableOrView(tenantConn.unwrap(PhoenixConnection.class), - DEFAULT_TTL_FOR_TEST, childView); + assertTTLValue(tenantConn.unwrap(PhoenixConnection.class), defaultTTL, childView); String childView1 = createViewOnViewWithTTL(tenantConn1, viewName, false); - assertTTLValueOfTableOrView(tenantConn1.unwrap(PhoenixConnection.class), - DEFAULT_TTL_FOR_TEST, childView1); + assertTTLValue(tenantConn1.unwrap(PhoenixConnection.class), defaultTTL, childView1); - String alter = "ALTER TABLE " + tableName + " SET TTL = " + DEFAULT_TTL_FOR_ALTER; + String ttlAlter = (useExpression ? + String.format("'%s'", DEFAULT_TTL_EXPRESSION_FOR_ALTER) : + String.valueOf(DEFAULT_TTL_FOR_ALTER)); + String alter = "ALTER TABLE " + tableName + " SET TTL = " + ttlAlter; conn.createStatement().execute(alter); //Clear Cache for all Tables to reflect Alter TTL commands in hierarchy @@ -408,52 +547,55 @@ public void testAlteringTTLAtOneLevelAndCheckingAtAnotherLevel() throws Exceptio clearCache(tenantConn, null, childView); clearCache(tenantConn1, null, childView1); + TTLExpression expectedAlterTTl = useExpression ? + TTLExpression.create(DEFAULT_TTL_EXPRESSION_FOR_ALTER) : + TTLExpression.create(DEFAULT_TTL_FOR_ALTER); //Assert TTL for each entity again with altered value - assertTTLValueOfTableOrView(conn.unwrap(PhoenixConnection.class), - DEFAULT_TTL_FOR_ALTER, viewName); - assertTTLValueOfTableOrView(tenantConn.unwrap(PhoenixConnection.class), - DEFAULT_TTL_FOR_ALTER, childView); - assertTTLValueOfTableOrView(tenantConn1.unwrap(PhoenixConnection.class), - DEFAULT_TTL_FOR_ALTER, childView1); + assertTTLValue(conn.unwrap(PhoenixConnection.class), expectedAlterTTl, viewName); + assertTTLValue(tenantConn.unwrap(PhoenixConnection.class), expectedAlterTTl, childView); + assertTTLValue(tenantConn1.unwrap(PhoenixConnection.class), expectedAlterTTl, childView1); } } - private void assertTTLValueOfTableOrView(PhoenixConnection conn, long expected, String name) throws SQLException { + private void assertTTLValue(PhoenixConnection conn, TTLExpression expected, String name) throws SQLException { assertEquals("TTL value did not match :-", expected, PhoenixRuntime.getTableNoCache(conn, name).getTTL()); } - private void assertTTLValueOfIndex(long expected, PTable index) { - assertEquals("TTL value is not what expected :-", expected, index.getTTL()); + private void assertTTLValue(PTable table, TTLExpression expected) { + assertEquals("TTL value did not match :-", expected, table.getTTL()); } - private String createTableWithOrWithOutTTLAsItsProperty(Connection conn, boolean withTTL) throws SQLException { String tableName = generateUniqueName(); - conn.createStatement().execute("CREATE TABLE IF NOT EXISTS " + tableName + " (" - + " ID INTEGER NOT NULL," - + " COL1 INTEGER NOT NULL," - + " COL2 bigint NOT NULL," - + " CREATED_DATE DATE," - + " CREATION_TIME BIGINT," - + " CONSTRAINT NAME_PK PRIMARY KEY (ID, COL1, COL2)) MULTI_TENANT=true " - + ( withTTL ? ", TTL = " + DEFAULT_TTL_FOR_TEST : "")); + StringBuilder ddl = new StringBuilder(); + ddl.append(String.format(DDL_TEMPLATE, tableName)); + ddl.append(DEFAULT_DDL_OPTIONS); + if (withTTL) { + ddl.append(", TTL = " + this.defaultTTLDDLOption); + } + conn.createStatement().execute(ddl.toString()); return tableName; } private String createIndexOnTableOrViewProvidedWithTTL(Connection conn, String baseTableOrViewName, PTable.IndexType indexType, boolean withTTL) throws SQLException { + String ttl = (useExpression ? + String.format("'%s'", DEFAULT_TTL_EXPRESSION) : + String.valueOf(DEFAULT_TTL_FOR_CHILD)); switch (indexType) { case LOCAL: String localIndexName = baseTableOrViewName + "_Local_" + generateUniqueName(); conn.createStatement().execute("CREATE LOCAL INDEX " + localIndexName + " ON " + - baseTableOrViewName + " (COL2) " + (withTTL ? "TTL = " + DEFAULT_TTL_FOR_CHILD : "")); + baseTableOrViewName + " (COL2) INCLUDE (CREATION_TIME)" + + (withTTL ? "TTL = " + ttl : "")); return localIndexName; case GLOBAL: String globalIndexName = baseTableOrViewName + "_Global_" + generateUniqueName(); conn.createStatement().execute("CREATE INDEX " + globalIndexName + " ON " + - baseTableOrViewName + " (COL2) " + (withTTL ? "TTL = " + DEFAULT_TTL_FOR_CHILD : "")); + baseTableOrViewName + " (COL2) INCLUDE (CREATION_TIME) " + + (withTTL ? "TTL = " + ttl : "")); return globalIndexName; default: @@ -463,30 +605,39 @@ private String createIndexOnTableOrViewProvidedWithTTL(Connection conn, String b private String createReadOnlyViewOnTableWithTTL(Connection conn, String baseTableName, boolean withTTL) throws SQLException { + String ttl = (useExpression ? + String.format("'%s'", DEFAULT_TTL_EXPRESSION) : + String.valueOf(DEFAULT_TTL_FOR_CHILD)); String viewName = "VIEW_" + baseTableName + "_" + generateUniqueName(); conn.createStatement().execute("CREATE VIEW " + viewName + " (" + generateUniqueName() + " SMALLINT) as select * from " + baseTableName + " where COL1 > 1 " - + (withTTL ? "TTL = " + DEFAULT_TTL_FOR_CHILD : "") ); + + (withTTL ? "TTL = " + ttl : "") ); return viewName; } private String createUpdatableViewOnTableWithTTL(Connection conn, String baseTableName, boolean withTTL) throws SQLException { + String ttl = (useExpression ? + String.format("'%s'", DEFAULT_TTL_EXPRESSION) : + String.valueOf(DEFAULT_TTL_FOR_CHILD)); String viewName = "VIEW_" + baseTableName + "_" + generateUniqueName(); conn.createStatement().execute("CREATE VIEW " + viewName + " (" + generateUniqueName() + " SMALLINT) as select * from " + baseTableName + " where COL1 = 1 " - + (withTTL ? "TTL = " + DEFAULT_TTL_FOR_CHILD : "") ); + + (withTTL ? "TTL = " + ttl : "") ); return viewName; } private String createViewOnViewWithTTL(Connection conn, String parentViewName, boolean withTTL) throws SQLException { + String ttl = (useExpression ? + String.format("'%s'", DEFAULT_TTL_EXPRESSION) : + String.valueOf(DEFAULT_TTL_FOR_CHILD)); String childView = parentViewName + "_" + generateUniqueName(); conn.createStatement().execute("CREATE VIEW " + childView + " (E BIGINT, F BIGINT) AS SELECT * FROM " + parentViewName + - (withTTL ? " TTL = " + DEFAULT_TTL_FOR_CHILD : "")); + (withTTL ? " TTL = " + ttl : "")); return childView; } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLIT.java index 63bf22f5945..b901082d711 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLIT.java @@ -23,6 +23,7 @@ import org.apache.phoenix.query.PhoenixTestBuilder.SchemaBuilder; import org.apache.phoenix.query.PhoenixTestBuilder.SchemaBuilder.TableOptions; import org.apache.phoenix.schema.PTable; +import org.apache.phoenix.schema.LiteralTTLExpression; import org.apache.phoenix.thirdparty.com.google.common.collect.Lists; import org.apache.phoenix.util.PhoenixRuntime; import org.junit.Assert; @@ -136,19 +137,22 @@ private SchemaBuilder createTenantViewOnTableOrGlobalView(SchemaBuilder schemaBu */ private void assertTTLForGivenPTable(PTable table, int ttl) { - Assert.assertEquals(ttl, table.getTTL()); + LiteralTTLExpression expected = new LiteralTTLExpression(ttl); + Assert.assertEquals(expected, table.getTTL()); } private void assertTTLForGivenEntity(Connection connection, String entityName, int ttl) throws SQLException { PTable pTable = PhoenixRuntime.getTable(connection, entityName); - Assert.assertEquals(ttl,pTable.getTTL()); + LiteralTTLExpression expected = new LiteralTTLExpression(ttl); + Assert.assertEquals(expected, pTable.getTTL()); } private void assertTTLForIndexName(Connection connection, String indexName, int ttl) throws SQLException { if (!indexName.equals(SKIP_ASSERT)) { PTable index = PhoenixRuntime.getTable(connection, indexName); - Assert.assertEquals(ttl,index.getTTL()); + LiteralTTLExpression expected = new LiteralTTLExpression(ttl); + Assert.assertEquals(expected, index.getTTL()); } } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java index d3d04c63c6b..02b47771b9a 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java @@ -18,21 +18,34 @@ package org.apache.phoenix.end2end; -import org.apache.hadoop.hbase.CompareOperator; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.ResultScanner; +import static java.util.Arrays.asList; +import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.COLUMN_TYPES; +import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.DEFAULT_SCHEMA_NAME; +import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.MAX_ROWS; +import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_COLUMNS; +import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_INCLUDE_COLUMNS; +import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_INDEX_COLUMNS; +import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_PK_COLUMNS; +import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_PK_TYPES; +import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; + import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.filter.BinaryComparator; -import org.apache.hadoop.hbase.filter.FilterList; -import org.apache.hadoop.hbase.filter.QualifierFilter; -import org.apache.hadoop.hbase.filter.RowFilter; -import org.apache.hadoop.hbase.filter.SubstringComparator; import org.apache.hadoop.hbase.util.Bytes; import org.apache.phoenix.compile.QueryPlan; import org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants; -import org.apache.phoenix.coprocessor.BaseScannerRegionObserver; import org.apache.phoenix.exception.SQLExceptionCode; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixResultSet; @@ -74,34 +87,16 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; - -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; -import java.util.Random; - -import static java.util.Arrays.asList; -import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.COLUMN_TYPES; -import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.DEFAULT_SCHEMA_NAME; -import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.MAX_ROWS; -import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_COLUMNS; -import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_INCLUDE_COLUMNS; -import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_INDEX_COLUMNS; -import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_PK_COLUMNS; -import static org.apache.phoenix.query.PhoenixTestBuilder.DDLDefaults.TENANT_VIEW_PK_TYPES; -import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; @Category(NeedsOwnMiniClusterTest.class) public class ViewTTLIT extends BaseViewTTLIT { + public ViewTTLIT(boolean useCondExpression) { + super(useCondExpression); + } + @BeforeClass public static void doSetup() throws Exception { // Turn on the View TTL feature @@ -111,7 +106,6 @@ public static void doSetup() throws Exception { put("hbase.procedure.remote.dispatcher.delay.msec", "0"); // no max lookback put(BaseScannerRegionObserverConstants.PHOENIX_MAX_LOOKBACK_AGE_CONF_KEY, Integer.toString(0)); - put(QueryServices.PHOENIX_VIEW_TTL_ENABLED, Boolean.toString(true)); put(QueryServices.PHOENIX_VIEW_TTL_TENANT_VIEWS_PER_SCAN_LIMIT, String.valueOf(1)); }}; @@ -164,10 +158,11 @@ public void testPhoenixTTLWithTableLevelTTLFails() throws Exception { final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl()); TableOptions tableOptions = TableOptions.withDefaults(); - tableOptions.setTableProps("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true,TTL=100"); + tableOptions.setTableProps(String.format("COLUMN_ENCODED_BYTES=0,MULTI_TENANT=true," + + "TTL='%s'", getTTLExpression(100))); TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults(); - tenantViewOptions.setTableProps("TTL=1000"); + tenantViewOptions.setTableProps(String.format("TTL='%s'", getTTLExpression(1000))); try { schemaBuilder.withTableOptions(tableOptions).withTenantViewOptions(tenantViewOptions) .buildNewView(); @@ -182,7 +177,7 @@ public void testPhoenixTTLWithTableLevelTTLFails() throws Exception { public void testPhoenixTTLWithViewIndexFails() throws Exception { TenantViewIndexOptions tenantViewIndexOptions = TenantViewIndexOptions.withDefaults(); - tenantViewIndexOptions.setIndexProps("TTL=1000"); + tenantViewIndexOptions.setIndexProps(String.format("TTL='%s'", getTTLExpression(1000))); try { final SchemaBuilder schemaBuilder = createLevel1TenantView(null, tenantViewIndexOptions); @@ -199,7 +194,8 @@ public void testPhoenixTTLForLevelOneView() throws Exception { TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults(); // View TTL is set to 120s => 120000 ms - tenantViewOptions.setTableProps(String.format("TTL=%d", VIEW_TTL_120_SECS)); + String ttlExpr = getTTLExpression(VIEW_TTL_120_SECS); + tenantViewOptions.setTableProps(String.format("TTL='%s'", ttlExpr)); final SchemaBuilder schemaBuilder = createLevel1TenantView(tenantViewOptions, null); String tenantId = schemaBuilder.getDataOptions().getTenantId(); @@ -216,9 +212,9 @@ public void testPhoenixTTLForLevelOneView() throws Exception { assertViewHeaderRowsHavePhoenixTTLRelatedCells( schemaBuilder.getTableOptions().getSchemaName(), startTime, false, 1); assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, - PTableType.VIEW.getSerializedValue(), 120); + PTableType.VIEW.getSerializedValue(), ttlExpr); assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, - PTableType.INDEX.getSerializedValue(), 0); + PTableType.INDEX.getSerializedValue(), "0"); } @@ -237,17 +233,18 @@ public void testPhoenixTTLForLevelTwoView() throws Exception { String tenantViewName = stripQuotes( SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName())); + String ttlExpr = getTTLExpression(300); // Expected 1 rows - one for GlobalView. // Since the TTL property values are being set, // we expect the view header columns to show up in regular scans too. assertViewHeaderRowsHavePhoenixTTLRelatedCells( schemaBuilder.getTableOptions().getSchemaName(), startTime, false, 1); assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, globalViewName, - PTableType.VIEW.getSerializedValue(), 300); + PTableType.VIEW.getSerializedValue(), ttlExpr); // Since the TTL property values are not being overridden, // we expect the TTL value to be not set for the tenant view. assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, - PTableType.VIEW.getSerializedValue(), 0); + PTableType.VIEW.getSerializedValue(), "0"); } @Test @@ -273,9 +270,9 @@ public void testPhoenixTTLForWhenTTLIsZero() throws Exception { assertViewHeaderRowsHavePhoenixTTLRelatedCells( schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 3); assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, - PTableType.VIEW.getSerializedValue(), 0); + PTableType.VIEW.getSerializedValue(), "0"); assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, - PTableType.INDEX.getSerializedValue(), 0); + PTableType.INDEX.getSerializedValue(), "0"); } @@ -303,16 +300,16 @@ public void testPhoenixTTLWithAlterView() throws Exception { assertViewHeaderRowsHavePhoenixTTLRelatedCells( schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 3); assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, - PTableType.VIEW.getSerializedValue(), 0); + PTableType.VIEW.getSerializedValue(), "0"); assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, - PTableType.INDEX.getSerializedValue(), 0); + PTableType.INDEX.getSerializedValue(), "0"); String tenantURL = getUrl() + ';' + TENANT_ID_ATTRIB + '=' + tenantId; try (Connection connection = DriverManager.getConnection(tenantURL)) { try (Statement stmt = connection.createStatement()) { // View TTL is set to 120s => 120000 ms String sql = String - .format(ALTER_TTL_SQL, schemaName, tenantViewName, String.valueOf(ttl)); + .format(ALTER_TTL_SQL, schemaName, tenantViewName, getTTLExpression(ttl)); stmt.execute(sql); } } @@ -324,9 +321,9 @@ public void testPhoenixTTLWithAlterView() throws Exception { assertViewHeaderRowsHavePhoenixTTLRelatedCells( schemaBuilder.getTableOptions().getSchemaName(), startTime, false, 1); assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, - PTableType.VIEW.getSerializedValue(), ttl); + PTableType.VIEW.getSerializedValue(), getTTLExpression(ttl)); assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, - PTableType.INDEX.getSerializedValue(), 0); + PTableType.INDEX.getSerializedValue(), "0"); } @@ -335,7 +332,8 @@ public void testTTLAlreadyDefinedInHierarchyWhenCreateView() throws Exception { try { TenantViewOptions tenantViewWithOverrideOptions = TenantViewOptions.withDefaults(); // View TTL is set to 120s => 120000 ms - tenantViewWithOverrideOptions.setTableProps(String.format("TTL=%d", VIEW_TTL_120_SECS)); + tenantViewWithOverrideOptions.setTableProps( + String.format("TTL='%s'", getTTLExpression(VIEW_TTL_120_SECS))); final SchemaBuilder schemaBuilder = createLevel2TenantViewWithGlobalLevelTTL( tenantViewWithOverrideOptions, null, false); fail(); @@ -361,17 +359,18 @@ public void testTTLAlreadyDefinedInHierarchyWhenAlterTenantView() throws Excepti String tenantViewName = stripQuotes( SchemaUtil.getTableNameFromFullName(schemaBuilder.getEntityTenantViewName())); + String ttlExpr = getTTLExpression(300); // Expected 1 rows - one for GlobalView. // Since the PHOENIX_TTL property values are being set, // we expect the view header columns to show up in regular scans too. assertViewHeaderRowsHavePhoenixTTLRelatedCells( schemaBuilder.getTableOptions().getSchemaName(), startTime, false, 1); assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, globalViewName, - PTableType.VIEW.getSerializedValue(), 300); + PTableType.VIEW.getSerializedValue(), ttlExpr); // Since the TTL is set at the global level // we expect the TTL value to be not set (= 0) at tenant level. assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, - PTableType.VIEW.getSerializedValue(), 0); + PTableType.VIEW.getSerializedValue(), "0"); String tenantURL = getUrl() + ';' + TENANT_ID_ATTRIB + '=' + tenantId; try (Connection connection = DriverManager.getConnection(tenantURL)) { @@ -406,21 +405,22 @@ public void testTTLAlreadyDefinedInHierarchyWhenAlterGlobalView() throws Excepti String indexOnTenantViewName = String .format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix())); + String ttlExpr = getTTLExpression(300); // Expected 1 rows - one for TenantView. // Since the TTL property values are being set, // we expect the view header columns to show up in regular scans too. assertViewHeaderRowsHavePhoenixTTLRelatedCells( schemaBuilder.getTableOptions().getSchemaName(), startTime, false, 1); assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, - PTableType.VIEW.getSerializedValue(), 300); + PTableType.VIEW.getSerializedValue(), ttlExpr); assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, - PTableType.INDEX.getSerializedValue(), 0); + PTableType.INDEX.getSerializedValue(), "0"); try (Connection connection = DriverManager.getConnection(getUrl())) { try (Statement stmt = connection.createStatement()) { // View TTL is set to 120s => 120000 ms String sql = String - .format(ALTER_TTL_SQL, schemaName, globalViewName, "120"); + .format(ALTER_TTL_SQL, schemaName, globalViewName, getTTLExpression(120)); stmt.execute(sql); fail(); } @@ -448,15 +448,16 @@ public void testAlterViewWithNoTTLPropertySucceed() throws Exception { String indexOnTenantViewName = String .format("IDX_%s", stripQuotes(schemaBuilder.getEntityKeyPrefix())); + String ttlExpr = getTTLExpression(300); // Expected 1 rows - one for TenantView. // Since the TTL property values are being set, // we expect the view header columns to show up in regular scans too. assertViewHeaderRowsHavePhoenixTTLRelatedCells( schemaBuilder.getTableOptions().getSchemaName(), startTime, false, 1); assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, - PTableType.VIEW.getSerializedValue(), 300); + PTableType.VIEW.getSerializedValue(), ttlExpr); assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, - PTableType.INDEX.getSerializedValue(), 0); + PTableType.INDEX.getSerializedValue(), "0"); // ALTER global view try (Connection connection = DriverManager.getConnection(getUrl())) { @@ -479,7 +480,6 @@ public void testAlterViewWithNoTTLPropertySucceed() throws Exception { } - @Test public void testResetViewTTL() throws Exception { long startTime = EnvironmentEdgeManager.currentTimeMillis(); @@ -501,7 +501,7 @@ public void testResetViewTTL() throws Exception { try (Statement stmt = connection.createStatement()) { // View TTL is set to 'NONE' String sql = String - .format(ALTER_TTL_SQL, schemaName, globalViewName, "'NONE'"); + .format(ALTER_TTL_SQL, schemaName, globalViewName, "NONE"); stmt.execute(sql); } } @@ -513,15 +513,15 @@ public void testResetViewTTL() throws Exception { assertViewHeaderRowsHavePhoenixTTLRelatedCells( schemaBuilder.getTableOptions().getSchemaName(), startTime, true, 4); assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, globalViewName, - PTableType.VIEW.getSerializedValue(), 0); + PTableType.VIEW.getSerializedValue(), "0"); assertSyscatHavePhoenixTTLRelatedColumns("", schemaName, indexOnGlobalViewName, - PTableType.INDEX.getSerializedValue(), 0); + PTableType.INDEX.getSerializedValue(), "0"); // Since the TTL property values for the tenant view are not being reset, // we expect the TTL value to be same as before. assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, tenantViewName, - PTableType.VIEW.getSerializedValue(), 0); + PTableType.VIEW.getSerializedValue(), "0"); assertSyscatHavePhoenixTTLRelatedColumns(tenantId, schemaName, indexOnTenantViewName, - PTableType.INDEX.getSerializedValue(), 0); + PTableType.INDEX.getSerializedValue(), "0"); } @@ -555,7 +555,8 @@ public void testWithTenantViewAndNoGlobalView() throws Exception { if (isMultiTenant) { TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults(); - tenantViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + tenantViewOptions.setTableProps(String.format("TTL='%s'", + getTTLExpression(viewTTL))); schemaBuilder .withTableOptions(tableOptions) .withTenantViewOptions(tenantViewOptions) @@ -570,7 +571,8 @@ public void testWithTenantViewAndNoGlobalView() throws Exception { globalViewOptions.setGlobalViewColumnTypes(Lists.newArrayList(COLUMN_TYPES)); globalViewOptions.setGlobalViewPKColumns(Lists.newArrayList(TENANT_VIEW_PK_COLUMNS)); globalViewOptions.setGlobalViewPKColumnTypes(Lists.newArrayList(TENANT_VIEW_PK_TYPES)); - globalViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + globalViewOptions.setTableProps(String.format("TTL='%s'", + getTTLExpression(viewTTL))); GlobalViewIndexOptions globalViewIndexOptions = new GlobalViewIndexOptions(); globalViewIndexOptions.setGlobalViewIndexColumns( @@ -668,7 +670,7 @@ public void testWithSQLUsingIndexWithCoveredColsUpdates() throws Exception { tableOptions.getTableColumnTypes().clear(); GlobalViewOptions globalViewOptions = SchemaBuilder.GlobalViewOptions.withDefaults(); - globalViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + globalViewOptions.setTableProps(String.format("TTL='%s'", getTTLExpression(viewTTL))); GlobalViewIndexOptions globalViewIndexOptions = @@ -770,7 +772,7 @@ public void testWithSQLUsingIndexAndNoCoveredColsUpdates() throws Exception { tableOptions.getTableColumnTypes().clear(); GlobalViewOptions globalViewOptions = SchemaBuilder.GlobalViewOptions.withDefaults(); - globalViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + globalViewOptions.setTableProps(String.format("TTL='%s'", getTTLExpression(viewTTL))); GlobalViewIndexOptions globalViewIndexOptions = SchemaBuilder.GlobalViewIndexOptions.withDefaults(); @@ -907,7 +909,7 @@ public void testWithSQLUsingIndexAndMultiLevelViews() throws Exception { tableOptions.getTableColumnTypes().clear(); GlobalViewOptions globalViewOptions = SchemaBuilder.GlobalViewOptions.withDefaults(); - globalViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + globalViewOptions.setTableProps(String.format("TTL='%s'", getTTLExpression(viewTTL))); GlobalViewIndexOptions globalViewIndexOptions = SchemaBuilder.GlobalViewIndexOptions.withDefaults(); @@ -1048,7 +1050,7 @@ public void testWithVariousSQLs() throws Exception { tableOptions.getTableColumnTypes().clear(); GlobalViewOptions globalViewOptions = SchemaBuilder.GlobalViewOptions.withDefaults(); - globalViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + globalViewOptions.setTableProps(String.format("TTL='%s'", getTTLExpression(viewTTL))); GlobalViewIndexOptions globalViewIndexOptions = SchemaBuilder.GlobalViewIndexOptions.withDefaults(); @@ -1145,7 +1147,7 @@ public void testWithVariousSQLsForMultipleTenants() throws Exception { tableOptions.getTableColumnTypes().clear(); GlobalViewOptions globalViewOptions = SchemaBuilder.GlobalViewOptions.withDefaults(); - globalViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + globalViewOptions.setTableProps(String.format("TTL='%s'", getTTLExpression(viewTTL))); GlobalViewIndexOptions globalViewIndexOptions = SchemaBuilder.GlobalViewIndexOptions.withDefaults(); @@ -1252,7 +1254,7 @@ public void testWithVariousSQLsForMultipleViews() throws Exception { tenantViewOptions .setTenantViewColumnTypes(asList("CHAR(15)", "VARCHAR", "VARCHAR", "VARCHAR")); - tenantViewOptions.setTableProps(String.format("TTL=%d", viewTTL)); + tenantViewOptions.setTableProps(String.format("TTL='%s'", getTTLExpression(viewTTL))); OtherOptions testCaseWhenAllCFMatchAndAllDefault = new OtherOptions(); testCaseWhenAllCFMatchAndAllDefault.setTestName("testCaseWhenAllCFMatchAndAllDefault"); @@ -1339,7 +1341,7 @@ public void testWithVariousSQLsForMultipleViews() throws Exception { @Test public void testGlobalAndTenantViewTTLInheritance1() throws Exception { // View TTL is set in seconds (for e.g 200 secs) - long tenantPhoenixTTL = 200; + int tenantPhoenixTTL = 200; // Define the test schema. // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP) @@ -1361,7 +1363,8 @@ public void testGlobalAndTenantViewTTLInheritance1() throws Exception { tenantViewWithOverrideOptions.setTenantViewColumns(asList("ZID", "COL7", "COL8", "COL9")); tenantViewWithOverrideOptions .setTenantViewColumnTypes(asList("CHAR(15)", "VARCHAR", "VARCHAR", "VARCHAR")); - tenantViewWithOverrideOptions.setTableProps(String.format("TTL=%d", tenantPhoenixTTL)); + tenantViewWithOverrideOptions.setTableProps(String.format("TTL='%s'", + getTTLExpression(tenantPhoenixTTL))); OtherOptions testCaseWhenAllCFMatchAndAllDefault = new OtherOptions(); testCaseWhenAllCFMatchAndAllDefault.setTestName("testCaseWhenAllCFMatchAndAllDefault"); @@ -1454,7 +1457,7 @@ public void testGlobalAndTenantViewTTLInheritance1() throws Exception { @Test public void testGlobalAndTenantViewTTLInheritance2() throws Exception { // View TTL is set in seconds (for e.g 300 secs) - long globalViewTTL = VIEW_TTL_300_SECS; + int globalViewTTL = VIEW_TTL_300_SECS; // Define the test schema. // 1. Table with default columns => (ORG_ID, KP, COL1, COL2, COL3), PK => (ORG_ID, KP) @@ -1467,7 +1470,8 @@ public void testGlobalAndTenantViewTTLInheritance2() throws Exception { SchemaBuilder.GlobalViewOptions globalViewOptions = SchemaBuilder.GlobalViewOptions.withDefaults(); - globalViewOptions.setTableProps(String.format("TTL=%d", globalViewTTL)); + globalViewOptions.setTableProps(String.format("TTL='%s'", + getTTLExpression(globalViewTTL))); SchemaBuilder.GlobalViewIndexOptions globalViewIndexOptions = SchemaBuilder.GlobalViewIndexOptions.withDefaults(); @@ -1563,13 +1567,14 @@ public void testGlobalAndTenantViewTTLInheritance2() throws Exception { public void testScanAttributes() throws Exception { // PHOENIX TTL is set in seconds (for e.g 10 secs) - long phoenixTTL = 10; + int phoenixTTL = 10; TableOptions tableOptions = TableOptions.withDefaults(); tableOptions.getTableColumns().clear(); tableOptions.getTableColumnTypes().clear(); TenantViewOptions tenantViewOptions = TenantViewOptions.withDefaults(); - tenantViewOptions.setTableProps(String.format("TTL=%d", phoenixTTL)); + String ttlExpr = getTTLExpression(phoenixTTL); + tenantViewOptions.setTableProps(String.format("TTL='%s'", ttlExpr)); // Define the test schema. final SchemaBuilder schemaBuilder = new SchemaBuilder(getUrl()); @@ -1602,8 +1607,8 @@ public void testScanAttributes() throws Exception { queryPlan.getContext()); Assert.assertFalse("Should not have any rows", rs.next()); Assert.assertEquals("Should have atleast one element", 1, queryPlan.getScans().size()); - Assert.assertEquals("PhoenixTTL does not match", phoenixTTL, - ScanUtil.getTTL(queryPlan.getScans().get(0).get(0))); + Assert.assertEquals("PhoenixTTL does not match", table.getTTL(), + ScanUtil.getTTLExpression(queryPlan.getScans().get(0).get(0))); Assert.assertTrue("Masking attribute should be set", ScanUtil.isMaskTTLExpiredRows(queryPlan.getScans().get(0).get(0))); Assert.assertFalse("Delete Expired attribute should not set", @@ -1638,7 +1643,7 @@ public void testScanAttributes() throws Exception { scan.setAttribute(BaseScannerRegionObserverConstants.DELETE_PHOENIX_TTL_EXPIRED, PDataType.TRUE_BYTES); scan.setAttribute(BaseScannerRegionObserverConstants.TTL, - Bytes.toBytes(Long.valueOf(table.getTTL()))); + table.getTTL().getTTLForScanAttribute(conn, table)); PhoenixResultSet rs = @@ -1646,8 +1651,8 @@ public void testScanAttributes() throws Exception { queryPlan.getContext()); Assert.assertFalse("Should not have any rows", rs.next()); Assert.assertEquals("Should have atleast one element", 1, queryPlan.getScans().size()); - Assert.assertEquals("PhoenixTTL does not match", phoenixTTL, - ScanUtil.getTTL(queryPlan.getScans().get(0).get(0))); + Assert.assertEquals("PhoenixTTL does not match", table.getTTL(), + ScanUtil.getTTLExpression(queryPlan.getScans().get(0).get(0))); Assert.assertFalse("Masking attribute should not be set", ScanUtil.isMaskTTLExpiredRows(queryPlan.getScans().get(0).get(0))); Assert.assertTrue("Delete Expired attribute should be set", diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLWithLongViewIndexEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLWithLongViewIndexEnabledIT.java index 102ef9e368a..3bf07eabc88 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLWithLongViewIndexEnabledIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLWithLongViewIndexEnabledIT.java @@ -18,7 +18,9 @@ package org.apache.phoenix.end2end; -import org.apache.phoenix.coprocessor.BaseScannerRegionObserver; +import java.util.HashMap; +import java.util.Map; + import org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants; import org.apache.phoenix.query.QueryServices; import org.apache.phoenix.schema.types.PInteger; @@ -29,12 +31,13 @@ import org.junit.Test; import org.junit.experimental.categories.Category; -import java.util.HashMap; -import java.util.Map; - @Category(NeedsOwnMiniClusterTest.class) public class ViewTTLWithLongViewIndexEnabledIT extends BaseViewTTLIT { + public ViewTTLWithLongViewIndexEnabledIT(boolean useCondExpression) { + super(useCondExpression); + } + @BeforeClass public static final void doSetup() throws Exception { // Turn on the TTL feature diff --git a/phoenix-core/src/it/java/org/apache/phoenix/schema/ConditionalTTLExpressionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/schema/ConditionalTTLExpressionIT.java new file mode 100644 index 00000000000..ebac0e052d2 --- /dev/null +++ b/phoenix-core/src/it/java/org/apache/phoenix/schema/ConditionalTTLExpressionIT.java @@ -0,0 +1,964 @@ +/* + * 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.schema; + +import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_BEYOND_MAXLOOKBACK_MISSING_INDEX_ROW_COUNT; +import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_EXPIRED_INDEX_ROW_COUNT; +import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_INVALID_INDEX_ROW_COUNT; +import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_MISSING_INDEX_ROW_COUNT; +import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.BEFORE_REBUILD_VALID_INDEX_ROW_COUNT; +import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.REBUILT_INDEX_ROW_COUNT; +import static org.apache.phoenix.mapreduce.index.PhoenixIndexToolJobCounters.SCANNED_DATA_ROW_COUNT; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.Collection; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.mapreduce.CounterGroup; +import org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants; +import org.apache.phoenix.end2end.IndexToolIT; +import org.apache.phoenix.end2end.NeedsOwnMiniClusterTest; +import org.apache.phoenix.end2end.ParallelStatsDisabledIT; +import org.apache.phoenix.jdbc.PhoenixConnection; +import org.apache.phoenix.mapreduce.index.IndexTool; +import org.apache.phoenix.query.PhoenixTestBuilder.SchemaBuilder; +import org.apache.phoenix.query.PhoenixTestBuilder.SchemaBuilder.OtherOptions; +import org.apache.phoenix.query.PhoenixTestBuilder.SchemaBuilder.TableOptions; +import org.apache.phoenix.query.QueryConstants; +import org.apache.phoenix.thirdparty.com.google.common.base.Joiner; +import org.apache.phoenix.thirdparty.com.google.common.collect.Lists; +import org.apache.phoenix.thirdparty.com.google.common.collect.Maps; +import org.apache.phoenix.util.CDCUtil; +import org.apache.phoenix.util.EnvironmentEdgeManager; +import org.apache.phoenix.util.ManualEnvironmentEdge; +import org.apache.phoenix.util.PhoenixRuntime; +import org.apache.phoenix.util.ReadOnlyProps; +import org.apache.phoenix.util.SchemaUtil; +import org.apache.phoenix.util.TestUtil; +import org.apache.phoenix.util.TestUtil.CellCount; +import org.apache.phoenix.util.bson.TestFieldValue; +import org.apache.phoenix.util.bson.TestFieldsMap; +import org.bson.BsonDocument; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category(NeedsOwnMiniClusterTest.class) +@RunWith(Parameterized.class) +public class ConditionalTTLExpressionIT extends ParallelStatsDisabledIT { + private static final Logger LOG = LoggerFactory.getLogger(ConditionalTTLExpression.class); + private static final Random RAND = new Random(11); + private static final int MAX_ROWS = 1000; + private static final String[] PK_COLUMNS = {"ID1", "ID2"}; + private static final String[] PK_COLUMN_TYPES = {"VARCHAR", "BIGINT"}; + private static final String[] COLUMNS = { + "VAL1", "VAL2", "VAL3", "VAL4", "VAL5", "VAL6" + }; + private static final String[] COLUMN_TYPES = { + "CHAR(15)", "SMALLINT", "DATE", "TIMESTAMP", "BOOLEAN", "BSON" + }; + private static final String[] DEFAULT_COLUMN_FAMILIES = new String [COLUMNS.length]; + + static { + assert PK_COLUMNS.length == PK_COLUMN_TYPES.length; + assert COLUMNS.length == COLUMN_TYPES.length; + } + + private ManualEnvironmentEdge injectEdge; + private String tableDDLOptions; + private final boolean columnEncoded; + private final Integer tableLevelMaxLookback; + // column names -> fully qualified column names + private SchemaBuilder schemaBuilder; + // map of row-pos -> HBase row-key, used for verification + private Map dataRowPosToKey = Maps.newHashMap(); + private Map indexRowPosToKey = Maps.newHashMap(); + + public ConditionalTTLExpressionIT(boolean columnEncoded, + Integer tableLevelMaxLooback) { + this.columnEncoded = columnEncoded; + this.tableLevelMaxLookback = tableLevelMaxLooback; // in ms + schemaBuilder = new SchemaBuilder(getUrl()); + } + + @Parameterized.Parameters(name = "columnEncoded={0}, tableLevelMaxLookback={1}") + public static synchronized Collection data() { + // maxlookback value is in ms + return Arrays.asList(new Object[][]{ + {false, 0}, + {true, 0}, + {false, 15}, + {true, 15} + }); + } + + @BeforeClass + public static synchronized void doSetup() throws Exception { + Map props = Maps.newHashMapWithExpectedSize(2); + // disabling global max lookback, will use table level max lookback + props.put(BaseScannerRegionObserverConstants.PHOENIX_MAX_LOOKBACK_AGE_CONF_KEY, + Integer.toString(0)); + props.put("hbase.procedure.remote.dispatcher.delay.msec", "0"); + setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator())); + } + + @Before + public void beforeTest() { + StringBuilder optionBuilder = new StringBuilder(); + optionBuilder.append(" TTL = '%s'"); // placeholder for TTL + optionBuilder.append(", MAX_LOOKBACK_AGE=" + tableLevelMaxLookback); + if (columnEncoded) { + optionBuilder.append(", COLUMN_ENCODED_BYTES=2"); + } else { + optionBuilder.append(", COLUMN_ENCODED_BYTES=0"); + } + this.tableDDLOptions = optionBuilder.toString(); + EnvironmentEdgeManager.reset(); + injectEdge = new ManualEnvironmentEdge(); + injectEdge.setValue(EnvironmentEdgeManager.currentTimeMillis()); + } + + @After + public synchronized void afterTest() { + EnvironmentEdgeManager.reset(); + } + + @Test + public void testBasicMaskingAndCompaction() throws Exception { + String ttlCol = "VAL5"; + String ttlExpression = String.format("%s=TRUE", ttlCol); + createTable(ttlExpression); + String tableName = schemaBuilder.getEntityTableName(); + List indexedColumns = Lists.newArrayList("VAL1"); + List includedColumns = Lists.newArrayList(ttlCol); + String indexName = createIndex(indexedColumns, includedColumns, false); + injectEdge(); + int rowCount = 5; + long actual; + try (Connection conn = DriverManager.getConnection(getUrl())) { + populateTable(conn, rowCount); + // populate index row key map + populateRowPosToRowKey(conn, true); + ResultSet rs = readRow(conn, 3); + assertTrue(rs.next()); + assertFalse(rs.getBoolean(ttlCol)); + // expire 1 row by setting to true + updateColumn(conn, 3, ttlCol, true); + actual = TestUtil.getRowCount(conn, tableName, true); + Assert.assertEquals(rowCount - 1, actual); + actual = TestUtil.getRowCountFromIndex(conn, tableName, indexName); + Assert.assertEquals(rowCount - 1, actual); + + // read the row again, this time it should be masked + rs = readRow(conn, 3); + assertFalse(rs.next()); + + // expire 1 more row + updateColumn(conn, 2, ttlCol, true); + actual = TestUtil.getRowCount(conn, tableName, true); + Assert.assertEquals(rowCount - 2, actual); + actual = TestUtil.getRowCountFromIndex(conn, tableName, indexName); + Assert.assertEquals(rowCount - 2, actual); + + // refresh the row again + updateColumn(conn, 3, ttlCol, false); + rs = readRow(conn, 3); + assertTrue(rs.next()); + assertFalse(rs.getBoolean(ttlCol)); + actual = TestUtil.getRowCount(conn, tableName, true); + Assert.assertEquals(rowCount - 1, actual); + actual = TestUtil.getRowCountFromIndex(conn, tableName, indexName); + Assert.assertEquals(rowCount - 1, actual); + + // expire the row again + updateColumn(conn, 3, ttlCol, true); + + // increment by at least 2*maxlookback so that there are no updates within the + // maxlookback window and no updates visible through the maxlookback window + injectEdge.incrementValue(2* tableLevelMaxLookback + 5); + doMajorCompaction(tableName); + CellCount expectedCellCount = new CellCount(); + for (int i = 0; i < rowCount; ++i) { + // additional cell for empty column + expectedCellCount.insertRow(dataRowPosToKey.get(i), COLUMNS.length + 1); + } + // remove the expired rows + expectedCellCount.removeRow(dataRowPosToKey.get(2)); + expectedCellCount.removeRow(dataRowPosToKey.get(3)); + validateTable(conn, tableName, expectedCellCount, dataRowPosToKey.values()); + doMajorCompaction(indexName); + expectedCellCount = new CellCount(); + for (int i = 0; i < rowCount; ++i) { + // additional cell for empty column + expectedCellCount.insertRow(indexRowPosToKey.get(i), includedColumns.size() + 1); + } + // remove the expired rows + expectedCellCount.removeRow(indexRowPosToKey.get(2)); + expectedCellCount.removeRow(indexRowPosToKey.get(3)); + validateTable(conn, indexName, expectedCellCount, indexRowPosToKey.values()); + } + } + + @Test + public void testEverythingRetainedWithinMaxLookBack() throws Exception { + if (tableLevelMaxLookback == 0) { + return; + } + String ttlCol = "VAL5"; + String ttlExpression = String.format("%s=TRUE", ttlCol); + createTable(ttlExpression); + String tableName = schemaBuilder.getEntityTableName(); + injectEdge(); + int rowCount = 5; + long actual; + long startTime = injectEdge.currentTime(); + try (Connection conn = DriverManager.getConnection(getUrl())) { + populateTable(conn, rowCount); + ResultSet rs = readRow(conn, 3); + assertTrue(rs.next()); + assertFalse(rs.getBoolean(ttlCol)); + + // expire 1 row by setting to true + updateColumn(conn, 3, ttlCol, true); + actual = TestUtil.getRowCount(conn, tableName, true); + assertEquals(rowCount - 1, actual); + + // read the row again, this time it should be masked + rs = readRow(conn, 3); + assertFalse(rs.next()); + + // expire 1 more row + updateColumn(conn, 2, ttlCol, true); + actual = TestUtil.getRowCount(conn, tableName, true); + assertEquals(rowCount - 2, actual); + + // refresh the row again + updateColumn(conn, 3, ttlCol, false); + rs = readRow(conn, 3); + assertTrue(rs.next()); + assertFalse(rs.getBoolean(ttlCol)); + actual = TestUtil.getRowCount(conn, tableName, true); + assertEquals(rowCount - 1, actual); + + // expire the row again + updateColumn(conn, 3, ttlCol, true); + + // all the updates are within the maxlookback window + injectEdge.setValue(startTime + tableLevelMaxLookback); + doMajorCompaction(tableName); + CellCount expectedCellCount = new CellCount(); + for (int i = 0; i < rowCount; ++i) { + // additional cell for empty column + expectedCellCount.insertRow(dataRowPosToKey.get(i), COLUMNS.length + 1); + } + // row position 2 updated 1 time 2 cells (column and empty column) + expectedCellCount.addOrUpdateCells(dataRowPosToKey.get(2), 1*2); + // row position 3 updated 3 times + expectedCellCount.addOrUpdateCells(dataRowPosToKey.get(3), 3*2); + validateTable(conn, tableName, expectedCellCount, dataRowPosToKey.values()); + } + } + + @Test + public void testPartialRowRetainedInMaxLookBack() throws Exception { + if (tableLevelMaxLookback == 0) { + return; + } + String ttlCol = "VAL5"; + String ttlExpression = String.format("%s=TRUE", ttlCol, ttlCol); + createTable(ttlExpression); + String tableName = schemaBuilder.getEntityTableName(); + injectEdge(); + int rowCount = 1; + long actual; + injectEdge.currentTime(); + try (Connection conn = DriverManager.getConnection(getUrl())) { + populateTable(conn, rowCount); + // expire 1 row by setting to true + updateColumn(conn, 0, ttlCol, true); + actual = TestUtil.getRowCount(conn, tableName, true); + assertEquals(rowCount - 1, actual); + // all previous updates of the expired row fall out of maxlookback window + injectEdge.incrementValue(tableLevelMaxLookback+5); + // update another column not part of ttl expression + updateColumn(conn, 0, "VAL2", 2345); + // only the last update should be visible in the maxlookback window + injectEdge.incrementValue(1); + doMajorCompaction(tableName); + // the row should still be present because of maxlookback but masked + CellCount expectedCellCount = new CellCount(); + for (int i = 0; i < rowCount; ++i) { + // additional cell for empty column + expectedCellCount.insertRow(dataRowPosToKey.get(i), COLUMNS.length + 1); + } + expectedCellCount.addOrUpdateCells(dataRowPosToKey.get(0), 2); + validateTable(conn, tableName, expectedCellCount, dataRowPosToKey.values()); + // verify that the row is being masked + actual = TestUtil.getRowCount(conn, tableName, true); + assertEquals(rowCount - 1, actual); + // no row versions in maxlookback + injectEdge.incrementValue(tableLevelMaxLookback + 5); + doMajorCompaction(tableName); + expectedCellCount.removeRow(dataRowPosToKey.get(0)); + validateTable(conn, tableName, expectedCellCount, dataRowPosToKey.values()); + } + } + + @Test + public void testPhoenixRowTimestamp() throws Exception { + int ttl = 50; + // equivalent to a ttl of 50ms + String ttlExpression = String.format( + "TO_NUMBER(CURRENT_TIME()) - TO_NUMBER(PHOENIX_ROW_TIMESTAMP()) >= %d", ttl); + createTable(ttlExpression); + String tableName = schemaBuilder.getEntityTableName(); + injectEdge(); + int rowCount = 5; + long actual; + try (Connection conn = DriverManager.getConnection(getUrl())) { + populateTable(conn, rowCount); + + // bump the time so that the ttl expression evaluates to true + injectEdge.incrementValue(ttl); + actual = TestUtil.getRowCount(conn, tableName, true); + assertEquals(0, actual); + + // update VAL3 column of row 1 + updateColumn(conn, 1, "VAL4", injectEdge.currentTime()); + actual = TestUtil.getRowCount(conn, tableName, true); + assertEquals(1, actual); + + // advance the time by more than maxlookbackwindow + injectEdge.incrementValue(tableLevelMaxLookback + 2); + doMajorCompaction(tableName); + CellCount expectedCellCount = new CellCount(); + expectedCellCount.insertRow(dataRowPosToKey.get(1), COLUMNS.length + 1); + validateTable(conn, tableName, expectedCellCount, dataRowPosToKey.values()); + } + } + + @Test + public void testDeleteMarkers() throws Exception { + String ttlCol = "VAL5"; + String ttlExpression = String.format("%s=TRUE", ttlCol); + createTable(ttlExpression); + String tableName = schemaBuilder.getEntityTableName(); + injectEdge(); + int rowCount = 5; + long actual; + try (Connection conn = DriverManager.getConnection(getUrl())) { + populateTable(conn, rowCount); + int [] rowsToDelete = new int[]{2, 3}; + for (int rowPosition : rowsToDelete) { + deleteRow(conn, rowPosition); + } + // expire row # 1 + updateColumn(conn, 1, ttlCol, true); + actual = TestUtil.getRowCount(conn, tableName, true); + // 1 row expired, 2 deleted + assertEquals(2, actual); + if (tableLevelMaxLookback == 0) { + // increment so that all updates are outside of max lookback + injectEdge.incrementValue(2); + doMajorCompaction(tableName); + // only 2 rows should be retained + CellCount expectedCellCount = new CellCount(); + expectedCellCount.insertRow(dataRowPosToKey.get(0), COLUMNS.length + 1); + expectedCellCount.insertRow(dataRowPosToKey.get(4), COLUMNS.length + 1); + validateTable(conn, tableName, expectedCellCount, dataRowPosToKey.values()); + } else { + // all updates within the max lookback window, retain everything + doMajorCompaction(tableName); + CellCount expectedCellCount = new CellCount(); + for (int i = 0; i < rowCount; ++i) { + // additional cell for empty column + expectedCellCount.insertRow(dataRowPosToKey.get(i), COLUMNS.length + 1); + } + // update cell count for expired rows 1 for the column and 1 for empty column + expectedCellCount.addOrUpdateCells(dataRowPosToKey.get(1), 2); + for (int rowPosition : rowsToDelete) { + // one DeleteFamily cell + expectedCellCount.addOrUpdateCell(dataRowPosToKey.get(rowPosition)); + } + validateTable(conn, tableName, expectedCellCount, dataRowPosToKey.values()); + // increment so that the delete markers are outside of max lookback + injectEdge.incrementValue(tableLevelMaxLookback + 1); + doMajorCompaction(tableName); + for (int rowPosition : rowsToDelete) { + expectedCellCount.removeRow(dataRowPosToKey.get(rowPosition)); + } + // purge the expired row also + expectedCellCount.removeRow(dataRowPosToKey.get(1)); + validateTable(conn, tableName, expectedCellCount, dataRowPosToKey.values()); + } + } + } + + @Test + public void testDateExpression() throws Exception { + // ttl = 'CURRENT_DATE() >= VAL3 + 1' // 1 day beyond the value stored in VAL3 + String ttlCol = "VAL3"; + String ttlExpression = String.format("CURRENT_DATE() >= %s + 1", ttlCol); + createTable(ttlExpression); + String tableName = schemaBuilder.getEntityTableName(); + injectEdge(); + int rowCount = 5; + long actual; + try (Connection conn = DriverManager.getConnection(getUrl())) { + populateTable(conn, rowCount); + // bump the time so that the ttl expression evaluates to true + injectEdge.incrementValue(QueryConstants.MILLIS_IN_DAY + 1200); + actual = TestUtil.getRowCount(conn, tableName, true); + assertEquals(0, actual); + + // update column of row 2 + updateColumn(conn, 2, ttlCol, new Date(injectEdge.currentTime())); + actual = TestUtil.getRowCount(conn, tableName, true); + assertEquals(1, actual); + + // advance the time by more than maxlookbackwindow + injectEdge.incrementValue(tableLevelMaxLookback + 2); + doMajorCompaction(tableName); + CellCount expectedCellCount = new CellCount(); + expectedCellCount.insertRow(dataRowPosToKey.get(2), COLUMNS.length + 1); + validateTable(conn, tableName, expectedCellCount, dataRowPosToKey.values()); + } + } + + @Ignore + public void testSCN() throws Exception { + int ttl = 2000; + // equivalent to a ttl of 2s + String ttlExpression = String.format( + "TO_NUMBER(CURRENT_TIME()) - TO_NUMBER(PHOENIX_ROW_TIMESTAMP()) >= %d", ttl); + ttlExpression = String.format("%d", ttl/1000); + createTable(ttlExpression); + createTable(ttlExpression); + String tableName = schemaBuilder.getEntityTableName(); + injectEdge(); + int rowCount = 5; + long actual = 0; + try (Connection conn = DriverManager.getConnection(getUrl())) { + populateTable(conn, rowCount); + } + injectEdge.incrementValue(ttl + rowCount + 1); + Properties props = new Properties(); + long scn = injectEdge.currentTime() - ttl; + props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(scn)); + try (Connection conn = DriverManager.getConnection(getUrl(), props)) { + actual = TestUtil.getRowCount(conn, tableName, true); + assertEquals(0, actual); + } + } + + @Test + public void testRowWithExpressionEvalFailure() throws Exception { + String ttlCol = "VAL2"; + String ttlExpression = String.format("%s > 5", ttlCol); + createTable(ttlExpression); + String tableName = schemaBuilder.getEntityTableName(); + int rowCount = 1; + injectEdge(); + try (Connection conn = DriverManager.getConnection(getUrl())) { + populateTable(conn, rowCount); + // set the ttl column to null so that expression evaluates fail + updateColumn(conn, 0, ttlCol, null); + long actual = TestUtil.getRowCount(conn, tableName, true); + // the row shouldn't be masked + assertEquals(1, actual); + } + } + + @Test + public void testIndexTool() throws Exception { + String ttlCol = "VAL5"; + String ttlExpression = String.format("%s=TRUE", ttlCol); + createTable(ttlExpression); + String fullDataTableName = schemaBuilder.getEntityTableName(); + String schemaName = SchemaUtil.getSchemaNameFromFullName(fullDataTableName); + String tableName = SchemaUtil.getTableNameFromFullName(fullDataTableName); + injectEdge(); + int rowCount = 5; + long actual; + try (Connection conn = DriverManager.getConnection(getUrl())) { + populateTable(conn, rowCount); + deleteRow(conn, 2); + // expire some rows + updateColumn(conn, 0, ttlCol, true); + updateColumn(conn, 4, ttlCol, true); + // now create the index async + String indexName = generateUniqueName(); + String fullIndexName = SchemaUtil.getTableName(schemaName, indexName); + String indexDDL = String.format("create index %s on %s (%s) include (%s) async", + indexName, fullDataTableName, "VAL1", ttlCol); + conn.createStatement().execute(indexDDL); + IndexTool it = IndexToolIT.runIndexTool(false, schemaName, tableName, indexName, + null, 0, IndexTool.IndexVerifyType.BEFORE); + CounterGroup mrJobCounters = IndexToolIT.getMRJobCounters(it); + try { + assertEquals(rowCount - 2, // only the expired rows are masked but not deleted rows + mrJobCounters.findCounter(SCANNED_DATA_ROW_COUNT.name()).getValue()); + assertEquals(rowCount - 2, + mrJobCounters.findCounter(REBUILT_INDEX_ROW_COUNT.name()).getValue()); + assertEquals(0, + mrJobCounters.findCounter( + BEFORE_REBUILD_VALID_INDEX_ROW_COUNT.name()).getValue()); + assertEquals(0, + mrJobCounters.findCounter( + BEFORE_REBUILD_INVALID_INDEX_ROW_COUNT.name()).getValue()); + assertEquals(0, + mrJobCounters.findCounter( + BEFORE_REBUILD_EXPIRED_INDEX_ROW_COUNT.name()).getValue()); + String missingIndexRowCounter = tableLevelMaxLookback != 0 ? + BEFORE_REBUILD_MISSING_INDEX_ROW_COUNT.name() : + BEFORE_REBUILD_BEYOND_MAXLOOKBACK_MISSING_INDEX_ROW_COUNT.name(); + assertEquals(rowCount - 2, + mrJobCounters.findCounter(missingIndexRowCounter).getValue()); + } catch (AssertionError e) { + IndexToolIT.dumpMRJobCounters(mrJobCounters); + throw e; + } + populateRowPosToRowKey(conn, true); + + // Both the tables should have the same row count from Phoenix + actual = TestUtil.getRowCount(conn, fullDataTableName, true); + assertEquals(rowCount -(2+1), actual); // 2 expired, 1 deleted + actual = TestUtil.getRowCountFromIndex(conn, fullDataTableName, fullIndexName); + assertEquals(rowCount -(2+1), actual); + + injectEdge.incrementValue(2*tableLevelMaxLookback + 5); + doMajorCompaction(fullDataTableName); + doMajorCompaction(fullIndexName); + + CellCount expectedCellCount = new CellCount(); + for (int i = 0; i < rowCount; ++i) { + // additional cell for empty column + expectedCellCount.insertRow(dataRowPosToKey.get(i), COLUMNS.length + 1); + } + // remove the expired rows + expectedCellCount.removeRow(dataRowPosToKey.get(0)); + expectedCellCount.removeRow(dataRowPosToKey.get(4)); + // remove the deleted row + expectedCellCount.removeRow(dataRowPosToKey.get(2)); + validateTable(conn, fullDataTableName, expectedCellCount, dataRowPosToKey.values()); + + expectedCellCount = new CellCount(); + for (int i = 0; i < rowCount; ++i) { + // 1 cell for empty column and 1 for included column + expectedCellCount.insertRow(indexRowPosToKey.get(i), 2); + } + // remove the expired rows + expectedCellCount.removeRow(indexRowPosToKey.get(0)); + expectedCellCount.removeRow(indexRowPosToKey.get(4)); + // remove the deleted row + expectedCellCount.removeRow(indexRowPosToKey.get(2)); + validateTable(conn, fullIndexName, expectedCellCount, indexRowPosToKey.values()); + + // run index verification + it = IndexToolIT.runIndexTool(false, schemaName, tableName, indexName, + null, 0, IndexTool.IndexVerifyType.ONLY); + mrJobCounters = IndexToolIT.getMRJobCounters(it); + try { + assertEquals(rowCount -(2+1), // deleted rows and expired rows should be purged + mrJobCounters.findCounter(SCANNED_DATA_ROW_COUNT.name()).getValue()); + assertEquals(0, + mrJobCounters.findCounter(REBUILT_INDEX_ROW_COUNT.name()).getValue()); + assertEquals(rowCount - (2+1), + mrJobCounters.findCounter( + BEFORE_REBUILD_VALID_INDEX_ROW_COUNT.name()).getValue()); + assertEquals(0, + mrJobCounters.findCounter( + BEFORE_REBUILD_INVALID_INDEX_ROW_COUNT.name()).getValue()); + assertEquals(0, + mrJobCounters.findCounter( + BEFORE_REBUILD_EXPIRED_INDEX_ROW_COUNT.name()).getValue()); + assertEquals(0, + mrJobCounters.findCounter( + BEFORE_REBUILD_MISSING_INDEX_ROW_COUNT.name()).getValue()); + } catch (AssertionError e) { + IndexToolIT.dumpMRJobCounters(mrJobCounters); + throw e; + } + } + } + + @Test + public void testLocalIndex() throws Exception { + String ttlCol = "VAL5"; + String ttlExpression = String.format("%s=TRUE", ttlCol); + createTable(ttlExpression); + String fullDataTableName = schemaBuilder.getEntityTableName(); + String schemaName = SchemaUtil.getSchemaNameFromFullName(fullDataTableName); + String tableName = SchemaUtil.getTableNameFromFullName(fullDataTableName); + injectEdge(); + int rowCount = 5; + long actual; + try (Connection conn = DriverManager.getConnection(getUrl())) { + populateTable(conn, rowCount); + deleteRow(conn, 2); + // expire some rows + updateColumn(conn, 0, ttlCol, true); + updateColumn(conn, 4, ttlCol, true); + // now create the index async + String indexName = generateUniqueName(); + String fullIndexName = SchemaUtil.getTableName(schemaName, indexName); + String indexDDL = String.format("create local index %s on %s (%s) include (%s) async", + indexName, fullDataTableName, "VAL1", ttlCol); + conn.createStatement().execute(indexDDL); + IndexToolIT.runIndexTool(false, schemaName, tableName, indexName, + null, 0, IndexTool.IndexVerifyType.BEFORE); + populateRowPosToRowKey(conn, true); + + // Both the tables should have the same row count from Phoenix + actual = TestUtil.getRowCount(conn, fullDataTableName, true); + assertEquals(rowCount -(2+1), actual); // 2 expired, 1 deleted + actual = TestUtil.getRowCountFromIndex(conn, fullDataTableName, fullIndexName); + assertEquals(rowCount -(2+1), actual); + + injectEdge.incrementValue(2*tableLevelMaxLookback + 5); + // local index tables rows are in a separate column family store and will be compacted + doMajorCompaction(fullDataTableName); + + CellCount expectedCellCount = new CellCount(); + for (int i = 0; i < rowCount; ++i) { + // additional cell for empty column + expectedCellCount.insertRow(dataRowPosToKey.get(i), COLUMNS.length + 1); + } + // remove the expired rows + expectedCellCount.removeRow(dataRowPosToKey.get(0)); + expectedCellCount.removeRow(dataRowPosToKey.get(4)); + // remove the deleted row + expectedCellCount.removeRow(dataRowPosToKey.get(2)); + // add the local index row keys + expectedCellCount.insertRow(indexRowPosToKey.get(1), 2); + expectedCellCount.insertRow(indexRowPosToKey.get(3), 2); + + List rowKeys = Stream.concat(dataRowPosToKey.values().stream(), + indexRowPosToKey.values().stream()).collect(Collectors.toList()); + validateTable(conn, fullDataTableName, expectedCellCount, rowKeys); + } + } + + @Test + public void testBsonDataType() throws Exception { + String ttlCol = "VAL6"; + String ttlExpression = String.format( + "BSON_VALUE(%s, ''attr_0'', ''VARCHAR'') IS NULL", ttlCol); + createTable(ttlExpression); + String tableName = schemaBuilder.getEntityTableName(); + injectEdge(); + int rowCount = 5; + long actual; + try (Connection conn = DriverManager.getConnection(getUrl())) { + populateTable(conn, rowCount); + actual = TestUtil.getRowCount(conn, tableName, true); + // only odd rows (1,3) have non null attribute value + assertEquals(2, actual); + + // increment by at least 2*maxlookback so that there are no updates within the + // maxlookback window and no updates visible through the maxlookback window + injectEdge.incrementValue(2* tableLevelMaxLookback + 5); + doMajorCompaction(tableName); + CellCount expectedCellCount = new CellCount(); + for (int i = 0; i < rowCount; ++i) { + // only odd rows should be retained + if (i % 2 != 0) { + // additional cell for empty column + expectedCellCount.insertRow(dataRowPosToKey.get(i), COLUMNS.length + 1); + } + } + validateTable(conn, tableName, expectedCellCount, dataRowPosToKey.values()); + } + } + + @Test + public void testCDCIndex() throws Exception { + String ttlCol = "VAL2"; + // VAL2 = -1 + String ttlExpression = String.format("%s = -1", ttlCol); + createTable(ttlExpression); + String tableName = schemaBuilder.getEntityTableName(); + injectEdge(); + int rowCount = 5; + long actual; + try (Connection conn = DriverManager.getConnection(getUrl())) { + String cdcName = generateUniqueName(); + String cdc_sql = "CREATE CDC " + cdcName + " ON " + tableName; + conn.createStatement().execute(cdc_sql); + populateTable(conn, rowCount); + String schemaName = SchemaUtil.getSchemaNameFromFullName(tableName); + String cdcIndexName = SchemaUtil.getTableName(schemaName, + CDCUtil.getCDCIndexName(cdcName)); + PTable cdcIndex = ((PhoenixConnection) conn).getTableNoCache(cdcIndexName); + assertEquals(cdcIndex.getTTL(), TTLExpression.TTL_EXPRESSION_FORVER); + + // get row count on base table no row should be masked + actual = TestUtil.getRowCount(conn, tableName, true); + assertEquals(rowCount, actual); + + // get raw row count on cdc index table + actual = TestUtil.getRawRowCount(conn, TableName.valueOf(cdcIndexName)); + assertEquals(rowCount, actual); + + // Advance time by the max lookback age. This will cause all rows in cdc index to expire + injectEdge.incrementValue(tableLevelMaxLookback + 2); + + // Major compact the CDC index. This will remove all expired rows + TestUtil.doMajorCompaction(conn, cdcIndexName); + // get raw row count on cdc index table + actual = TestUtil.getRawRowCount(conn, TableName.valueOf(cdcIndexName)); + assertEquals(0, actual); + + // table should still have all the rows intact + actual = TestUtil.getRowCount(conn, tableName, true); + assertEquals(rowCount, actual); + + String alterDDL = String.format("alter table %s set TTL='%s = %d'", tableName, ttlCol, 0); + conn.createStatement().execute(alterDDL); + cdcIndex = ((PhoenixConnection) conn).getTableNoCache(cdcIndexName); + assertEquals(cdcIndex.getTTL(), TTLExpression.TTL_EXPRESSION_FORVER); + } + } + + private void validateTable(Connection conn, + String tableName, + CellCount expectedCellCount, + Collection rowKeys) throws Exception { + + CellCount actualCellCount = TestUtil.getRawCellCount(conn, TableName.valueOf(tableName)); + try { + assertEquals(expectedCellCount, actualCellCount); + } catch (AssertionError e) { + try { + TestUtil.dumpTable(conn, TableName.valueOf(tableName)); + for (String rowKey : rowKeys) { + LOG.info(String.format("Key=%s expected=%d, actual=%d", + Bytes.toStringBinary(rowKey.getBytes()), + expectedCellCount.getCellCount(rowKey), + actualCellCount.getCellCount(rowKey))); + } + } finally { + throw e; + } + } + } + + private void doMajorCompaction(String tableName) throws IOException, InterruptedException { + TestUtil.flush(getUtility(), TableName.valueOf(tableName)); + TestUtil.majorCompact(getUtility(), TableName.valueOf(tableName)); + } + + private void createTable(String ttlExpression) throws Exception { + TableOptions tableOptions = new TableOptions(); + tableOptions.setTablePKColumns(Arrays.asList(PK_COLUMNS)); + tableOptions.setTablePKColumnTypes(Arrays.asList(PK_COLUMN_TYPES)); + tableOptions.setTableColumns(Arrays.asList(COLUMNS)); + tableOptions.setTableColumnTypes(Arrays.asList(COLUMN_TYPES)); + tableOptions.setTableProps(String.format(tableDDLOptions, ttlExpression)); + tableOptions.setMultiTenant(false); + OtherOptions otherOptions = new OtherOptions(); + otherOptions.setTableCFs(Arrays.asList(DEFAULT_COLUMN_FAMILIES)); + schemaBuilder.withTableOptions(tableOptions).withOtherOptions(otherOptions).build(); + } + + private String createIndex(List indexedColumns, + List includedColumns, + boolean isAsync) throws SQLException { + String indexName = "I_" + generateUniqueName(); + String tableName = schemaBuilder.getEntityTableName(); + String schema = SchemaUtil.getSchemaNameFromFullName(tableName); + String indexDDL = String.format("create index %s on %s (%s) include (%s)", + indexName, tableName, + Joiner.on(",").join(indexedColumns), + Joiner.on(",").join(includedColumns)); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(indexDDL); + } + return SchemaUtil.getTableName(schema, indexName); + } + + private void injectEdge() { + long startTime = System.currentTimeMillis() + 1000; + startTime = (startTime / 1000) * 1000; + injectEdge.setValue(startTime); + EnvironmentEdgeManager.injectEdge(injectEdge); + } + + private List generatePKColumnValues(int rowPosition) { + final String ID1_FORMAT = "id1_%d"; + String id1 = String.format(ID1_FORMAT, rowPosition / 2); + int id2 = rowPosition; + return Lists.newArrayList(id1, id2); + } + + private BsonDocument generateBsonDocument(int rowPosition) { + Map map = new HashMap<>(); + if (rowPosition % 2 != 0) { + map.put("attr_0", new TestFieldValue().withS("str_val_" + rowPosition)); + } + map.put("attr_1", new TestFieldValue().withN(rowPosition * rowPosition)); + map.put("attr_2", new TestFieldValue().withBOOL(rowPosition % 2 == 0)); + TestFieldsMap testFieldsMap = new TestFieldsMap(); + testFieldsMap.setMap(map); + return org.apache.phoenix.util.bson.TestUtil.getBsonDocument(testFieldsMap); + } + + private List generateRow(int rowPosition) { + long startTime = EnvironmentEdgeManager.currentTimeMillis(); + List pkCols = generatePKColumnValues(rowPosition); + String val1 = "val1_" + RAND.nextInt(MAX_ROWS); + int val2 = RAND.nextInt(MAX_ROWS); + Date val3 = new Date(startTime + RAND.nextInt(MAX_ROWS)); + Timestamp val4 = new Timestamp(val3.getTime()); + boolean val5 = false; + BsonDocument val6 = generateBsonDocument(rowPosition); + List cols = Lists.newArrayList(val1, val2, val3, val4, val5, val6); + List values = Lists.newArrayListWithExpectedSize(pkCols.size() + cols.size()); + values.addAll(pkCols); + values.addAll(cols); + return values; + } + + private void updateColumn(Connection conn, + int rowPosition, + String columnName, + Object newColumnValue) throws Exception { + String tableName = schemaBuilder.getEntityTableName(); + List upsertColumns = Lists.newArrayList(); + upsertColumns.addAll(Arrays.asList(PK_COLUMNS)); + upsertColumns.add(columnName); + StringBuilder buf = new StringBuilder("UPSERT INTO "); + buf.append(tableName); + buf.append(" (").append(Joiner.on(",").join(upsertColumns)).append(") VALUES("); + for (int i = 0; i < upsertColumns.size(); i++) { + buf.append("?,"); + } + buf.setCharAt(buf.length() - 1, ')'); + List upsertValues = Lists.newArrayList(); + upsertValues.addAll(generatePKColumnValues(rowPosition)); + upsertValues.add(newColumnValue); + try (PreparedStatement stmt = conn.prepareStatement(buf.toString())) { + for (int i = 0; i < upsertValues.size(); i++) { + stmt.setObject(i + 1, upsertValues.get(i)); + } + stmt.executeUpdate(); + conn.commit(); + } + injectEdge.incrementValue(1); + } + + private void updateRow(Connection conn, int rowPosition) throws Exception { + String tableName = schemaBuilder.getEntityTableName(); + List upsertValues = generateRow(rowPosition); + StringBuilder buf = new StringBuilder("UPSERT INTO "); + buf.append(tableName); + buf.append(" VALUES("); + for (int i = 0; i < upsertValues.size(); i++) { + buf.append("?,"); + } + buf.setCharAt(buf.length() - 1, ')'); + try (PreparedStatement stmt = conn.prepareStatement(buf.toString())) { + for (int i = 0; i < upsertValues.size(); i++) { + stmt.setObject(i + 1, upsertValues.get(i)); + } + stmt.executeUpdate(); + conn.commit(); + } + injectEdge.incrementValue(1); + } + + private void deleteRow(Connection conn, int rowPosition) throws SQLException { + String tableName = schemaBuilder.getEntityTableName(); + String dml = String.format("delete from %s where ID1 = ? and ID2 = ?", tableName); + try (PreparedStatement ps = conn.prepareStatement(dml)) { + List pkCols = generatePKColumnValues(rowPosition); + for (int i = 0; i < pkCols.size(); ++i) { + ps.setObject(i + 1, pkCols.get(i)); + } + ps.executeUpdate(); + conn.commit(); + } + injectEdge.incrementValue(1); + } + + private void populateTable(Connection conn, int rowCount) throws Exception { + for (int i = 0; i < rowCount; ++i) { + updateRow(conn, i); + } + // used for verification purposes + populateRowPosToRowKey(conn, false); + } + + /** + * TODO + * @param conn + * @throws SQLException + */ + private void populateRowPosToRowKey(Connection conn, boolean useIndex) throws SQLException { + String tableName = schemaBuilder.getEntityTableName(); + String query = String.format("SELECT %s ID2, ROWKEY_BYTES_STRING() FROM %s", + (useIndex ? "" : "/*+ NO_INDEX */"), tableName); + Map rowPosToKey = useIndex ? indexRowPosToKey : dataRowPosToKey; + try (ResultSet rs = conn.createStatement().executeQuery(query)) { + while (rs.next()) { + int rowPos = rs.getInt(1); + String rowKey = rs.getString(2); // StringBinary format + rowPosToKey.put(rowPos, Bytes.toString(Bytes.toBytesBinary(rowKey))); + } + } + } + + private ResultSet readRow(Connection conn, int rowPosition) throws SQLException { + String tableName = schemaBuilder.getEntityTableName(); + String query = String.format("select * FROM %s where ID1 = ? AND ID2 = ?", tableName); + List pkCols = generatePKColumnValues(rowPosition); + PreparedStatement ps = conn.prepareStatement(query); + for (int i = 0; i < pkCols.size(); ++i) { + ps.setObject(i + 1, pkCols.get(i)); + } + return ps.executeQuery(); + } + +} diff --git a/phoenix-core/src/test/java/org/apache/phoenix/index/VerifySingleIndexRowTest.java b/phoenix-core/src/test/java/org/apache/phoenix/index/VerifySingleIndexRowTest.java index 5873766221b..fb35e7fcec1 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/index/VerifySingleIndexRowTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/index/VerifySingleIndexRowTest.java @@ -69,7 +69,6 @@ public class VerifySingleIndexRowTest extends BaseConnectionlessQueryTest { - private static final int INDEX_TABLE_EXPIRY_SEC = 1; private static final String UNEXPECTED_COLUMN = "0:UNEXPECTED_COLUMN"; public static final String FIRST_ID = "FIRST_ID"; public static final String SECOND_ID = "SECOND_ID"; @@ -272,10 +271,8 @@ private void convertUpsertToMutations(Connection conn) throws SQLException, IOEx } private void initializeRebuildScannerAttributes() throws SQLException { - when(rebuildScanner.setIndexTableTTL(ArgumentMatchers.anyInt())).thenCallRealMethod(); when(rebuildScanner.setIndexMaintainer(ArgumentMatchers.any())).thenCallRealMethod(); when(rebuildScanner.setMaxLookBackInMills(ArgumentMatchers.anyLong())).thenCallRealMethod(); - rebuildScanner.setIndexTableTTL(HConstants.FOREVER); indexMaintainer = pIndexTable.getIndexMaintainer(pDataTable, pconn); rebuildScanner.setIndexMaintainer(indexMaintainer); // set the maxLookBack to infinite to avoid the compaction @@ -367,26 +364,6 @@ public void testVerifySingleIndexRow_allUnverified() throws IOException { } } - @Test - public void testVerifySingleIndexRow_expiredIndexRowCount_nonZero() throws IOException { - IndexToolVerificationResult.PhaseResult - expectedPR = new IndexToolVerificationResult.PhaseResult(0, 1, 0, 0, 0, 0, 0, 0, 0, 0); - try { - for (Map.Entry> - entry : indexKeyToMutationMap.entrySet()) { - initializeLocalMockitoSetup(entry, TestType.EXPIRED); - expireThisRow(); - //test code - rebuildScanner.verifySingleIndexRow(indexRow.getRow(), actualMutationList, - indexKeyToMutationMap.get(indexRow.getRow()), mostRecentIndexRowKeys, Collections.EMPTY_LIST, actualPR, true); - - assertEquals(actualPR, expectedPR); - } - } finally { - EnvironmentEdgeManager.reset(); - } - } - @Test public void testVerifySingleIndexRow_invalidIndexRowCount_cellValue() throws IOException { IndexToolVerificationResult.PhaseResult expectedPR = getInvalidPhaseResult(); @@ -747,12 +724,6 @@ private Cell getCellWithPut(Cell c) { Bytes.toBytes("zxcv")); } - private void expireThisRow() { - rebuildScanner.setIndexTableTTL(INDEX_TABLE_EXPIRY_SEC); - UnitTestClock expiryClock = new UnitTestClock(5000); - EnvironmentEdgeManager.injectEdge(expiryClock); - } - private Mutation getDeleteMutation(Mutation orig, Long ts) { Mutation m = new Delete(orig.getRow()); List origList = orig.getFamilyCellMap().firstEntry().getValue(); diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/ConditionalTTLExpressionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/ConditionalTTLExpressionTest.java new file mode 100644 index 00000000000..f1102367d2e --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/ConditionalTTLExpressionTest.java @@ -0,0 +1,599 @@ +/* + * 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.schema; + +import static org.apache.phoenix.exception.SQLExceptionCode.AGGREGATE_EXPRESSION_NOT_ALLOWED_IN_CONDITIONAL_TTL; +import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_DROP_COL_REFERENCED_IN_CONDITIONAL_TTL; +import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_SET_CONDITIONAL_TTL_ON_TABLE_WITH_MULTIPLE_COLUMN_FAMILIES; +import static org.apache.phoenix.schema.PTableType.INDEX; +import static org.apache.phoenix.util.TestUtil.retainSingleQuotes; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.util.Map; +import java.util.NavigableSet; +import java.util.Set; + +import org.apache.hadoop.hbase.client.Scan; +import org.apache.phoenix.compile.QueryPlan; +import org.apache.phoenix.exception.PhoenixParserException; +import org.apache.phoenix.hbase.index.covered.update.ColumnReference; +import org.apache.phoenix.jdbc.PhoenixConnection; +import org.apache.phoenix.jdbc.PhoenixPreparedStatement; +import org.apache.phoenix.query.BaseConnectionlessQueryTest; +import org.junit.Test; + +public class ConditionalTTLExpressionTest extends BaseConnectionlessQueryTest { + + private static void assertConditonTTL(Connection conn, String tableName, String ttlExpr) throws SQLException { + TTLExpression expected = new ConditionalTTLExpression(ttlExpr); + assertTTL(conn, tableName, expected); + } + + private static void assertTTL(Connection conn, String tableName, TTLExpression expected) throws SQLException { + PTable table = conn.unwrap(PhoenixConnection.class).getTable(tableName); + assertEquals(expected, table.getTTL()); + } + + private void validateScan(Connection conn, + String tableName, + String query, + String ttl, + boolean useIndex, + int expectedNonPKColsInTTLExpr) throws SQLException { + PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class); + PhoenixPreparedStatement pstmt = new PhoenixPreparedStatement(pconn, query); + QueryPlan plan = pstmt.optimizeQuery(); + if (useIndex) { + assertTrue(plan.getTableRef().getTable().getType() == INDEX); + } + plan.iterator(); // create the iterator to initialize the scan + Scan scan = plan.getContext().getScan(); + Map> familyMap = scan.getFamilyMap(); + PTable table = pconn.getTable(tableName); + ConditionalTTLExpression condTTL = (ConditionalTTLExpression) table.getTTL(); + Set columnsReferenced = condTTL.getColumnsReferenced(pconn, table); + assertEquals(expectedNonPKColsInTTLExpr, columnsReferenced.size()); + for (ColumnReference colRef : columnsReferenced) { + NavigableSet set = familyMap.get(colRef.getFamily()); + assertNotNull(set); + assertTrue(set.contains(colRef.getQualifier())); + } + } + + @Test + public void testBasicExpression() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "col1 varchar, col2 date constraint pk primary key (k1,k2 desc)) TTL = '%s'"; + String ttl = "k1 > 5 AND col1 < 'zzzzzz'"; + String tableName = generateUniqueName(); + String ddl = String.format(ddlTemplate, tableName, retainSingleQuotes(ttl)); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + assertConditonTTL(conn, tableName, ttl); + String query = String.format("SELECT count(*) from %s where k1 > 3", tableName); + validateScan(conn, tableName, query, ttl, false, 1); + } + } + + @Test(expected = TypeMismatchException.class) + public void testNotBooleanExpr() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "col1 varchar, col2 date constraint pk primary key (k1,k2 desc)) TTL = '%s'"; + String ttl = "k1 + 100"; + String tableName = generateUniqueName(); + String ddl = String.format(ddlTemplate, tableName, ttl); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + } + } + + @Test(expected = TypeMismatchException.class) + public void testWrongArgumentValue() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "col1 varchar, col2 date constraint pk primary key (k1,k2 desc)) TTL = '%s'"; + String ttl = "k1 = ''abc''"; + String tableName = generateUniqueName(); + String ddl = String.format(ddlTemplate, tableName, ttl); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + } + } + + @Test(expected = PhoenixParserException.class) + public void testParsingError() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "col1 varchar, col2 date constraint pk primary key (k1,k2 desc)) TTL = '%s'"; + String ttl = "k2 == 23"; + String tableName = generateUniqueName(); + String ddl = String.format(ddlTemplate, tableName, ttl); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + } + } + + @Test + public void testMultipleColumnFamilyNotAllowed() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "A.col1 varchar, col2 date constraint pk primary key (k1,k2 desc)) TTL = '%s'"; + String ttl = "A.col1 = 'expired'"; + String tableName = generateUniqueName(); + String ddl = String.format(ddlTemplate, tableName, retainSingleQuotes(ttl)); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + fail(); + } catch (SQLException e) { + assertEquals( + CANNOT_SET_CONDITIONAL_TTL_ON_TABLE_WITH_MULTIPLE_COLUMN_FAMILIES.getErrorCode(), + e.getErrorCode() + ); + } catch (Exception e) { + fail("Unknown exception " + e); + } + } + + @Test + public void testSingleNonDefaultColumnFamilyIsAllowed() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "A.col1 varchar, A.col2 date constraint pk primary key (k1,k2 desc)) TTL = '%s'"; + String ttl = "col1 = 'expired' AND col2 + 10 > CURRENT_DATE()"; + String tableName = generateUniqueName(); + String ddl = String.format(ddlTemplate, tableName, retainSingleQuotes(ttl)); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + assertConditonTTL(conn, tableName, ttl); + // create global index + String indexName = "I_" + generateUniqueName(); + ddl = String.format("create index %s on %s (col2) include(col1)", + indexName, tableName); + conn.createStatement().execute(ddl); + assertConditonTTL(conn, indexName, ttl); + // create local index + indexName = "L_" + generateUniqueName(); + ddl = String.format("create local index %s on %s (col2) include(col1)", + indexName, tableName); + conn.createStatement().execute(ddl); + assertConditonTTL(conn, indexName, ttl); + } + } + + @Test + public void testDefaultColumnFamily() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "col1 varchar, col2 date constraint pk primary key (k1,k2 desc)) TTL = '%s'," + + "DEFAULT_COLUMN_FAMILY='CF'"; + String ttl = "col1 = 'expired' AND CF.col2 + 10 > CURRENT_DATE()"; + String tableName = "T_" + generateUniqueName(); + String ddl = String.format(ddlTemplate, tableName, retainSingleQuotes(ttl)); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + assertConditonTTL(conn, tableName, ttl); + // create view + String viewName = "GV_" + generateUniqueName(); + ddl = String.format("create view %s (col3 varchar) as select * from %s where k1 = 2", + viewName, tableName); + conn.createStatement().execute(ddl); + assertConditonTTL(conn, viewName, ttl); + // create global index + String indexName = "I_" + generateUniqueName(); + ddl = String.format("create index %s on %s (col2) include(col1)", + indexName, tableName); + conn.createStatement().execute(ddl); + assertConditonTTL(conn, indexName, ttl); + // create local index + indexName = "L_" + generateUniqueName(); + ddl = String.format("create local index %s on %s (col2) include(col1)", + indexName, tableName); + conn.createStatement().execute(ddl); + assertConditonTTL(conn, indexName, ttl); + } + } + + @Test + public void testMultipleColumnFamilyNotAllowedOnAlter() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "A.col1 varchar, col2 date constraint pk primary key (k1,k2 desc))"; + String ttl = "A.col1 = 'expired'"; + String tableName = generateUniqueName(); + String ddl = String.format(ddlTemplate, tableName); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + String alterDDL = String.format("alter table %s set TTL = '%s'", + tableName, retainSingleQuotes(ttl)); + conn.createStatement().execute(alterDDL); + fail(); + } catch (SQLException e) { + assertEquals( + CANNOT_SET_CONDITIONAL_TTL_ON_TABLE_WITH_MULTIPLE_COLUMN_FAMILIES.getErrorCode(), + e.getErrorCode() + ); + } catch (Exception e) { + fail("Unknown exception " + e); + } + } + + @Test + public void testMultipleColumnFamilyNotAllowedOnAddColumn() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "A.col1 varchar, A.col2 date constraint pk primary key (k1,k2 desc)) TTL = '%s'"; + String ttl = "A.col1 = 'expired'"; + String tableName = generateUniqueName(); + String ddl = String.format(ddlTemplate, tableName, retainSingleQuotes(ttl)); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + // add a new column in a different column family + String alterDDL = String.format("alter table %s add col3 varchar", tableName); + try { + conn.createStatement().execute(alterDDL); + fail(); + } catch (SQLException e) { + assertEquals(CANNOT_SET_CONDITIONAL_TTL_ON_TABLE_WITH_MULTIPLE_COLUMN_FAMILIES + .getErrorCode(), e.getErrorCode()); + } catch (Exception e) { + fail("Unknown exception " + e); + } + alterDDL = String.format("alter table %s add A.col3 varchar", tableName); + conn.createStatement().execute(alterDDL); + } + } + + @Test + public void testMultipleColumnFamilyNotAllowedOnAddColumn2() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "col1 varchar, col2 date constraint pk primary key (k1,k2 desc)) TTL = '%s'"; + String ttl = "col1 = 'expired'"; + String tableName = generateUniqueName(); + String ddl = String.format(ddlTemplate, tableName, retainSingleQuotes(ttl)); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + // add a new column in a different column family + String alterDDL = String.format("alter table %s add A.col3 varchar", tableName); + try { + conn.createStatement().execute(alterDDL); + fail(); + } catch (SQLException e) { + assertEquals(CANNOT_SET_CONDITIONAL_TTL_ON_TABLE_WITH_MULTIPLE_COLUMN_FAMILIES + .getErrorCode(), e.getErrorCode()); + } catch (Exception e) { + fail("Unknown exception " + e); + } + alterDDL = String.format("alter table %s add col3 varchar", tableName); + conn.createStatement().execute(alterDDL); + } + } + + @Test + public void testMultipleColumnFamilyNotAllowedOnAddColumn3() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "col1 varchar, col2 date constraint pk primary key (k1,k2 desc)) TTL = '%s'," + + "DEFAULT_COLUMN_FAMILY='CF'"; + String ttl = "col1 = 'expired' AND CF.col2 + 10 > CURRENT_DATE()"; + String tableName = generateUniqueName(); + String ddl = String.format(ddlTemplate, tableName, retainSingleQuotes(ttl)); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + assertConditonTTL(conn, tableName, ttl); + // add a new column in a different column family + String alterDDL = String.format("alter table %s add A.col3 varchar", tableName); + try { + conn.createStatement().execute(alterDDL); + fail(); + } catch (SQLException e) { + assertEquals(CANNOT_SET_CONDITIONAL_TTL_ON_TABLE_WITH_MULTIPLE_COLUMN_FAMILIES + .getErrorCode(), e.getErrorCode()); + } catch (Exception e) { + fail("Unknown exception " + e); + } + alterDDL = String.format("alter table %s add col3 varchar", tableName); + conn.createStatement().execute(alterDDL); + } + } + + @Test + public void testDropColumnNotAllowed() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "col1 varchar, col2 date constraint pk primary key (k1,k2 desc)) TTL = '%s'"; + String ttl = "col1 = 'expired' AND col2 + 30 < CURRENT_DATE()"; + String tableName = generateUniqueName(); + String ddl = String.format(ddlTemplate, tableName, retainSingleQuotes(ttl)); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + // drop column referenced in TTL expression + String alterDDL = String.format("alter table %s drop column col2", tableName); + try { + conn.createStatement().execute(alterDDL); + fail(); + } catch (SQLException e) { + assertEquals(CANNOT_DROP_COL_REFERENCED_IN_CONDITIONAL_TTL.getErrorCode(), + e.getErrorCode()); + } catch (Exception e) { + fail("Unknown exception " + e); + } + } + } + + @Test + public void testAggregateExpressionNotAllowed() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "col1 varchar, col2 date constraint pk primary key (k1,k2 desc)) TTL = '%s'"; + String ttl = "SUM(k2) > 23"; + String tableName = generateUniqueName(); + String ddl = String.format(ddlTemplate, tableName, ttl); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + fail(); + } catch (SQLException e) { + assertEquals( + AGGREGATE_EXPRESSION_NOT_ALLOWED_IN_CONDITIONAL_TTL.getErrorCode(), + e.getErrorCode()); + } catch (Exception e) { + fail("Unknown exception " + e); + } + } + + @Test + public void testNullExpression() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "col1 varchar, col2 date constraint pk primary key (k1,k2 desc)) TTL = '%s'"; + String tableName = generateUniqueName(); + String ttl = "col1 is NULL AND col2 < CURRENT_DATE() + 30000"; + String ddl = String.format(ddlTemplate, tableName, ttl); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + assertConditonTTL(conn, tableName, ttl); + String query = String.format("SELECT count(*) from %s", tableName); + validateScan(conn, tableName, query, ttl, false, 2); + } + } + + @Test + public void testBooleanColumn() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "val varchar, expired BOOLEAN constraint pk primary key (k1,k2 desc)) TTL = '%s'"; + String tableName = generateUniqueName(); + String indexName = "I_" + tableName; + String indexTemplate = "create index %s on %s (val) include (expired)"; + String ttl = "expired"; + String query; + try (Connection conn = DriverManager.getConnection(getUrl())) { + String ddl = String.format(ddlTemplate, tableName, ttl); + conn.createStatement().execute(ddl); + assertConditonTTL(conn, tableName, ttl); + + query = String.format("SELECT k1, k2 from %s where (k1,k2) IN ((1,2), (3,4))", + tableName); + validateScan(conn, tableName, query, ttl, false, 1); + + ddl = String.format(indexTemplate, indexName, tableName); + conn.createStatement().execute(ddl); + assertConditonTTL(conn, indexName, ttl); + + // validate the scan on index + query = String.format("SELECT count(*) from %s", tableName); + validateScan(conn, tableName, query, ttl, true, 1); + } + } + + @Test + public void testNot() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "expired BOOLEAN constraint pk primary key (k1,k2 desc)) TTL = '%s'"; + String ttl = "NOT expired"; + String tableName = generateUniqueName(); + String ddl = String.format(ddlTemplate, tableName, ttl); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + assertConditonTTL(conn, tableName, ttl); + } + } + + @Test + public void testPhoenixRowTimestamp() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "col1 varchar, col2 date constraint pk primary key (k1,k2 desc)) TTL = '%s'"; + String tableName = generateUniqueName(); + String ttl = "PHOENIX_ROW_TIMESTAMP() < CURRENT_DATE() - 100"; + String ddl = String.format(ddlTemplate, tableName, ttl); + String query; + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + assertConditonTTL(conn, tableName, ttl); + query = String.format("select col1 from %s where k1 = 7 AND k2 > 12", tableName); + validateScan(conn, tableName, query, ttl, false, 0); + } + } + + @Test + public void testBooleanCaseExpression() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null, k2 bigint not null," + + "col1 varchar, status char(1) constraint pk primary key (k1,k2 desc)) TTL = '%s'"; + String ttl = "CASE WHEN status = ''E'' THEN TRUE ELSE FALSE END"; + String expectedTTLExpr = "CASE WHEN status = 'E' THEN TRUE ELSE FALSE END"; + String tableName = generateUniqueName(); + String ddl = String.format(ddlTemplate, tableName, ttl); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + assertConditonTTL(conn, tableName, expectedTTLExpr); + } + } + + @Test + public void testCondTTLOnTopLevelView() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null primary key," + + "k2 bigint, col1 varchar, status char(1))"; + String tableName = generateUniqueName(); + String viewName = generateUniqueName(); + String viewTemplate = "create view %s (k3 smallint) as select * from %s WHERE k1=7 " + + "TTL = '%s'"; + String ttl = "k2 = 34 and k3 = -1"; + try (Connection conn = DriverManager.getConnection(getUrl())) { + String ddl = String.format(ddlTemplate, tableName); + conn.createStatement().execute(ddl); + ddl = String.format(viewTemplate, viewName, tableName, ttl); + conn.createStatement().execute(ddl); + assertTTL(conn, tableName, TTLExpression.TTL_EXPRESSION_NOT_DEFINED); + assertConditonTTL(conn, viewName, ttl); + String query = String.format("select k3 from %s", viewName); + validateScan(conn, viewName, query, ttl, false, 2); + } + } + + @Test + public void testCondTTLOnMultiLevelView() throws SQLException { + String ddlTemplate = "create table %s (k1 bigint not null primary key," + + "k2 bigint, col1 varchar, status char(1))"; + String tableName = generateUniqueName(); + String parentView = generateUniqueName(); + String childView = generateUniqueName(); + String indexName = generateUniqueName(); + String parentViewTemplate = "create view %s (k3 smallint) as select * from %s WHERE k1=7"; + String childViewTemplate = "create view %s as select * from %s TTL = '%s'"; + String indexOnChildTemplate = "create index %s ON %s (k3) include (col1, k2)"; + String ttl = "k2 = 34 and k3 = -1"; + String ddl = String.format(ddlTemplate, tableName); + try (Connection conn = DriverManager.getConnection(getUrl())) { + conn.createStatement().execute(ddl); + ddl = String.format(parentViewTemplate, parentView, tableName); + conn.createStatement().execute(ddl); + ddl = String.format(childViewTemplate, childView, parentView, ttl); + conn.createStatement().execute(ddl); + assertTTL(conn, tableName, TTLExpression.TTL_EXPRESSION_NOT_DEFINED); + assertTTL(conn, parentView, TTLExpression.TTL_EXPRESSION_NOT_DEFINED); + assertConditonTTL(conn, childView, ttl); + // create an index on child view + ddl = String.format(indexOnChildTemplate, indexName, childView); + conn.createStatement().execute(ddl); + assertConditonTTL(conn, indexName, ttl); + } + } + + @Test + public void testInListTTLExpr() throws Exception { + String ddlTemplate = "create table %s (id varchar not null primary key, " + + "col1 integer, col2 varchar) TTL = '%s'"; + String tableName = generateUniqueName(); + String ttl = "col2 IN ('expired', 'cancelled')"; + String query; + try (Connection conn = DriverManager.getConnection(getUrl())) { + String ddl = String.format(ddlTemplate, tableName, retainSingleQuotes(ttl)); + conn.createStatement().execute(ddl); + assertConditonTTL(conn, tableName, ttl); + query = String.format("select col1 from %s where id IN ('abc', 'fff')", tableName); + validateScan(conn, tableName, query, ttl, false, 1); + } + } + + @Test + public void testPartialIndex() throws Exception { + String ddlTemplate = "create table %s (id varchar not null primary key, " + + "col1 integer, col2 integer, col3 double, col4 varchar) TTL = '%s'"; + String tableName = generateUniqueName(); + String indexTemplate = "create index %s on %s (col1) " + + "include (col2, col3, col4) where col1 > 50"; + String indexName = generateUniqueName(); + String ttl = "col2 > 100 AND col4='expired'"; + String query; + try (Connection conn = DriverManager.getConnection(getUrl())) { + String ddl = String.format(ddlTemplate, tableName, retainSingleQuotes(ttl)); + conn.createStatement().execute(ddl); + assertConditonTTL(conn, tableName, ttl); + ddl = String.format(indexTemplate, indexName, tableName); + conn.createStatement().execute(ddl); + assertConditonTTL(conn, indexName, ttl); + query = String.format("select col3 from %s where col1 > 60", tableName); + validateScan(conn, tableName, query, ttl, false, 2); + } + } + + @Test + public void testUncoveredIndex() throws Exception { + String ddlTemplate = "create table %s (id varchar not null primary key, " + + "col1 integer, col2 integer, col3 double, col4 varchar) TTL = '%s'"; + String tableName = generateUniqueName(); + String indexTemplate = "create uncovered index %s on %s (col1) "; + String indexName = generateUniqueName(); + String ttl = "col2 > 100 AND col4='expired'"; + String query; + try (Connection conn = DriverManager.getConnection(getUrl())) { + String ddl = String.format(ddlTemplate, tableName, retainSingleQuotes(ttl)); + conn.createStatement().execute(ddl); + assertConditonTTL(conn, tableName, ttl); + ddl = String.format(indexTemplate, indexName, tableName); + try { + conn.createStatement().execute(ddl); + fail("Should have thrown ColumnNotFoundException"); + } catch (SQLException e) { + assertTrue(e.getCause() instanceof ColumnNotFoundException); + } + indexTemplate = "create uncovered index %s on %s (col4, col2) "; + ddl = String.format(indexTemplate, indexName, tableName); + conn.createStatement().execute(ddl); + assertConditonTTL(conn, indexName, ttl); + } + } + + @Test + public void testCreatingIndexWithMissingExprCols() throws Exception { + String ddlTemplate = "create table %s (id varchar not null primary key, " + + "col1 integer, col2 integer, col3 double, col4 varchar) TTL = '%s'"; + String tableName = generateUniqueName(); + String indexTemplate = "create index %s on %s (col1) include (col2)"; + String indexName = generateUniqueName(); + String ttl = "col2 > 100 AND col4='expired'"; + try (Connection conn = DriverManager.getConnection(getUrl())) { + String ddl = String.format(ddlTemplate, tableName, retainSingleQuotes(ttl)); + conn.createStatement().execute(ddl); + ddl = String.format(indexTemplate, indexName, tableName); + try { + conn.createStatement().execute(ddl); + fail("Should have thrown ColumnNotFoundException"); + } catch (SQLException e) { + assertTrue(e.getCause() instanceof ColumnNotFoundException); + } + } + } + + @Test + public void testSettingCondTTLOnTableWithIndexWithMissingExprCols() throws Exception { + String ddlTemplate = "create table %s (id varchar not null primary key, " + + "col1 integer, col2 integer, col3 double, col4 varchar)"; + String tableName = generateUniqueName(); + String indexTemplate = "create index %s on %s (col1) include (col2)"; + String indexName = generateUniqueName(); + String ttl = "col2 > 100 AND col4='expired'"; + try (Connection conn = DriverManager.getConnection(getUrl())) { + String ddl = String.format(ddlTemplate, tableName); + conn.createStatement().execute(ddl); + ddl = String.format(indexTemplate, indexName, tableName); + conn.createStatement().execute(ddl); + ddl = String.format("alter table %s set TTL = '%s'", + tableName, retainSingleQuotes(ttl)); + try { + conn.createStatement().execute(ddl); + fail("Should have thrown ColumnNotFoundException"); + } catch (SQLException e) { + assertTrue(e.getCause() instanceof ColumnNotFoundException); + } + } + } +} diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/TTLExpressionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/TTLExpressionTest.java new file mode 100644 index 00000000000..afcb5f9ced4 --- /dev/null +++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/TTLExpressionTest.java @@ -0,0 +1,83 @@ +/* + * 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.schema; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import java.sql.SQLException; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.phoenix.jdbc.PhoenixConnection; +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +public class TTLExpressionTest { + + @Mock + private PhoenixConnection pconn; + @Mock + private PTable table; + + @Before + public void setup() { + MockitoAnnotations.initMocks(this); + } + + @Test + public void testLiteralExpression() { + int ttl = 100; + LiteralTTLExpression literal = new LiteralTTLExpression(ttl); + assertEquals(literal, TTLExpression.create(ttl)); + assertEquals(literal, TTLExpression.create(String.valueOf(ttl))); + } + + @Test + public void testForever() { + assertEquals(TTLExpression.TTL_EXPRESSION_FORVER, + TTLExpression.create(PhoenixDatabaseMetaData.FOREVER_TTL)); + assertEquals(TTLExpression.TTL_EXPRESSION_FORVER, + TTLExpression.create(HConstants.FOREVER)); + } + + @Test + public void testNone() throws SQLException { + assertEquals(TTLExpression.TTL_EXPRESSION_NOT_DEFINED, + TTLExpression.create(PhoenixDatabaseMetaData.NONE_TTL)); + assertEquals(TTLExpression.TTL_EXPRESSION_NOT_DEFINED, + TTLExpression.create(PhoenixDatabaseMetaData.TTL_NOT_DEFINED)); + assertNull(TTLExpression.TTL_EXPRESSION_NOT_DEFINED.getTTLForScanAttribute(pconn, table)); + } + + @Test(expected = IllegalArgumentException.class) + public void testInvalidLiteral() { + TTLExpression.create(-1); + } + + @Test + public void testConditionalExpression() throws SQLException { + String ttl = "PK1 = 5 AND COL1 > 'abc'"; + ConditionalTTLExpression expected = new ConditionalTTLExpression(ttl); + TTLExpression actual = TTLExpression.create(ttl); + assertEquals(expected, actual); + assertEquals(ttl, expected.getTTLExpression()); + } +} diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/ScanUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/ScanUtilTest.java index e088bbfab53..4465d07ab53 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/util/ScanUtilTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/util/ScanUtilTest.java @@ -34,6 +34,7 @@ import org.apache.phoenix.query.KeyRange.Bound; import org.apache.phoenix.query.QueryConstants; import org.apache.phoenix.query.QueryServices; +import org.apache.phoenix.schema.LiteralTTLExpression; import org.apache.phoenix.schema.PDatum; import org.apache.phoenix.schema.PTable; import org.apache.phoenix.schema.PTableKey; @@ -55,6 +56,7 @@ import org.apache.phoenix.thirdparty.com.google.common.base.Function; import org.apache.phoenix.thirdparty.com.google.common.collect.Lists; +import java.io.IOException; import java.sql.Connection; import java.sql.SQLException; import java.util.Arrays; @@ -486,7 +488,7 @@ public SortOrder getSortOrder() { public static class PhoenixTTLScanUtilTest extends BaseConnectionlessQueryTest { @Test - public void testPhoenixTTLUtilMethods() throws SQLException { + public void testPhoenixTTLUtilMethods() throws SQLException, IOException { Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); try (Connection conn = driver.connect(getUrl(), props)) { PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class); @@ -536,7 +538,9 @@ public void testPhoenixTTLUtilMethods() throws SQLException { long timestamp44 = 44L; Scan testScan = new Scan(); - testScan.setAttribute(BaseScannerRegionObserverConstants.TTL, Bytes.toBytes(1L)); + LiteralTTLExpression ttl = new LiteralTTLExpression(1); + byte[] ttlBytes = ttl.toProto(phxConn, table).toByteArray(); + testScan.setAttribute(BaseScannerRegionObserverConstants.TTL, ttlBytes); // Test isTTLExpired Assert.assertTrue(ScanUtil.isTTLExpired(cell42, testScan, timestamp44)); Assert.assertFalse(ScanUtil.isTTLExpired(cell43, testScan, timestamp44)); diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java index 6ea2a2eb656..fa8458e6474 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java @@ -128,6 +128,7 @@ import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; import org.apache.phoenix.jdbc.PhoenixPreparedStatement; +import org.apache.phoenix.jdbc.PhoenixResultSet; import org.apache.phoenix.jdbc.PhoenixStatement; import org.apache.phoenix.parse.FilterableStatement; import org.apache.phoenix.parse.SQLParser; @@ -147,6 +148,7 @@ import org.apache.phoenix.schema.PTableKey; import org.apache.phoenix.schema.RowKeyValueAccessor; import org.apache.phoenix.schema.SortOrder; +import org.apache.phoenix.schema.TTLExpression; import org.apache.phoenix.schema.TableRef; import org.apache.phoenix.schema.stats.GuidePostsInfo; import org.apache.phoenix.schema.stats.GuidePostsKey; @@ -973,7 +975,6 @@ public static void dumpTable(Table table) throws IOException { } public static int getRawRowCount(Table table) throws IOException { - dumpTable(table); return getRowCount(table, true); } @@ -1002,31 +1003,55 @@ public static CellCount getCellCount(Table table, boolean isRaw) throws IOExcept Cell current = null; while (cellScanner.advance()) { current = cellScanner.current(); - cellCount.addCell(Bytes.toString(CellUtil.cloneRow(current))); + cellCount.addOrUpdateCell(Bytes.toString(CellUtil.cloneRow(current))); } } } return cellCount; } - static class CellCount { + public static class CellCount { private Map rowCountMap = new HashMap(); - void addCell(String key) { + public void addOrUpdateCell(String key) { + addOrUpdateCells(key, 1); + } + + public void addOrUpdateCells(String key, int count) { if (rowCountMap.containsKey(key)) { - rowCountMap.put(key, rowCountMap.get(key) + 1); + rowCountMap.put(key, rowCountMap.get(key) + count); } else { - rowCountMap.put(key, 1); + insertRow(key, count); } } - int getCellCount(String key) { + public void insertRow(String key, int count) { + rowCountMap.put(key, count); + } + + public void removeRow(String key) { + rowCountMap.remove(key); + } + + public int getCellCount(String key) { if (rowCountMap.containsKey(key)) { return rowCountMap.get(key); } else { return 0; } } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CellCount other = (CellCount) o; + return rowCountMap.equals(other.rowCountMap); + } } @@ -1206,6 +1231,28 @@ public static long getRowCount(Connection conn, String tableName) throws SQLExce return rs.getLong(1); } + public static long getRowCount(Connection conn, String tableName, boolean skipIndex) + throws SQLException { + String query = String.format("SELECT %s count(*) FROM %s", + (skipIndex ? "/*+ NO_INDEX */" : ""), tableName); + try(ResultSet rs = conn.createStatement().executeQuery(query)) { + assertTrue(rs.next()); + return rs.getLong(1); + } + } + + public static long getRowCountFromIndex(Connection conn, String tableName, String indexName) + throws SQLException { + String query = String.format("SELECT count(*) FROM %s", tableName); + try(ResultSet rs = conn.createStatement().executeQuery(query)) { + PhoenixResultSet prs = rs.unwrap(PhoenixResultSet.class); + String explainPlan = QueryUtil.getExplainPlan(prs.getUnderlyingIterator()); + assertTrue(explainPlan.contains(indexName)); + assertTrue(rs.next()); + return rs.getLong(1); + } + } + public static void addCoprocessor(Connection conn, String tableName, Class coprocessorClass) throws Exception { int priority = QueryServicesOptions.DEFAULT_COPROCESSOR_PRIORITY + 100; ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices(); @@ -1354,14 +1401,15 @@ public static void assertSqlExceptionCode(SQLExceptionCode code, SQLException se public static void assertTableHasTtl(Connection conn, TableName tableName, int ttl, boolean phoenixTTLEnabled) throws SQLException, IOException { - long tableTTL = -1; + TTLExpression tableTTL; if (phoenixTTLEnabled) { tableTTL = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, tableName.getNameAsString())).getTTL(); } else { - tableTTL = getColumnDescriptor(conn, tableName).getTimeToLive(); + tableTTL = TTLExpression.create(getColumnDescriptor(conn, tableName).getTimeToLive()); } - Assert.assertEquals(ttl, tableTTL); + TTLExpression expectedTTL = TTLExpression.create(ttl); + Assert.assertEquals(expectedTTL, tableTTL); } public static void assertTableHasVersions(Connection conn, TableName tableName, int versions) @@ -1397,6 +1445,14 @@ public static int getRawCellCount(Connection conn, TableName tableName, byte[] r CellCount cellCount = getCellCount(table, true); return cellCount.getCellCount(Bytes.toString(row)); } + + public static CellCount getRawCellCount(Connection conn, TableName tableName) + throws IOException, SQLException { + ConnectionQueryServices cqs = conn.unwrap(PhoenixConnection.class).getQueryServices(); + Table table = cqs.getTable(tableName.getName()); + return getCellCount(table, true); + } + public static void assertRawCellCount(Connection conn, TableName tableName, byte[] row, int expectedCellCount) throws SQLException, IOException { @@ -1450,4 +1506,16 @@ public static Path createTempDirectory() throws IOException { return Files.createTempDirectory(Paths.get(System.getProperty("java.io.tmpdir")), null); } + public static String retainSingleQuotes(String input) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < input.length(); ++i) { + char ch = input.charAt(i); + sb.append(ch); + if (ch == '\'') { + sb.append('\''); + } + } + return sb.toString(); + } + } From 7b542c74adde53ecbf400c75907464e2cc7de499 Mon Sep 17 00:00:00 2001 From: Tanuj Khurana Date: Sat, 1 Feb 2025 22:12:49 -0800 Subject: [PATCH 2/2] More fixes and refactoring --- .../phoenix/compile/StatementContext.java | 1 + .../phoenix/iterate/BaseResultIterators.java | 1 + .../schema/ConditionalTTLExpression.java | 38 +++++++++------- .../apache/phoenix/schema/DelegateTable.java | 10 ++++- .../phoenix/schema/LiteralTTLExpression.java | 8 +++- .../apache/phoenix/schema/MetaDataClient.java | 33 +++++++------- .../org/apache/phoenix/schema/PTable.java | 11 ++++- .../org/apache/phoenix/schema/PTableImpl.java | 14 ++++-- .../apache/phoenix/schema/TTLExpression.java | 18 +++++--- .../apache/phoenix/schema/TableProperty.java | 4 +- .../org/apache/phoenix/util/ScanUtil.java | 13 ++++-- .../coprocessor/CompactionScanner.java | 44 ++++++++----------- .../coprocessor/MetaDataEndpointImpl.java | 14 +++--- .../phoenix/coprocessor/TTLRegionScanner.java | 4 +- .../mapreduce/index/IndexScrutinyMapper.java | 4 +- .../DefaultPhoenixMultiViewListProvider.java | 2 +- .../org/apache/phoenix/end2end/CDCBaseIT.java | 2 +- .../apache/phoenix/end2end/CreateTableIT.java | 11 +++-- .../apache/phoenix/end2end/SetPropertyIT.java | 16 +++---- .../phoenix/end2end/TTLAsPhoenixTTLIT.java | 8 ++-- .../org/apache/phoenix/end2end/TTLIT.java | 6 +-- .../org/apache/phoenix/end2end/ViewTTLIT.java | 9 ++-- .../schema/ConditionalTTLExpressionIT.java | 4 +- .../schema/ConditionalTTLExpressionTest.java | 9 ++-- .../phoenix/schema/TTLExpressionTest.java | 4 +- .../org/apache/phoenix/util/TestUtil.java | 2 +- 26 files changed, 166 insertions(+), 124 deletions(-) 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 3226da69fda..d6a13ce52d6 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 @@ -42,6 +42,7 @@ import org.apache.phoenix.schema.PColumn; import org.apache.phoenix.schema.PTable; import org.apache.phoenix.schema.PTableType; +import org.apache.phoenix.schema.TTLExpression; import org.apache.phoenix.schema.TableRef; import org.apache.phoenix.schema.types.PDate; import org.apache.phoenix.schema.types.PTime; diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core-client/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java index 60f11b557d3..4decfdb1571 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java @@ -106,6 +106,7 @@ import org.apache.phoenix.schema.PTableType; import org.apache.phoenix.schema.RowKeySchema; import org.apache.phoenix.schema.StaleRegionBoundaryCacheException; +import org.apache.phoenix.schema.TTLExpression; import org.apache.phoenix.schema.TableRef; import org.apache.phoenix.schema.ValueSchema.Field; import org.apache.phoenix.schema.stats.GuidePostsInfo; diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/ConditionalTTLExpression.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/ConditionalTTLExpression.java index 16ebc080cb4..49e0ae46092 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/ConditionalTTLExpression.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/ConditionalTTLExpression.java @@ -86,12 +86,14 @@ public class ConditionalTTLExpression extends TTLExpression { private final String ttlExpr; // compiled expression according to the table schema. For indexes the expression is // first re-written to use index column references and then compiled. - private Expression compiledExpr; + private final Expression compiledExpr; // columns referenced in the ttl expression to be added to scan - private Set conditionExprColumns; + private final Set conditionExprColumns; public ConditionalTTLExpression(String ttlExpr) { this.ttlExpr = ttlExpr; + this.compiledExpr = null; + this.conditionExprColumns = null; } private ConditionalTTLExpression(String ttlExpr, @@ -102,6 +104,12 @@ private ConditionalTTLExpression(String ttlExpr, this.conditionExprColumns = conditionExprColumns; } + public ConditionalTTLExpression(ConditionalTTLExpression expr) { + this.ttlExpr = expr.ttlExpr; + this.compiledExpr = null; + this.conditionExprColumns = null; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -205,12 +213,14 @@ public boolean isExpired(List result) { } @Override - public synchronized void compileTTLExpression(PhoenixConnection connection, - PTable table) throws SQLException { + public TTLExpression compileTTLExpression(PhoenixConnection connection, + PTable table) throws SQLException { Pair> exprAndCols = buildExpression(connection, table); - compiledExpr = exprAndCols.getFirst(); - conditionExprColumns = exprAndCols.getSecond(); + return new ConditionalTTLExpression( + ttlExpr, + exprAndCols.getFirst(), + exprAndCols.getSecond()); } private Pair> buildExpression( @@ -243,11 +253,10 @@ private Pair> buildExpression( } // Returns the columns referenced in the ttl expression to be added to scan - public synchronized Set getColumnsReferenced( - PhoenixConnection connection, - PTable table) throws SQLException { + public Set getColumnsReferenced() { if (conditionExprColumns == null) { - compileTTLExpression(connection, table); + throw new RuntimeException( + String.format("Conditional TTL Expression %s not compiled", this.ttlExpr)); } return conditionExprColumns; } @@ -290,11 +299,10 @@ public static ConditionalTTLExpression createFromProto(PTableProtos.ConditionTTL @Override public PTableProtos.TTLExpression toProto(PhoenixConnection connection, PTable table) throws SQLException, IOException { - // we want to compile the expression every time we pass it as a scan attribute. This is - // needed so that any stateless expressions like CURRENT_TIME() are always evaluated. - // Otherwise we can cache stale values and keep reusing the stale values which can give - // incorrect results. - compileTTLExpression(connection, table); + if (compiledExpr == null || conditionExprColumns == null) { + throw new RuntimeException( + String.format("Conditional TTL Expression %s not compiled", this.ttlExpr)); + } PTableProtos.TTLExpression.Builder ttl = PTableProtos.TTLExpression.newBuilder(); PTableProtos.ConditionTTL.Builder condition = PTableProtos.ConditionTTL.newBuilder(); condition.setTtlExpression(ttlExpr); 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 8f1ae8d62c2..15f5d240990 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 @@ -381,8 +381,14 @@ public Boolean useStatsForParallelization() { return delegate.hasViewModifiedUseStatsForParallelization(); } - @Override public TTLExpression getTTL() { - return delegate.getTTL(); + @Override public TTLExpression getTTLExpression() { + return delegate.getTTLExpression(); + } + + @Override + public TTLExpression getCompiledTTLExpression(PhoenixConnection connection) + throws SQLException { + return delegate.getCompiledTTLExpression(connection); } @Override diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/LiteralTTLExpression.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/LiteralTTLExpression.java index 92209129bb2..49634e08a48 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/LiteralTTLExpression.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/LiteralTTLExpression.java @@ -23,6 +23,7 @@ import java.util.Objects; import org.apache.hadoop.hbase.Cell; +import org.apache.phoenix.compile.StatementContext; import org.apache.phoenix.coprocessor.generated.PTableProtos; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.parse.CreateTableStatement; @@ -36,6 +37,10 @@ public LiteralTTLExpression(int ttl) { this.ttlValue = ttl; } + public LiteralTTLExpression(LiteralTTLExpression ttlExpr) { + this.ttlValue = ttlExpr.ttlValue; + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -81,7 +86,8 @@ public void validateTTLOnCreate(PhoenixConnection conn, public void validateTTLOnAlter(PhoenixConnection connection, PTable table) {} @Override - public void compileTTLExpression(PhoenixConnection connection, PTable table) { + public TTLExpression compileTTLExpression(PhoenixConnection connection, PTable table) { + return this; } public static LiteralTTLExpression createFromProto(PTableProtos.LiteralTTL literal) { 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 86baae83fce..458220b4673 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 @@ -32,7 +32,7 @@ import static org.apache.phoenix.query.QueryConstants.SYSTEM_SCHEMA_NAME; import static org.apache.phoenix.query.QueryServices.INDEX_CREATE_DEFAULT_STATE; import static org.apache.phoenix.schema.PTableType.CDC; -import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_FORVER; +import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_FOREVER; import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_NOT_DEFINED; import static org.apache.phoenix.thirdparty.com.google.common.collect.Sets.newLinkedHashSet; import static org.apache.phoenix.thirdparty.com.google.common.collect.Sets.newLinkedHashSetWithExpectedSize; @@ -2428,9 +2428,9 @@ private void populateFamilyPropsList(Map familyNames, Map origColumnDefs, ttlAlreadyDefined = checkAndGetTTLFromHierarchy(PhoenixRuntime.getTableNoCache( connection, table.getParentName().toString()), tableName); } - if (ttlAlreadyDefined != TTL_EXPRESSION_NOT_DEFINED) { + if (!ttlAlreadyDefined.equals(TTL_EXPRESSION_NOT_DEFINED)) { throw new SQLExceptionInfo.Builder(SQLExceptionCode. TTL_ALREADY_DEFINED_IN_HIERARCHY) .setSchemaName(schemaName) @@ -5313,9 +5315,10 @@ else if (columnToDrop.isViewReferenced()) { throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_VIEW_REFERENCED_COL) .setColumnName(columnToDrop.getName().getString()).build().buildException(); } else if (table.hasConditionalTTL()) { - ConditionalTTLExpression ttlExpr = (ConditionalTTLExpression) table.getTTL(); + ConditionalTTLExpression ttlExpr = (ConditionalTTLExpression) + table.getCompiledTTLExpression(connection); Set colsReferencedInTTLExpr = - ttlExpr.getColumnsReferenced(connection, table); + ttlExpr.getColumnsReferenced(); ColumnReference colDropRef = new ColumnReference( columnToDrop.getFamilyName() == null ? null : columnToDrop.getFamilyName().getBytes(), @@ -6200,7 +6203,7 @@ private boolean evaluateStmtProperties(MetaProperties metaProperties, .build() .buildException(); } - if (metaProperties.getTTL() != table.getTTL()) { + if (metaProperties.getTTL() != table.getTTLExpression()) { TTLExpression newTTL = metaProperties.getTTL(); newTTL.validateTTLOnAlter(connection, table); metaPropertiesEvaluated.setTTL(metaProperties.getTTL()); 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 e5644013809..7b828fcf72c 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 @@ -937,7 +937,16 @@ IndexMaintainer getIndexMaintainer(PTable dataTable, PTable cdcTable, * @return The TTL expression associated with the entity when Phoenix level TTL is enabled. * The expression can be a Literal value or a boolean Condition. */ - TTLExpression getTTL(); + TTLExpression getTTLExpression(); + + /** + * @return Returns the compiled TTL expression associated with the entity when Phoenix + * level TTL is enabled. For Literal expressions there is no difference between getTTL() + * and getCompiledTTLExpression(). For Conditional TTL, getTTL() returns the expression as + * stored in syscat and the compiled version returns the expression that has been compiled + * taking into account re-writing the expression for indexes. + */ + TTLExpression getCompiledTTLExpression(PhoenixConnection connection) throws SQLException; /** * @return Returns true if table has conditional TTL set 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 7d9f72bfaa7..f841a60cebb 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 @@ -1112,7 +1112,7 @@ public static PTableImpl.Builder builderFromExisting(PTable table) { .setMaxLookbackAge(table.getMaxLookbackAge()) .setCDCIncludeScopes(table.getCDCIncludeScopes()) .setAncestorLastDDLTimestampMap(table.getAncestorLastDDLTimestampMap()) - .setTTL(table.getTTL()) + .setTTL(table.getTTLExpression()) .setRowKeyMatcher(table.getRowKeyMatcher()); } @@ -2295,9 +2295,9 @@ public static PTableProtos.PTable toProto(PTable table) { builder.setCDCIncludeScopes(CDCUtil.makeChangeScopeStringFromEnums( table.getCDCIncludeScopes() != null ? table.getCDCIncludeScopes() : Collections.EMPTY_SET)); - if (table.getTTL() != null) { + if (table.getTTLExpression() != null) { builder.setTtl(ByteStringer.wrap(PVarchar.INSTANCE.toBytes( - table.getTTL().getTTLExpression()))); + table.getTTLExpression().getTTLExpression()))); } if (table.getRowKeyMatcher() != null) { builder.setRowKeyMatcher(ByteStringer.wrap(table.getRowKeyMatcher())); @@ -2398,10 +2398,16 @@ public Boolean useStatsForParallelization() { } @Override - public TTLExpression getTTL() { + public TTLExpression getTTLExpression() { return ttl; } + @Override + public TTLExpression getCompiledTTLExpression(PhoenixConnection connection) + throws SQLException { + return ttl.compileTTLExpression(connection, this); + } + @Override public boolean hasConditionalTTL() { return ttl instanceof ConditionalTTLExpression; diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TTLExpression.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TTLExpression.java index 7512a9d2b17..54b9128d049 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TTLExpression.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TTLExpression.java @@ -31,7 +31,7 @@ public abstract class TTLExpression { - public static final TTLExpression TTL_EXPRESSION_FORVER = + public static final TTLExpression TTL_EXPRESSION_FOREVER = new LiteralTTLExpression(HConstants.FOREVER); public static final TTLExpression TTL_EXPRESSION_NOT_DEFINED = new LiteralTTLExpression(PhoenixDatabaseMetaData.TTL_NOT_DEFINED); @@ -40,7 +40,7 @@ public static TTLExpression create(String ttlExpr) { if (PhoenixDatabaseMetaData.NONE_TTL.equalsIgnoreCase(ttlExpr)) { return TTL_EXPRESSION_NOT_DEFINED; } else if (PhoenixDatabaseMetaData.FOREVER_TTL.equalsIgnoreCase(ttlExpr)) { - return TTL_EXPRESSION_FORVER; + return TTL_EXPRESSION_FOREVER; } else { try { int ttlValue = Integer.parseInt(ttlExpr); @@ -55,12 +55,20 @@ public static TTLExpression create (int ttlValue) { if (ttlValue == PhoenixDatabaseMetaData.TTL_NOT_DEFINED) { return TTL_EXPRESSION_NOT_DEFINED; } else if (ttlValue == HConstants.FOREVER) { - return TTL_EXPRESSION_FORVER; + return TTL_EXPRESSION_FOREVER; } else { return new LiteralTTLExpression(ttlValue); } } + public static TTLExpression create (TTLExpression ttlExpr) { + if (ttlExpr instanceof LiteralTTLExpression) { + return new LiteralTTLExpression((LiteralTTLExpression) ttlExpr); + } else { + return new ConditionalTTLExpression((ConditionalTTLExpression) ttlExpr); + } + } + public static TTLExpression create(byte[] phoenixTTL) throws IOException { return createFromProto(PTableProtos.TTLExpression.parseFrom(phoenixTTL)); } @@ -120,8 +128,8 @@ abstract public void validateTTLOnCreate(PhoenixConnection conn, abstract public void validateTTLOnAlter(PhoenixConnection connection, PTable table) throws SQLException; - abstract public void compileTTLExpression(PhoenixConnection connection, - PTable table) throws SQLException; + abstract public TTLExpression compileTTLExpression( + PhoenixConnection connection, PTable table) throws SQLException; abstract public PTableProtos.TTLExpression toProto( PhoenixConnection connection, PTable table) throws SQLException, IOException; diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TableProperty.java b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TableProperty.java index b182de70ff6..68c21d18caa 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TableProperty.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/schema/TableProperty.java @@ -24,12 +24,10 @@ import static org.apache.phoenix.exception.SQLExceptionCode.SALT_ONLY_ON_CREATE_TABLE; import static org.apache.phoenix.exception.SQLExceptionCode.VIEW_WITH_PROPERTIES; import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME; -import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TTL_NOT_DEFINED; import java.sql.SQLException; import java.util.Map; -import org.apache.hadoop.hbase.HConstants; import org.apache.phoenix.exception.SQLExceptionCode; import org.apache.phoenix.exception.SQLExceptionInfo; import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; @@ -273,7 +271,7 @@ public Object getValue(Object value) { @Override public Object getPTableValue(PTable table) { - return table.getTTL(); + return table.getTTLExpression(); } }, diff --git a/phoenix-core-client/src/main/java/org/apache/phoenix/util/ScanUtil.java b/phoenix-core-client/src/main/java/org/apache/phoenix/util/ScanUtil.java index 3d44c015906..454fd50a424 100644 --- a/phoenix-core-client/src/main/java/org/apache/phoenix/util/ScanUtil.java +++ b/phoenix-core-client/src/main/java/org/apache/phoenix/util/ScanUtil.java @@ -1167,7 +1167,7 @@ public static int getTTL(Scan scan) throws IOException { public static TTLExpression getTTLExpression(Scan scan) throws IOException { byte[] phoenixTTL = scan.getAttribute(BaseScannerRegionObserverConstants.TTL); if (phoenixTTL == null) { - return TTLExpression.TTL_EXPRESSION_FORVER; + return TTLExpression.TTL_EXPRESSION_FOREVER; } return TTLExpression.create(phoenixTTL); } @@ -1441,7 +1441,11 @@ public static void setScanAttributesForPhoenixTTL(Scan scan, PTable table, return; } } - TTLExpression ttlExpr = table.getTTL(); + // we want to compile the expression every time we pass it as a scan attribute. This is + // needed so that any stateless expressions like CURRENT_TIME() are always evaluated. + // Otherwise, we can cache stale values and keep reusing the stale values which can give + // incorrect results. + TTLExpression ttlExpr = table.getCompiledTTLExpression(phoenixConnection); byte[] ttlForScan = ttlExpr.getTTLForScanAttribute(phoenixConnection, table); if (ttlForScan != null) { byte[] emptyColumnFamilyName = SchemaUtil.getEmptyColumnFamily(table); @@ -1490,8 +1494,9 @@ public static void addConditionalTTLColumnsToScan(Scan scan, return; } - ConditionalTTLExpression ttlExpr = (ConditionalTTLExpression) table.getTTL(); - Set colsReferenced = ttlExpr.getColumnsReferenced(connection, table); + ConditionalTTLExpression ttlExpr = + (ConditionalTTLExpression) table.getCompiledTTLExpression(connection); + Set colsReferenced = ttlExpr.getColumnsReferenced(); for (ColumnReference colref : colsReferenced) { // TODO Single Cell scan.addColumn(colref.getFamily(), colref.getQualifier()); diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/CompactionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/CompactionScanner.java index f31154e2271..39b6393ce42 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/CompactionScanner.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/CompactionScanner.java @@ -26,7 +26,7 @@ import static org.apache.phoenix.query.QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX; import static org.apache.phoenix.query.QueryServices.PHOENIX_VIEW_TTL_TENANT_VIEWS_PER_SCAN_LIMIT; import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_PHOENIX_VIEW_TTL_TENANT_VIEWS_PER_SCAN_LIMIT; -import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_FORVER; +import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_FOREVER; import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_NOT_DEFINED; import static org.apache.phoenix.util.ByteUtil.EMPTY_BYTE_ARRAY; @@ -615,7 +615,7 @@ private RowKeyMatcher initializeMatcher(MatcherType type) throws SQLException { if (tableList != null && !tableList.isEmpty()) { tableList.forEach(m -> { - if (m.getTTL() != TTL_EXPRESSION_NOT_DEFINED) { + if (!m.getTTL().equals(TTL_EXPRESSION_NOT_DEFINED)) { // add the ttlInfo to the cache. // each new/unique ttlInfo object added returns a unique tableId. int tableId = -1; @@ -695,7 +695,7 @@ private void refreshMatcher(MatcherType type) throws SQLException { if (tableList != null && !tableList.isEmpty()) { tableList.forEach(m -> { - if (m.getTTL() != TTL_EXPRESSION_NOT_DEFINED) { + if (!m.getTTL().equals(TTL_EXPRESSION_NOT_DEFINED)) { // add the ttlInfo to the cache. // each new/unique ttlInfo object added returns a unique tableId. int tableId = -1; @@ -1079,16 +1079,13 @@ private void getTTLInfo(String physicalTableName, viewIndexIdBytes = PLong.INSTANCE.toBytes(index.getViewIndexId()); } - TTLExpression indexTTL = index.getTTL(); - if (indexTTL instanceof ConditionalTTLExpression) { - indexTTL.compileTTLExpression( - tableConnection.unwrap(PhoenixConnection.class), - index); - } + TTLExpression indexTTL = + index.getCompiledTTLExpression(tableConnection + .unwrap(PhoenixConnection.class)); tableTTLInfoList.add( new TableTTLInfo(pTable.getPhysicalName().getBytes(), tenantIdBytes, index.getTableName().getBytes(), - viewIndexIdBytes, index.getTTL())); + viewIndexIdBytes, indexTTL)); } } @@ -1099,9 +1096,8 @@ private void getTTLInfo(String physicalTableName, configuration)) { PTable pTable = PhoenixRuntime.getTableNoCache( tableConnection, fullTableName); - viewTTL.compileTTLExpression( - tableConnection.unwrap(PhoenixConnection.class), - pTable); + viewTTL = pTable.getCompiledTTLExpression(tableConnection + .unwrap(PhoenixConnection.class)); } } tableTTLInfoList.add( @@ -1240,7 +1236,7 @@ private class TableTTLTrackerForFlushesAndMinor implements TTLTracker { public TableTTLTrackerForFlushesAndMinor(String tableName) { - ttlExpr = TTL_EXPRESSION_FORVER; + ttlExpr = TTL_EXPRESSION_FOREVER; LOGGER.info(String.format( "TableTTLTrackerForFlushesAndMinor params:- " + "(table-name=%s, ttl=%s)", @@ -1268,15 +1264,14 @@ public NonPartitionedTableTTLTracker( Store store) throws IOException { boolean isSystemTable = pTable.getType() == PTableType.SYSTEM; - if (isSystemTable) { - ColumnFamilyDescriptor cfd = store.getColumnFamilyDescriptor(); - ttlExpr = TTLExpression.create(cfd.getTimeToLive()); - } else { - ttlExpr = pTable.getTTL() != TTL_EXPRESSION_NOT_DEFINED - ? pTable.getTTL() : TTL_EXPRESSION_FORVER; - } try { - ttlExpr.compileTTLExpression(pConn, pTable); + if (isSystemTable) { + ColumnFamilyDescriptor cfd = store.getColumnFamilyDescriptor(); + ttlExpr = TTLExpression.create(cfd.getTimeToLive()); + } else { + ttlExpr = !pTable.getTTLExpression().equals(TTL_EXPRESSION_NOT_DEFINED) + ? pTable.getCompiledTTLExpression(pConn) : TTL_EXPRESSION_FOREVER; + } } catch (SQLException e) { throw ClientUtil.createIOException( String.format("Error compiling ttl expression %s", ttlExpr), e); @@ -1325,10 +1320,9 @@ public PartitionedTableTTLTracker( this.tableRowKeyMatcher = new PartitionedTableRowKeyMatcher(table, isSalted, isSharedIndex, isLongViewIndexEnabled, viewTTLTenantViewsPerScanLimit); - this.ttlExpr = table.getTTL() != TTL_EXPRESSION_NOT_DEFINED - ? table.getTTL() : TTL_EXPRESSION_FORVER; try { - this.ttlExpr.compileTTLExpression(pConn, table); + this.ttlExpr = !table.getTTLExpression().equals(TTL_EXPRESSION_NOT_DEFINED) + ? table.getCompiledTTLExpression(pConn) : TTL_EXPRESSION_FOREVER; } catch (SQLException e) { throw ClientUtil.createIOException( String.format("Error compiling ttl expression %s", this.ttlExpr), e); 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 6c0c7a7782e..26d13e053b3 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 @@ -95,7 +95,7 @@ import static org.apache.phoenix.schema.PTableType.CDC; import static org.apache.phoenix.schema.PTableType.INDEX; import static org.apache.phoenix.schema.PTableType.VIEW; -import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_FORVER; +import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_FOREVER; import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_NOT_DEFINED; import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB; import static org.apache.phoenix.util.SchemaUtil.*; @@ -1491,8 +1491,8 @@ private PTable getTableFromCells(List tableCellList, List> allC ttl = TTLExpression.create(ttlStr); } ttl = ttlKv != null ? ttl : oldTable != null - ? oldTable.getTTL() : TTL_EXPRESSION_NOT_DEFINED; - if (tableType == VIEW && viewType != MAPPED && ttl == TTL_EXPRESSION_NOT_DEFINED) { + ? oldTable.getTTLExpression() : TTL_EXPRESSION_NOT_DEFINED; + if (tableType == VIEW && viewType != MAPPED && ttl.equals(TTL_EXPRESSION_NOT_DEFINED)) { //Scan SysCat to get TTL from Parent View/Table byte[] viewKey = SchemaUtil.getTableKey(tenantId == null ? null : tenantId.getBytes(), schemaName == null ? null : schemaName.getBytes(), tableNameBytes); @@ -1679,7 +1679,7 @@ private PTable getTableFromCells(List tableCellList, List> allC builder.setMaxLookbackAge(maxLookbackAge != null ? maxLookbackAge : (oldTable != null ? oldTable.getMaxLookbackAge() : null)); - if (tableType == INDEX && !isThisAViewIndex && ttl == TTL_EXPRESSION_NOT_DEFINED) { + if (tableType == INDEX && !isThisAViewIndex && ttl.equals(TTL_EXPRESSION_NOT_DEFINED)) { //If this is an index on Table get TTL from Table byte[] tableKey = getTableKey(tenantId == null ? null : tenantId.getBytes(), parentSchemaName == null ? null : parentSchemaName.getBytes(), @@ -1688,8 +1688,8 @@ private PTable getTableFromCells(List tableCellList, List> allC } if (tableType == INDEX && CDCUtil.isCDCIndex(tableName.getString()) - && ttl != TTL_EXPRESSION_NOT_DEFINED) { - ttl = TTL_EXPRESSION_FORVER; + && !ttl.equals(TTL_EXPRESSION_NOT_DEFINED)) { + ttl = TTL_EXPRESSION_FOREVER; } builder.setTTL(ttl); builder.setEncodedCQCounter(cqCounter); @@ -3868,7 +3868,7 @@ private boolean validateTTLAttributeSettingForEntity( String newTTLStr = (String) PVarchar.INSTANCE.toObject(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()); TTLExpression newTTL = TTLExpression.create(newTTLStr); - return newTTL != TTL_EXPRESSION_NOT_DEFINED; + return !newTTL.equals(TTL_EXPRESSION_NOT_DEFINED); } } } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/TTLRegionScanner.java b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/TTLRegionScanner.java index a17b5ee8d4f..67098cef019 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/TTLRegionScanner.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/coprocessor/TTLRegionScanner.java @@ -44,7 +44,7 @@ import static org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants.EMPTY_COLUMN_QUALIFIER_NAME; import static org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants.IS_PHOENIX_TTL_SCAN_TABLE_SYSTEM; import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.isPhoenixTableTTLEnabled; -import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_FORVER; +import static org.apache.phoenix.schema.TTLExpression.TTL_EXPRESSION_FOREVER; /** * TTLRegionScanner masks expired rows using the empty column cell timestamp @@ -89,7 +89,7 @@ public TTLRegionScanner(final RegionCoprocessorEnvironment env, final Scan scan, // an older client and does not supply the empty column parameters, the masking should not // be done here. We also disable masking when TTL is HConstants.FOREVER. isMaskingEnabled = emptyCF != null && emptyCQ != null - && ttlExpression != TTL_EXPRESSION_FORVER + && !ttlExpression.equals(TTL_EXPRESSION_FOREVER) && (isPhoenixTableTTLEnabled(env.getConfiguration()) && (isSystemTable == null || !Bytes.toBoolean(isSystemTable))); } diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyMapper.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyMapper.java index 925edc9ac53..8083e059f78 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyMapper.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/index/IndexScrutinyMapper.java @@ -345,8 +345,8 @@ private int getTableTTL(Configuration configuration) throws SQLException, IOExce SchemaUtil.isNamespaceMappingEnabled(null, cqsi.getProps())); if (configuration.getBoolean(QueryServices.PHOENIX_TABLE_TTL_ENABLED, QueryServicesOptions.DEFAULT_PHOENIX_TABLE_TTL_ENABLED)) { - return pSourceTable.getTTL() == TTL_EXPRESSION_NOT_DEFINED ? DEFAULT_TTL - : ((LiteralTTLExpression) pSourceTable.getTTL()).getTTLValue(); // TODO + return pSourceTable.getTTLExpression().equals(TTL_EXPRESSION_NOT_DEFINED) ? DEFAULT_TTL + : ((LiteralTTLExpression) pSourceTable.getTTLExpression()).getTTLValue(); // TODO } else { TableDescriptor tableDesc; try (Admin admin = cqsi.getAdmin()) { diff --git a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/DefaultPhoenixMultiViewListProvider.java b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/DefaultPhoenixMultiViewListProvider.java index 9c4f6093a8a..480ae3b3f2b 100644 --- a/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/DefaultPhoenixMultiViewListProvider.java +++ b/phoenix-core-server/src/main/java/org/apache/phoenix/mapreduce/util/DefaultPhoenixMultiViewListProvider.java @@ -139,7 +139,7 @@ private boolean isParentHasTTL(PhoenixConnection connection, PTable parentTable = connection.getTable(null, pTable.getParentName().toString()); System.out.println("Parent Table"); if (parentTable.getType() == PTableType.VIEW && - parentTable.getTTL() != null) { + parentTable.getTTLExpression() != null) { /* if the current view parent already has a TTL value, we want to skip the current view cleanup job because we want to run the cleanup job for at the GlobalView level instead of running multi-jobs at diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CDCBaseIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CDCBaseIT.java index 46332469c8f..a50570baded 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CDCBaseIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CDCBaseIT.java @@ -221,7 +221,7 @@ protected void assertPTable(String cdcName, Set expInclud assertEquals(cdcTable.getPhysicalName().getString(), tableName == datatableName ? indexFullName : getViewIndexPhysicalName(datatableName)); PTable cdcIndexTable = PhoenixRuntime.getTable(conn, indexFullName); - assertEquals(cdcIndexTable.getTTL(), TTLExpression.TTL_EXPRESSION_FORVER); + assertEquals(cdcIndexTable.getTTLExpression(), TTLExpression.TTL_EXPRESSION_FOREVER); } protected void assertSaltBuckets(Connection conn, String tableName, Integer nbuckets) 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 7ca41f5338f..45ded7c03b1 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 @@ -45,7 +45,6 @@ import java.util.List; import java.util.Map; import java.util.Properties; -import java.util.UUID; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.KeepDeletedCells; @@ -479,7 +478,7 @@ public void testCreateTableColumnFamilyHBaseAttribs1() throws Exception { assertEquals(86400, columnFamilies[0].getTimeToLive()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable assertEquals(new LiteralTTLExpression(86400), conn.unwrap(PhoenixConnection.class).getTable( - new PTableKey(null, tableName)).getTTL()); + new PTableKey(null, tableName)).getTTLExpression()); } @Test @@ -538,7 +537,7 @@ public void testCreateTableColumnFamilyHBaseAttribs2() throws Exception { assertEquals("C", columnFamilies[1].getNameAsString()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable assertEquals(new LiteralTTLExpression(86400), conn.unwrap(PhoenixConnection.class).getTable( - new PTableKey(null, tableName)).getTTL()); + new PTableKey(null, tableName)).getTTLExpression()); } /** @@ -567,7 +566,7 @@ public void testCreateTableColumnFamilyHBaseAttribs3() throws Exception { assertEquals(86400, columnFamilies[1].getTimeToLive()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable assertEquals(new LiteralTTLExpression(86400), conn.unwrap(PhoenixConnection.class).getTable( - new PTableKey(null, tableName)).getTTL()); + new PTableKey(null, tableName)).getTTLExpression()); } /** @@ -648,7 +647,7 @@ public void testCreateTableColumnFamilyHBaseAttribs6() throws Exception { assertEquals(10000, columnFamilies[0].getTimeToLive()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable assertEquals(new LiteralTTLExpression(10000), conn.unwrap(PhoenixConnection.class).getTable( - new PTableKey(null, tableName)).getTTL()); + new PTableKey(null, tableName)).getTTLExpression()); } /** @@ -673,7 +672,7 @@ public void testCreateTableColumnFamilyHBaseAttribs7() throws Exception { assertEquals(10000, columnFamilies[0].getTimeToLive()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable assertEquals(new LiteralTTLExpression(10000), conn.unwrap(PhoenixConnection.class).getTable( - new PTableKey(null, tableName)).getTTL()); + new PTableKey(null, tableName)).getTTLExpression()); } @Test diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SetPropertyIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SetPropertyIT.java index 0a487a5df55..176ec27c0c0 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SetPropertyIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SetPropertyIT.java @@ -431,7 +431,7 @@ public void testSettingPropertiesWhenTableHasDefaultColFamilySpecified() throws assertEquals(Boolean.toString(false), tableDesc.getValue(TableDescriptorBuilder.COMPACTION_ENABLED)); //Check if Alter Table TTL also changes TTL stored in SYSCAT with phoenix.table.ttl disabled assertEquals(new LiteralTTLExpression(1000), conn.unwrap(PhoenixConnection.class).getTable( - new PTableKey(null, dataTableFullName)).getTTL()); + new PTableKey(null, dataTableFullName)).getTTLExpression()); } } @@ -794,7 +794,7 @@ public void testSetTTLForTableWithOnlyPKCols() throws Exception { assertEquals(86400, columnFamilies[0].getTimeToLive()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable assertEquals(new LiteralTTLExpression(86400), conn.unwrap(PhoenixConnection.class).getTable( - new PTableKey(null, dataTableFullName)).getTTL()); + new PTableKey(null, dataTableFullName)).getTTLExpression()); } ddl = "ALTER TABLE " + dataTableFullName + " SET TTL=30"; conn.createStatement().execute(ddl); @@ -807,7 +807,7 @@ public void testSetTTLForTableWithOnlyPKCols() throws Exception { assertEquals("XYZ", columnFamilies[0].getNameAsString()); //Check if Alter Table TTL also changes TTL stored in SYSCAT with phoenix.table.ttl disabled assertEquals(new LiteralTTLExpression(30), conn.unwrap(PhoenixConnection.class).getTable( - new PTableKey(null, dataTableFullName)).getTTL()); + new PTableKey(null, dataTableFullName)).getTTLExpression()); } } finally { conn.close(); @@ -835,7 +835,7 @@ public void testSetTTLForTableWithForeverAndNoneValue() throws Exception { assertEquals(HConstants.FOREVER, columnFamilies[0].getTimeToLive()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable assertEquals(TTLExpression.TTL_EXPRESSION_NOT_DEFINED, conn.unwrap(PhoenixConnection.class).getTable( - new PTableKey(null, dataTableFullName)).getTTL()); + new PTableKey(null, dataTableFullName)).getTTLExpression()); } ddl = "ALTER TABLE " + dataTableFullName + " SET TTL=FOREVER"; conn.createStatement().execute(ddl); @@ -847,8 +847,8 @@ public void testSetTTLForTableWithForeverAndNoneValue() throws Exception { assertEquals(HConstants.FOREVER, columnFamilies[0].getTimeToLive()); assertEquals("XYZ", columnFamilies[0].getNameAsString()); //Check if Alter Table TTL also changes TTL stored in SYSCAT with phoenix.table.ttl disabled - assertEquals(TTLExpression.TTL_EXPRESSION_FORVER, conn.unwrap(PhoenixConnection.class).getTable( - new PTableKey(null, dataTableFullName)).getTTL()); + assertEquals(TTLExpression.TTL_EXPRESSION_FOREVER, conn.unwrap(PhoenixConnection.class).getTable( + new PTableKey(null, dataTableFullName)).getTTLExpression()); } } finally { conn.close(); @@ -997,7 +997,7 @@ public void testTTLAssignmentForNewEmptyCF() throws Exception { assertEquals(86400, columnFamilies[1].getTimeToLive()); //Check if TTL is stored in SYSCAT as well and we are getting ttl from get api in PTable assertEquals(new LiteralTTLExpression(86400), conn.unwrap(PhoenixConnection.class).getTable( - new PTableKey(null, dataTableFullName)).getTTL()); + new PTableKey(null, dataTableFullName)).getTTLExpression()); } ddl = "ALTER TABLE " + dataTableFullName + " SET TTL=1000"; @@ -1015,7 +1015,7 @@ public void testTTLAssignmentForNewEmptyCF() throws Exception { assertEquals(1000, columnFamilies[1].getTimeToLive()); //Check if Alter Table TTL also changes TTL stored in SYSCAT with phoenix.table.ttl disabled assertEquals(new LiteralTTLExpression(1000), conn.unwrap(PhoenixConnection.class).getTable( - new PTableKey(null, dataTableFullName)).getTTL()); + new PTableKey(null, dataTableFullName)).getTTLExpression()); } // the new column will be assigned to the column family XYZ. With the a KV column getting added for XYZ, diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLAsPhoenixTTLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLAsPhoenixTTLIT.java index 56ce1b80799..e35864fbc68 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLAsPhoenixTTLIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLAsPhoenixTTLIT.java @@ -194,7 +194,7 @@ public void testCreateAndAlterTableDDLWithForeverAndNoneTTLValues() throws Excep try (Connection conn = DriverManager.getConnection(getUrl())) { conn.createStatement().execute(ddl); assertTTLValue(conn.unwrap(PhoenixConnection.class), - TTLExpression.TTL_EXPRESSION_FORVER, tableName); + TTLExpression.TTL_EXPRESSION_FOREVER, tableName); ddl = "ALTER TABLE " + tableName + " SET TTL=NONE"; conn.createStatement().execute(ddl); @@ -220,7 +220,7 @@ public void testCreateAndAlterTableDDLWithForeverAndNoneTTLValues() throws Excep ddl = "ALTER TABLE " + tableName + " SET TTL=FOREVER"; conn.createStatement().execute(ddl); assertTTLValue(conn.unwrap(PhoenixConnection.class), - TTLExpression.TTL_EXPRESSION_FORVER, tableName); + TTLExpression.TTL_EXPRESSION_FOREVER, tableName); //Setting TTL should not be stored as CF Descriptor properties when //phoenix.table.ttl.enabled is true columnFamilies = @@ -559,11 +559,11 @@ public void testAlteringTTLAtOneLevelAndCheckingAtAnotherLevel() throws Exceptio private void assertTTLValue(PhoenixConnection conn, TTLExpression expected, String name) throws SQLException { assertEquals("TTL value did not match :-", expected, - PhoenixRuntime.getTableNoCache(conn, name).getTTL()); + PhoenixRuntime.getTableNoCache(conn, name).getTTLExpression()); } private void assertTTLValue(PTable table, TTLExpression expected) { - assertEquals("TTL value did not match :-", expected, table.getTTL()); + assertEquals("TTL value did not match :-", expected, table.getTTLExpression()); } private String createTableWithOrWithOutTTLAsItsProperty(Connection conn, boolean withTTL) throws SQLException { diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLIT.java index b901082d711..e0ecb205e45 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TTLIT.java @@ -138,21 +138,21 @@ private SchemaBuilder createTenantViewOnTableOrGlobalView(SchemaBuilder schemaBu private void assertTTLForGivenPTable(PTable table, int ttl) { LiteralTTLExpression expected = new LiteralTTLExpression(ttl); - Assert.assertEquals(expected, table.getTTL()); + Assert.assertEquals(expected, table.getTTLExpression()); } private void assertTTLForGivenEntity(Connection connection, String entityName, int ttl) throws SQLException { PTable pTable = PhoenixRuntime.getTable(connection, entityName); LiteralTTLExpression expected = new LiteralTTLExpression(ttl); - Assert.assertEquals(expected, pTable.getTTL()); + Assert.assertEquals(expected, pTable.getTTLExpression()); } private void assertTTLForIndexName(Connection connection, String indexName, int ttl) throws SQLException { if (!indexName.equals(SKIP_ASSERT)) { PTable index = PhoenixRuntime.getTable(connection, indexName); LiteralTTLExpression expected = new LiteralTTLExpression(ttl); - Assert.assertEquals(expected, index.getTTL()); + Assert.assertEquals(expected, index.getTTLExpression()); } } diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java index 02b47771b9a..8f23ddf4e4a 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewTTLIT.java @@ -43,7 +43,6 @@ import java.util.Random; import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.phoenix.compile.QueryPlan; import org.apache.phoenix.coprocessorclient.BaseScannerRegionObserverConstants; import org.apache.phoenix.exception.SQLExceptionCode; @@ -87,8 +86,6 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; @Category(NeedsOwnMiniClusterTest.class) public class ViewTTLIT extends BaseViewTTLIT { @@ -1607,7 +1604,7 @@ public void testScanAttributes() throws Exception { queryPlan.getContext()); Assert.assertFalse("Should not have any rows", rs.next()); Assert.assertEquals("Should have atleast one element", 1, queryPlan.getScans().size()); - Assert.assertEquals("PhoenixTTL does not match", table.getTTL(), + Assert.assertEquals("PhoenixTTL does not match", table.getTTLExpression(), ScanUtil.getTTLExpression(queryPlan.getScans().get(0).get(0))); Assert.assertTrue("Masking attribute should be set", ScanUtil.isMaskTTLExpiredRows(queryPlan.getScans().get(0).get(0))); @@ -1643,7 +1640,7 @@ public void testScanAttributes() throws Exception { scan.setAttribute(BaseScannerRegionObserverConstants.DELETE_PHOENIX_TTL_EXPIRED, PDataType.TRUE_BYTES); scan.setAttribute(BaseScannerRegionObserverConstants.TTL, - table.getTTL().getTTLForScanAttribute(conn, table)); + table.getCompiledTTLExpression(conn).getTTLForScanAttribute(conn, table)); PhoenixResultSet rs = @@ -1651,7 +1648,7 @@ public void testScanAttributes() throws Exception { queryPlan.getContext()); Assert.assertFalse("Should not have any rows", rs.next()); Assert.assertEquals("Should have atleast one element", 1, queryPlan.getScans().size()); - Assert.assertEquals("PhoenixTTL does not match", table.getTTL(), + Assert.assertEquals("PhoenixTTL does not match", table.getTTLExpression(), ScanUtil.getTTLExpression(queryPlan.getScans().get(0).get(0))); Assert.assertFalse("Masking attribute should not be set", ScanUtil.isMaskTTLExpiredRows(queryPlan.getScans().get(0).get(0))); diff --git a/phoenix-core/src/it/java/org/apache/phoenix/schema/ConditionalTTLExpressionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/schema/ConditionalTTLExpressionIT.java index ebac0e052d2..8e886b15005 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/schema/ConditionalTTLExpressionIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/schema/ConditionalTTLExpressionIT.java @@ -732,7 +732,7 @@ public void testCDCIndex() throws Exception { String cdcIndexName = SchemaUtil.getTableName(schemaName, CDCUtil.getCDCIndexName(cdcName)); PTable cdcIndex = ((PhoenixConnection) conn).getTableNoCache(cdcIndexName); - assertEquals(cdcIndex.getTTL(), TTLExpression.TTL_EXPRESSION_FORVER); + assertEquals(cdcIndex.getTTLExpression(), TTLExpression.TTL_EXPRESSION_FOREVER); // get row count on base table no row should be masked actual = TestUtil.getRowCount(conn, tableName, true); @@ -758,7 +758,7 @@ public void testCDCIndex() throws Exception { String alterDDL = String.format("alter table %s set TTL='%s = %d'", tableName, ttlCol, 0); conn.createStatement().execute(alterDDL); cdcIndex = ((PhoenixConnection) conn).getTableNoCache(cdcIndexName); - assertEquals(cdcIndex.getTTL(), TTLExpression.TTL_EXPRESSION_FORVER); + assertEquals(cdcIndex.getTTLExpression(), TTLExpression.TTL_EXPRESSION_FOREVER); } } diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/ConditionalTTLExpressionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/ConditionalTTLExpressionTest.java index f1102367d2e..a165d38200f 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/schema/ConditionalTTLExpressionTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/ConditionalTTLExpressionTest.java @@ -52,7 +52,7 @@ private static void assertConditonTTL(Connection conn, String tableName, String private static void assertTTL(Connection conn, String tableName, TTLExpression expected) throws SQLException { PTable table = conn.unwrap(PhoenixConnection.class).getTable(tableName); - assertEquals(expected, table.getTTL()); + assertEquals(expected, table.getTTLExpression()); } private void validateScan(Connection conn, @@ -70,9 +70,10 @@ private void validateScan(Connection conn, plan.iterator(); // create the iterator to initialize the scan Scan scan = plan.getContext().getScan(); Map> familyMap = scan.getFamilyMap(); - PTable table = pconn.getTable(tableName); - ConditionalTTLExpression condTTL = (ConditionalTTLExpression) table.getTTL(); - Set columnsReferenced = condTTL.getColumnsReferenced(pconn, table); + PTable table = plan.getTableRef().getTable(); + ConditionalTTLExpression condTTL = + (ConditionalTTLExpression) table.getCompiledTTLExpression(pconn); + Set columnsReferenced = condTTL.getColumnsReferenced(); assertEquals(expectedNonPKColsInTTLExpr, columnsReferenced.size()); for (ColumnReference colRef : columnsReferenced) { NavigableSet set = familyMap.get(colRef.getFamily()); diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/TTLExpressionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/TTLExpressionTest.java index afcb5f9ced4..8484d1d159d 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/schema/TTLExpressionTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/TTLExpressionTest.java @@ -52,9 +52,9 @@ public void testLiteralExpression() { @Test public void testForever() { - assertEquals(TTLExpression.TTL_EXPRESSION_FORVER, + assertEquals(TTLExpression.TTL_EXPRESSION_FOREVER, TTLExpression.create(PhoenixDatabaseMetaData.FOREVER_TTL)); - assertEquals(TTLExpression.TTL_EXPRESSION_FORVER, + assertEquals(TTLExpression.TTL_EXPRESSION_FOREVER, TTLExpression.create(HConstants.FOREVER)); } diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java index fa8458e6474..320a9223de7 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java @@ -1404,7 +1404,7 @@ public static void assertTableHasTtl(Connection conn, TableName tableName, int t TTLExpression tableTTL; if (phoenixTTLEnabled) { tableTTL = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, - tableName.getNameAsString())).getTTL(); + tableName.getNameAsString())).getTTLExpression(); } else { tableTTL = TTLExpression.create(getColumnDescriptor(conn, tableName).getTimeToLive()); }