From 0aa1d9c3677517db18f726ac446ef58324ec87e3 Mon Sep 17 00:00:00 2001 From: alpass163 Date: Tue, 20 Jan 2026 18:09:14 +0800 Subject: [PATCH 1/6] support that the user could specify the name of time column(columnCategory is time)(#17048) --- .../persistence/schema/ConfigMTree.java | 12 +-- .../TableInsertTabletStatementGenerator.java | 10 +- .../operator/process/TableIntoOperator.java | 7 +- .../config/TableConfigTaskVisitor.java | 76 ++++++++------- .../plan/planner/TableOperatorGenerator.java | 63 +++++++++++- .../analyzer/ExpressionAnalyzer.java | 97 +++++++++++++++++++ .../analyzer/StatementAnalyzer.java | 24 +++-- .../fetcher/TableHeaderSchemaValidator.java | 7 +- .../relational/sql/parser/AstBuilder.java | 80 ++++----------- .../schema/table/InformationSchema.java | 19 ---- .../schema/table/TsFileTableSchemaUtil.java | 9 +- .../iotdb/commons/schema/table/TsTable.java | 81 +++++++++------- .../table/column/TsTableColumnCategory.java | 4 + pom.xml | 2 +- 14 files changed, 310 insertions(+), 181 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ConfigMTree.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ConfigMTree.java index 2725053b84398..6b47773813e31 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ConfigMTree.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ConfigMTree.java @@ -35,7 +35,6 @@ import org.apache.iotdb.commons.schema.table.TableNodeStatus; import org.apache.iotdb.commons.schema.table.TreeViewSchema; import org.apache.iotdb.commons.schema.table.TsTable; -import org.apache.iotdb.commons.schema.table.column.TimeColumnSchema; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.commons.schema.table.column.TsTableColumnSchema; import org.apache.iotdb.commons.utils.MetadataUtils; @@ -98,7 +97,6 @@ import static org.apache.iotdb.commons.schema.SchemaConstant.NON_TEMPLATE; import static org.apache.iotdb.commons.schema.SchemaConstant.ROOT; import static org.apache.iotdb.commons.schema.SchemaConstant.TABLE_MNODE_TYPE; -import static org.apache.iotdb.commons.schema.table.TsTable.TIME_COLUMN_NAME; // Since the ConfigMTree is all stored in memory, thus it is not restricted to manage MNode through // MTreeStore. @@ -803,20 +801,14 @@ public void setTableColumnComment( throws MetadataException { final TsTable table = getTable(database, tableName); - final TsTableColumnSchema columnSchema = - !columnName.equals(TIME_COLUMN_NAME) || Objects.isNull(comment) - ? table.getColumnSchema(columnName) - : new TimeColumnSchema(TIME_COLUMN_NAME, TSDataType.TIMESTAMP); + final TsTableColumnSchema columnSchema = table.getColumnSchema(columnName); + if (Objects.isNull(columnSchema)) { throw new ColumnNotExistsException( PathUtils.unQualifyDatabaseName(database.getFullPath()), tableName, columnName); } if (Objects.nonNull(comment)) { columnSchema.getProps().put(TsTable.COMMENT_KEY, comment); - if (columnName.equals("time")) { - // Replace the original time column - table.addColumnSchema(columnSchema); - } } else { columnSchema.getProps().remove(TsTable.COMMENT_KEY); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TableInsertTabletStatementGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TableInsertTabletStatementGenerator.java index a8987d652fc57..aed6cd3a4c714 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TableInsertTabletStatementGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TableInsertTabletStatementGenerator.java @@ -35,8 +35,6 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicLong; -import static org.apache.iotdb.commons.schema.table.TsTable.TIME_COLUMN_NAME; - public class TableInsertTabletStatementGenerator extends InsertTabletStatementGenerator { private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(TableInsertTabletStatementGenerator.class); @@ -54,13 +52,14 @@ public TableInsertTabletStatementGenerator( List inputColumnTypes, List tsTableColumnCategories, boolean isAligned, - int rowLimit) { + int rowLimit, + String timeColumnName) { super( targetTable, measurementToDataTypeMap.keySet().toArray(new String[0]), measurementToDataTypeMap.values().toArray(new TSDataType[0]), measurementToInputLocationMap.entrySet().stream() - .filter(entry -> !entry.getKey().equalsIgnoreCase(TIME_COLUMN_NAME)) + .filter(entry -> !entry.getKey().equalsIgnoreCase(timeColumnName)) .map(Map.Entry::getValue) .toArray(InputLocation[]::new), inputColumnTypes.stream().map(TypeFactory::getType).toArray(Type[]::new), @@ -69,8 +68,7 @@ public TableInsertTabletStatementGenerator( this.databaseName = databaseName; this.writtenCounter = new AtomicLong(0); this.columnCategories = tsTableColumnCategories.toArray(new TsTableColumnCategory[0]); - this.timeColumnIndex = - measurementToInputLocationMap.get(TIME_COLUMN_NAME).getValueColumnIndex(); + this.timeColumnIndex = measurementToInputLocationMap.get(timeColumnName).getValueColumnIndex(); this.reset(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TableIntoOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TableIntoOperator.java index 40b1781b69d4b..bf6f9c5bc94af 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TableIntoOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/process/TableIntoOperator.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.queryengine.execution.operator.Operator; import org.apache.iotdb.db.queryengine.execution.operator.OperatorContext; import org.apache.iotdb.db.queryengine.plan.planner.plan.parameter.InputLocation; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertTabletStatement; import com.google.common.util.concurrent.Futures; @@ -67,7 +68,8 @@ public TableIntoOperator( Map measurementToDataTypeMap, boolean isAligned, ExecutorService intoOperationExecutor, - long statementSizePerLine) { + long statementSizePerLine, + ColumnSchema timeColumnOfTargetTable) { super(operatorContext, child, inputColumnTypes, intoOperationExecutor, statementSizePerLine); this.maxReturnSize = MAX_RETURN_SIZE; insertTabletStatementGenerator = @@ -79,7 +81,8 @@ public TableIntoOperator( inputColumnTypes, inputColumnCategories, isAligned, - maxRowNumberInStatement); + maxRowNumberInStatement, + timeColumnOfTargetTable.getName()); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java index 008cb7946fc98..e65991600b110 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java @@ -580,8 +580,22 @@ private Pair parseTable4CreateTableOrView( table.addProp(TsTable.COMMENT_KEY, node.getComment()); } + // check if the time column has been specified + long timeColumnCount = + node.getElements().stream() + .filter( + columnDefinition -> + columnDefinition.getColumnCategory() == TsTableColumnCategory.TIME) + .count(); + if (timeColumnCount > 1) { + throw new SemanticException("A table cannot have more than one time column"); + } + if (timeColumnCount == 0) { + // append the time column with default name "time" if user do not specify the time column + table.addColumnSchema(new TimeColumnSchema(TIME_COLUMN_NAME, TSDataType.TIMESTAMP)); + } + // TODO: Place the check at statement analyzer - boolean hasTimeColumn = false; final Set sourceNameSet = new HashSet<>(); boolean hasObject = false; for (final ColumnDefinition columnDefinition : node.getElements()) { @@ -590,15 +604,18 @@ private Pair parseTable4CreateTableOrView( final TSDataType dataType = getDataType(columnDefinition.getType()); hasObject |= dataType == TSDataType.OBJECT; final String comment = columnDefinition.getComment(); - if (checkTimeColumnIdempotent(category, columnName, dataType, comment, table) - && !hasTimeColumn) { - hasTimeColumn = true; - continue; - } + if (table.getColumnSchema(columnName) != null) { throw new SemanticException( String.format("Columns in table shall not share the same name %s.", columnName)); } + + // allow the user create time column + if (category == TsTableColumnCategory.TIME) { + validateAndGenerateTimeColumn(columnName, dataType, comment, table); + continue; + } + final TsTableColumnSchema schema = TableHeaderSchemaValidator.generateColumnSchema( category, @@ -627,6 +644,25 @@ private Pair parseTable4CreateTableOrView( return new Pair<>(database, table); } + private void validateAndGenerateTimeColumn( + final String columnName, + final TSDataType dataType, + final String comment, + final TsTable table) { + + if (dataType == TSDataType.TIMESTAMP) { + final TsTableColumnSchema timeColumnSchema = + new TimeColumnSchema(columnName, TSDataType.TIMESTAMP); + if (Objects.nonNull(comment)) { + timeColumnSchema.getProps().put(TsTable.COMMENT_KEY, comment); + } + table.addColumnSchema(timeColumnSchema); + + } else { + throw new SemanticException("The time column's type shall be 'timestamp'."); + } + } + @Override protected IConfigTask visitAlterColumnDataType( AlterColumnDataType node, MPPQueryContext context) { @@ -647,34 +683,6 @@ protected IConfigTask visitAlterColumnDataType( node.isView()); } - private boolean checkTimeColumnIdempotent( - final TsTableColumnCategory category, - final String columnName, - final TSDataType dataType, - final String comment, - final TsTable table) { - if (category == TsTableColumnCategory.TIME || columnName.equals(TIME_COLUMN_NAME)) { - if (category == TsTableColumnCategory.TIME - && columnName.equals(TIME_COLUMN_NAME) - && dataType == TSDataType.TIMESTAMP) { - if (Objects.nonNull(comment)) { - final TsTableColumnSchema columnSchema = - new TimeColumnSchema(TIME_COLUMN_NAME, TSDataType.TIMESTAMP); - columnSchema.getProps().put(TsTable.COMMENT_KEY, comment); - table.addColumnSchema(columnSchema); - } - return true; - } else if (dataType == TSDataType.TIMESTAMP) { - throw new SemanticException( - "The time column category shall be bounded with column name 'time'."); - } else { - throw new SemanticException("The time column's type shall be 'timestamp'."); - } - } - - return false; - } - @Override protected IConfigTask visitRenameTable(final RenameTable node, final MPPQueryContext context) { context.setQueryType(QueryType.WRITE); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java index 407b3154b1f2a..cdc7c8cc6586d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/TableOperatorGenerator.java @@ -234,6 +234,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.FunctionCall; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Literal; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LongLiteral; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.SymbolReference; import org.apache.iotdb.db.queryengine.plan.relational.type.InternalTypeManager; import org.apache.iotdb.db.queryengine.plan.statement.component.Ordering; import org.apache.iotdb.db.queryengine.transformation.dag.column.ColumnTransformer; @@ -3803,13 +3804,18 @@ public Operator visitInto(IntoNode node, LocalExecutionPlanContext context) { int index = map.get(originInputColumnNames.get(i)); inputColumns.set(index, originColumns.get(i)); } - + ColumnSchema timeColumnOfTargetTable = null; for (int i = 0; i < inputColumns.size(); i++) { String columnName = inputColumns.get(i).getName(); inputLocationMap.put(columnName, new InputLocation(0, i)); TsTableColumnCategory columnCategory = inputColumns.get(i).getColumnCategory(); if (columnCategory == TIME) { + if (timeColumnOfTargetTable == null) { + timeColumnOfTargetTable = inputColumns.get(i); + } else { + throw new SemanticException("Multiple columns with TIME category found"); + } continue; } @@ -3818,6 +3824,9 @@ public Operator visitInto(IntoNode node, LocalExecutionPlanContext context) { inputColumnTypes.add(columnType); inputColumnCategories.add(columnCategory); } + if (timeColumnOfTargetTable == null) { + throw new SemanticException("Missing TIME category column"); + } long statementSizePerLine = OperatorGeneratorUtil.calculateStatementSizePerLine(inputColumnTypes); @@ -3833,7 +3842,8 @@ public Operator visitInto(IntoNode node, LocalExecutionPlanContext context) { tsDataTypeMap, true, FragmentInstanceManager.getInstance().getIntoOperationExecutor(), - statementSizePerLine); + statementSizePerLine, + timeColumnOfTargetTable); } private boolean[] checkStatisticAndScanOrder( @@ -3878,6 +3888,22 @@ private boolean[] checkStatisticAndScanOrder( break; } + // first and last, the second argument has to be the time column + if (FIRST_AGGREGATION.equals(funcName) || LAST_AGGREGATION.equals(funcName)) { + if (!isTimeColumn(aggregation.getArguments().get(1), timeColumnName)) { + canUseStatistic = false; + break; + } + } + + // first_by and last_by, the second argument has to be the time column + if (FIRST_BY_AGGREGATION.equals(funcName) || LAST_BY_AGGREGATION.equals(funcName)) { + if (!isTimeColumn(aggregation.getArguments().get(2), timeColumnName)) { + canUseStatistic = false; + break; + } + } + // only last_by(time, x) or last_by(x,time) can use statistic if ((LAST_BY_AGGREGATION.equals(funcName) || FIRST_BY_AGGREGATION.equals(funcName)) && !isTimeColumn(aggregation.getArguments().get(0), timeColumnName) @@ -3923,6 +3949,12 @@ private OptimizeType canUseLastCacheOptimize( return OptimizeType.NOOP; } + // if the timeColumnName is null, the param of function is just a timestamp column other than + // the time column + if (timeColumnName == null || !checkOrderColumnIsTime(node.getAggregations(), timeColumnName)) { + return OptimizeType.NOOP; + } + if (canUseLastRowOptimize(aggregators)) { return OptimizeType.LAST_ROW; } @@ -3934,6 +3966,33 @@ private OptimizeType canUseLastCacheOptimize( return OptimizeType.NOOP; } + /** + * Checks if the ordering column in aggregations matches the time column. only check for + * FIRST/LAST/FIRST_BY/LAST_BY + */ + private boolean checkOrderColumnIsTime( + Map aggregations, String timeColumnName) { + + for (Map.Entry entry : aggregations.entrySet()) { + String functionName = + entry.getValue().getResolvedFunction().getSignature().getName().toLowerCase(); + List arguments = entry.getValue().getArguments(); + Expression lastParam = entry.getValue().getArguments().get(arguments.size() - 1); + + switch (functionName) { + case FIRST_AGGREGATION: + case LAST_AGGREGATION: + case FIRST_BY_AGGREGATION: + case LAST_BY_AGGREGATION: + if (!((SymbolReference) lastParam).getName().equalsIgnoreCase(timeColumnName)) { + return false; + } + break; + } + } + return true; + } + private boolean canUseLastRowOptimize(List aggregators) { for (TableAggregator aggregator : aggregators) { if (aggregator.getAccumulator() instanceof LastDescAccumulator) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/ExpressionAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/ExpressionAnalyzer.java index 663c92ad83e97..32d6d94460948 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/ExpressionAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/ExpressionAnalyzer.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.queryengine.plan.relational.analyzer; +import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.db.exception.sql.SemanticException; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.SessionInfo; @@ -154,12 +155,17 @@ import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.WindowFrame.Type.ROWS; import static org.apache.iotdb.db.queryengine.plan.relational.type.TypeSignatureTranslator.toTypeSignature; import static org.apache.iotdb.db.queryengine.plan.relational.utils.NodeUtils.getSortItemsFromOrderBy; +import static org.apache.iotdb.db.utils.constant.SqlConstant.FIRST_AGGREGATION; +import static org.apache.iotdb.db.utils.constant.SqlConstant.FIRST_BY_AGGREGATION; +import static org.apache.iotdb.db.utils.constant.SqlConstant.LAST_AGGREGATION; +import static org.apache.iotdb.db.utils.constant.SqlConstant.LAST_BY_AGGREGATION; import static org.apache.tsfile.read.common.type.BlobType.BLOB; import static org.apache.tsfile.read.common.type.BooleanType.BOOLEAN; import static org.apache.tsfile.read.common.type.DoubleType.DOUBLE; import static org.apache.tsfile.read.common.type.IntType.INT32; import static org.apache.tsfile.read.common.type.LongType.INT64; import static org.apache.tsfile.read.common.type.StringType.STRING; +import static org.apache.tsfile.read.common.type.TimestampType.TIMESTAMP; import static org.apache.tsfile.read.common.type.UnknownType.UNKNOWN; public class ExpressionAnalyzer { @@ -1051,6 +1057,38 @@ protected Type visitFunctionCall( throw new SemanticException("DISTINCT is not supported for non-aggregation functions"); } + int argumentsNum = node.getArguments().size(); + RelationType relationType = context.getContext().getScope().getRelationType(); + // Syntactic sugar: first(s1) => first(s1,time), first_by(s1,s2) => first_by(s1,s2,time) + // So do last and last_by. + switch (functionName.toLowerCase()) { + case FIRST_AGGREGATION: + case LAST_AGGREGATION: + if (argumentsNum == 1) { + addTimeArgument(node.getArguments(), getActualTimeFieldName(relationType)); + } else if (argumentsNum == 2) { + if (!checkArgumentIsTimestamp( + node.getArguments().get(1), (List) relationType.getVisibleFields())) { + throw new SemanticException( + String.format( + "The second argument of %s function must be actual time name", functionName)); + } + } + break; + case FIRST_BY_AGGREGATION: + case LAST_BY_AGGREGATION: + if (argumentsNum == 2) { + addTimeArgument(node.getArguments(), getActualTimeFieldName(relationType)); + } else if (argumentsNum == 3) { + if (!checkArgumentIsTimestamp( + node.getArguments().get(2), (List) relationType.getVisibleFields())) { + throw new SemanticException( + String.format( + "The third argument of %s function must be actual time name", functionName)); + } + } + } + List argumentTypes = getCallArgumentTypes(node.getArguments(), context); if (node.getArguments().size() > 127) { @@ -1132,6 +1170,65 @@ public List getCallArgumentTypes( return argumentTypesBuilder.build(); } + private void addTimeArgument(List arguments, String actualTimeField) { + + if (arguments.get(0) instanceof DereferenceExpression) { + arguments.add( + new DereferenceExpression( + ((DereferenceExpression) arguments.get(0)).getBase(), + new Identifier(actualTimeField.toLowerCase(Locale.ENGLISH)))); + } else { + arguments.add(new Identifier(actualTimeField.toLowerCase(Locale.ENGLISH))); + } + } + + private boolean checkArgumentIsTimestamp(Expression argument, List visibleFields) { + + String argumentName = + (argument instanceof DereferenceExpression) + ? ((DereferenceExpression) argument) + .getField() + .orElseThrow(() -> new SemanticException("the input field do not exists")) + .toString() + : argument.toString(); + + for (Field field : visibleFields) { + if (field + .getName() + .orElseThrow(() -> new SemanticException("the field in table do not hava the name")) + .equalsIgnoreCase(argumentName)) { + return field.getType() == TIMESTAMP; + } + } + // should never reach here + throw new SemanticException("the input argument do not exists"); + } + + /** Retrieves the effective time column name from the relation's visible fields. */ + private String getActualTimeFieldName(RelationType relation) { + + // Priority 1: Try to find a column explicitly marked as TIME category + Optional timeColumn = + relation.getVisibleFields().stream() + .filter(field -> field.getColumnCategory() == TsTableColumnCategory.TIME) + .findFirst() + .flatMap(Field::getName); + + if (timeColumn.isPresent()) { + return timeColumn.get(); + } + + // Priority 2: Fallback to the first TIMESTAMP column (e.g., for system schema compatibility) + return relation.getVisibleFields().stream() + .filter(field -> field.getType() == TIMESTAMP) + .findFirst() + .flatMap(Field::getName) + .orElseThrow( + () -> + new SemanticException( + "Missing valid time column. The table must contain either a column with the TIME category or at least one TIMESTAMP column.")); + } + private Type analyzeMatchNumber( FunctionCall node, StackableAstVisitorContext context) { if (!node.getArguments().isEmpty()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java index e9df62193c28e..ae15c70100b70 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/analyzer/StatementAnalyzer.java @@ -269,7 +269,6 @@ import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; import static org.apache.iotdb.commons.schema.table.TsTable.TABLE_ALLOWED_PROPERTIES; -import static org.apache.iotdb.commons.schema.table.TsTable.TIME_COLUMN_NAME; import static org.apache.iotdb.commons.udf.builtin.relational.TableBuiltinScalarFunction.DATE_BIN; import static org.apache.iotdb.db.queryengine.execution.warnings.StandardWarningCode.REDUNDANT_ORDER_BY; import static org.apache.iotdb.db.queryengine.plan.relational.analyzer.AggregationAnalyzer.verifyOrderByAggregations; @@ -684,10 +683,23 @@ protected Scope visitInsert(Insert insert, Optional scope) { .collect(toImmutableList()); analysis.registerTable(insert.getTable(), tableSchema, targetTable); - LinkedHashSet tableColumns = - columns.stream() - .map(ColumnSchema::getName) - .collect(Collectors.toCollection(LinkedHashSet::new)); + LinkedHashSet tableColumns = new LinkedHashSet<>(); + String actualTimeColumnName = null; + for (ColumnSchema column : columns) { + tableColumns.add(column.getName().toLowerCase(ENGLISH)); + + if (column.getColumnCategory() == TsTableColumnCategory.TIME) { + if (actualTimeColumnName != null) { + throw new SemanticException( + "Multiple columns found with TIME category in table schema"); + } + actualTimeColumnName = column.getName(); + } + } + if (actualTimeColumnName == null) { + throw new SemanticException("Target table schema misses a TIME category column"); + } + LinkedHashSet insertColumns; if (insert.getColumns().isPresent()) { insertColumns = @@ -712,7 +724,7 @@ protected Scope visitInsert(Insert insert, Optional scope) { } // insert columns should contain time - if (!insertColumns.contains(TIME_COLUMN_NAME)) { + if (!insertColumns.contains(actualTimeColumnName)) { throw new SemanticException("time column can not be null"); } // insert columns should contain at least one field column diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableHeaderSchemaValidator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableHeaderSchemaValidator.java index 99da3d005f721..2d629cb26ee93 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableHeaderSchemaValidator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/metadata/fetcher/TableHeaderSchemaValidator.java @@ -28,6 +28,7 @@ import org.apache.iotdb.commons.schema.table.column.AttributeColumnSchema; import org.apache.iotdb.commons.schema.table.column.FieldColumnSchema; import org.apache.iotdb.commons.schema.table.column.TagColumnSchema; +import org.apache.iotdb.commons.schema.table.column.TimeColumnSchema; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.commons.schema.table.column.TsTableColumnSchema; import org.apache.iotdb.db.auth.AuthorityChecker; @@ -718,8 +719,10 @@ public static TsTableColumnSchema generateColumnSchema( schema = new AttributeColumnSchema(columnName, dataType); break; case TIME: - throw new SemanticException( - "Create table or add column statement shall not specify column category TIME"); + // throw new SemanticException("Add column statement shall not specify column category + // TIME"); + schema = new TimeColumnSchema(columnName, dataType); + break; case FIELD: schema = dataType != TSDataType.UNKNOWN diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java index 5d529818a75e6..1ad212cf560af 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/parser/AstBuilder.java @@ -34,7 +34,6 @@ import org.apache.iotdb.db.exception.query.QueryProcessException; import org.apache.iotdb.db.exception.sql.SemanticException; import org.apache.iotdb.db.protocol.session.IClientSession; -import org.apache.iotdb.db.queryengine.plan.expression.leaf.TimestampOperand; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AddColumn; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AliasedRelation; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AllColumns; @@ -291,7 +290,6 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -305,7 +303,6 @@ import static java.util.Objects.requireNonNull; import static java.util.stream.Collectors.toList; import static org.apache.iotdb.commons.schema.column.ColumnHeaderConstant.DATA_NODE_ID_TABLE_MODEL; -import static org.apache.iotdb.commons.schema.table.TsTable.TIME_COLUMN_NAME; import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.ATTRIBUTE; import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.FIELD; import static org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory.TAG; @@ -338,10 +335,6 @@ import static org.apache.iotdb.db.utils.constant.SqlConstant.APPROX_COUNT_DISTINCT; import static org.apache.iotdb.db.utils.constant.SqlConstant.APPROX_MOST_FREQUENT; import static org.apache.iotdb.db.utils.constant.SqlConstant.APPROX_PERCENTILE; -import static org.apache.iotdb.db.utils.constant.SqlConstant.FIRST_AGGREGATION; -import static org.apache.iotdb.db.utils.constant.SqlConstant.FIRST_BY_AGGREGATION; -import static org.apache.iotdb.db.utils.constant.SqlConstant.LAST_AGGREGATION; -import static org.apache.iotdb.db.utils.constant.SqlConstant.LAST_BY_AGGREGATION; public class AstBuilder extends RelationalSqlBaseVisitor { @@ -855,15 +848,28 @@ private Node visitInsertValues( final List columnNames = identifiers.stream().map(Identifier::getValue).collect(toList()); int timeColumnIndex = -1; - for (int i = 0; i < columnNames.size(); i++) { - if (TIME_COLUMN_NAME.equalsIgnoreCase(columnNames.get(i))) { - if (timeColumnIndex == -1) { - timeColumnIndex = i; - } else { - throw new SemanticException("One row should only have one time value"); + + // retrieve the table schema to identify the actual time column + TsTable table = DataNodeTableCache.getInstance().getTable(databaseName, tableName); + List timeColumnCandidates = + table.getColumnList().stream() + .filter(col -> col.getColumnCategory() == TIME) + .collect(toList()); + if (timeColumnCandidates.size() != 1) { + throw new SemanticException("the table should only have one column found with TIME category"); + } else { + // locate the time column index in the input identifiers if time column exists in the schema + for (int i = 0; i < columnNames.size(); i++) { + if (timeColumnCandidates.get(0).getColumnName().equalsIgnoreCase(columnNames.get(i))) { + if (timeColumnIndex == -1) { + timeColumnIndex = i; + } else { + throw new SemanticException("One row should only have one time value"); + } } } } + if (timeColumnIndex != -1) { columnNames.remove(timeColumnIndex); } @@ -3417,29 +3423,7 @@ public Node visitFunctionCall(RelationalSqlParser.FunctionCallContext ctx) { new DereferenceExpression(getLocation(ctx.label), (Identifier) visit(ctx.label))); } - // Syntactic sugar: first(s1) => first(s1,time), first_by(s1,s2) => first_by(s1,s2,time) - // So do last and last_by. - if (name.toString().equalsIgnoreCase(FIRST_AGGREGATION) - || name.toString().equalsIgnoreCase(LAST_AGGREGATION)) { - if (arguments.size() == 1) { - appendTimeArgument(arguments); - } else if (arguments.size() == 2) { - check( - checkArgumentIsTime(arguments.get(1)), - "The second argument of 'first' or 'last' function must be 'time'", - ctx); - } - } else if (name.toString().equalsIgnoreCase(FIRST_BY_AGGREGATION) - || name.toString().equalsIgnoreCase(LAST_BY_AGGREGATION)) { - if (arguments.size() == 2) { - appendTimeArgument(arguments); - } else if (arguments.size() == 3) { - check( - checkArgumentIsTime(arguments.get(2)), - "The third argument of 'first_by' or 'last_by' function must be 'time'", - ctx); - } - } else if (name.toString().equalsIgnoreCase(APPROX_COUNT_DISTINCT)) { + if (name.toString().equalsIgnoreCase(APPROX_COUNT_DISTINCT)) { if (arguments.size() == 2 && !(arguments.get(1) instanceof DoubleLiteral || arguments.get(1) instanceof LongLiteral @@ -3467,30 +3451,6 @@ public Node visitFunctionCall(RelationalSqlParser.FunctionCallContext ctx) { return new FunctionCall(getLocation(ctx), name, window, nulls, distinct, mode, arguments); } - private void appendTimeArgument(List arguments) { - if (arguments.get(0) instanceof DereferenceExpression) { - arguments.add( - new DereferenceExpression( - ((DereferenceExpression) arguments.get(0)).getBase(), - new Identifier( - TimestampOperand.TIMESTAMP_EXPRESSION_STRING.toLowerCase(Locale.ENGLISH)))); - } else { - arguments.add( - new Identifier(TimestampOperand.TIMESTAMP_EXPRESSION_STRING.toLowerCase(Locale.ENGLISH))); - } - } - - private boolean checkArgumentIsTime(Expression argument) { - if (argument instanceof DereferenceExpression) { - return ((DereferenceExpression) argument) - .getField() - .get() - .toString() - .equalsIgnoreCase(TimestampOperand.TIMESTAMP_EXPRESSION_STRING); - } - return argument.toString().equalsIgnoreCase(TimestampOperand.TIMESTAMP_EXPRESSION_STRING); - } - @Override public Node visitDateBinGapFill(RelationalSqlParser.DateBinGapFillContext ctx) { TimeDuration timeDuration = DateTimeUtils.constructTimeDuration(ctx.timeDuration().getText()); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java index 3c15b2eaa281d..ac6c8ebab24f2 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java @@ -69,7 +69,6 @@ public class InformationSchema { new AttributeColumnSchema(ColumnHeaderConstant.STATEMENT_TABLE_MODEL, TSDataType.STRING)); queriesTable.addColumnSchema( new AttributeColumnSchema(ColumnHeaderConstant.USER_TABLE_MODEL, TSDataType.STRING)); - queriesTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(QUERIES, queriesTable); final TsTable databaseTable = new TsTable(DATABASES); @@ -94,7 +93,6 @@ public class InformationSchema { databaseTable.addColumnSchema( new AttributeColumnSchema( ColumnHeaderConstant.DATA_REGION_GROUP_NUM_TABLE_MODEL, TSDataType.INT32)); - databaseTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(DATABASES, databaseTable); final TsTable tableTable = new TsTable(TABLES); @@ -114,7 +112,6 @@ public class InformationSchema { ColumnHeaderConstant.COMMENT.toLowerCase(Locale.ENGLISH), TSDataType.STRING)); tableTable.addColumnSchema( new AttributeColumnSchema(ColumnHeaderConstant.TABLE_TYPE_TABLE_MODEL, TSDataType.STRING)); - tableTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(TABLES, tableTable); final TsTable columnTable = new TsTable(COLUMNS); @@ -137,7 +134,6 @@ public class InformationSchema { columnTable.addColumnSchema( new AttributeColumnSchema( ColumnHeaderConstant.COMMENT.toLowerCase(Locale.ENGLISH), TSDataType.STRING)); - columnTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(COLUMNS, columnTable); final TsTable regionTable = new TsTable(REGIONS); @@ -179,7 +175,6 @@ public class InformationSchema { regionTable.addColumnSchema( new AttributeColumnSchema( ColumnHeaderConstant.COMPRESSION_RATIO_TABLE_MODEL, TSDataType.DOUBLE)); - regionTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(REGIONS, regionTable); final TsTable pipeTable = new TsTable(PIPES); @@ -208,7 +203,6 @@ public class InformationSchema { pipeTable.addColumnSchema( new AttributeColumnSchema( ColumnHeaderConstant.ESTIMATED_REMAINING_SECONDS_TABLE_MODEL, TSDataType.DOUBLE)); - pipeTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(PIPES, pipeTable); final TsTable pipePluginTable = new TsTable(PIPE_PLUGINS); @@ -220,7 +214,6 @@ public class InformationSchema { new AttributeColumnSchema(ColumnHeaderConstant.CLASS_NAME_TABLE_MODEL, TSDataType.STRING)); pipePluginTable.addColumnSchema( new AttributeColumnSchema(ColumnHeaderConstant.PLUGIN_JAR_TABLE_MODEL, TSDataType.STRING)); - pipePluginTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(PIPE_PLUGINS, pipePluginTable); final TsTable topicTable = new TsTable(TOPICS); @@ -229,7 +222,6 @@ public class InformationSchema { topicTable.addColumnSchema( new AttributeColumnSchema( ColumnHeaderConstant.TOPIC_CONFIGS_TABLE_MODEL, TSDataType.STRING)); - topicTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(TOPICS, topicTable); final TsTable subscriptionTable = new TsTable(SUBSCRIPTIONS); @@ -241,7 +233,6 @@ public class InformationSchema { subscriptionTable.addColumnSchema( new AttributeColumnSchema( ColumnHeaderConstant.SUBSCRIBED_CONSUMERS_TABLE_MODEL, TSDataType.STRING)); - subscriptionTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(SUBSCRIPTIONS, subscriptionTable); final TsTable viewTable = new TsTable(VIEWS); @@ -253,7 +244,6 @@ public class InformationSchema { viewTable.addColumnSchema( new AttributeColumnSchema( ColumnHeaderConstant.VIEW_DEFINITION_TABLE_MODEL, TSDataType.STRING)); - viewTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(VIEWS, viewTable); final TsTable functionTable = new TsTable(FUNCTIONS); @@ -268,7 +258,6 @@ public class InformationSchema { functionTable.addColumnSchema( new AttributeColumnSchema( ColumnHeaderConstant.STATE.toLowerCase(Locale.ENGLISH), TSDataType.STRING)); - functionTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(FUNCTIONS, functionTable); final TsTable configurationsTable = new TsTable(CONFIGURATIONS); @@ -278,7 +267,6 @@ public class InformationSchema { configurationsTable.addColumnSchema( new AttributeColumnSchema( ColumnHeaderConstant.VALUE.toLowerCase(Locale.ENGLISH), TSDataType.STRING)); - configurationsTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(CONFIGURATIONS, configurationsTable); final TsTable keywordsTable = new TsTable(KEYWORDS); @@ -286,7 +274,6 @@ public class InformationSchema { new TagColumnSchema(ColumnHeaderConstant.WORD, TSDataType.STRING)); keywordsTable.addColumnSchema( new AttributeColumnSchema(ColumnHeaderConstant.RESERVED, TSDataType.INT32)); - keywordsTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(KEYWORDS, keywordsTable); final TsTable nodesTable = new TsTable(NODES); @@ -308,7 +295,6 @@ public class InformationSchema { ColumnHeaderConstant.VERSION.toLowerCase(Locale.ENGLISH), TSDataType.STRING)); nodesTable.addColumnSchema( new AttributeColumnSchema(ColumnHeaderConstant.BUILD_INFO_TABLE_MODEL, TSDataType.STRING)); - nodesTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(NODES, nodesTable); final TsTable configNodesTable = new TsTable(CONFIG_NODES); @@ -320,7 +306,6 @@ public class InformationSchema { configNodesTable.addColumnSchema( new AttributeColumnSchema( ColumnHeaderConstant.ROLE.toLowerCase(Locale.ENGLISH), TSDataType.STRING)); - configNodesTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(CONFIG_NODES, configNodesTable); final TsTable dataNodesTable = new TsTable(DATA_NODES); @@ -344,7 +329,6 @@ public class InformationSchema { dataNodesTable.addColumnSchema( new AttributeColumnSchema( ColumnHeaderConstant.SCHEMA_CONSENSUS_PORT_TABLE_MODEL, TSDataType.INT32)); - dataNodesTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(DATA_NODES, dataNodesTable); final TsTable connectionsTable = new TsTable(CONNECTIONS); @@ -360,7 +344,6 @@ public class InformationSchema { new AttributeColumnSchema(ColumnHeaderConstant.LAST_ACTIVE_TIME, TSDataType.TIMESTAMP)); connectionsTable.addColumnSchema( new AttributeColumnSchema(ColumnHeaderConstant.CLIENT_IP, TSDataType.STRING)); - connectionsTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(CONNECTIONS, connectionsTable); final TsTable currentQueriesTable = new TsTable(CURRENT_QUERIES); @@ -383,7 +366,6 @@ public class InformationSchema { new AttributeColumnSchema(ColumnHeaderConstant.USER_TABLE_MODEL, TSDataType.STRING)); currentQueriesTable.addColumnSchema( new AttributeColumnSchema(ColumnHeaderConstant.CLIENT_IP, TSDataType.STRING)); - currentQueriesTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(CURRENT_QUERIES, currentQueriesTable); final TsTable queriesCostsHistogramTable = new TsTable(QUERIES_COSTS_HISTOGRAM); @@ -393,7 +375,6 @@ public class InformationSchema { new AttributeColumnSchema(ColumnHeaderConstant.NUMS, TSDataType.INT32)); queriesCostsHistogramTable.addColumnSchema( new AttributeColumnSchema(ColumnHeaderConstant.DATANODE_ID, TSDataType.INT32)); - queriesCostsHistogramTable.removeColumnSchema(TsTable.TIME_COLUMN_NAME); schemaTables.put(QUERIES_COSTS_HISTOGRAM, queriesCostsHistogramTable); final TsTable servicesTable = new TsTable(SERVICES); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsFileTableSchemaUtil.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsFileTableSchemaUtil.java index 9a5abffbbcc65..7a74f4be651d8 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsFileTableSchemaUtil.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsFileTableSchemaUtil.java @@ -43,9 +43,8 @@ private TsFileTableSchemaUtil() { /** Column category filter for efficient parsing */ public enum ColumnCategoryFilter { - /** Include TAG and FIELD only (exclude TIME and ATTRIBUTE) - for TsFile writing */ - NO_ATTRIBUTE( - cat -> cat != TsTableColumnCategory.TIME && cat != TsTableColumnCategory.ATTRIBUTE); + /** Include TAG, time, and FIELD only (exclude ATTRIBUTE) - for TsFile writing */ + NO_ATTRIBUTE(cat -> cat != TsTableColumnCategory.ATTRIBUTE); private final java.util.function.Predicate predicate; @@ -160,8 +159,8 @@ public static TableSchema toTsFileTableSchemaNoAttribute(final TsTable table) { for (final TsTableColumnSchema columnSchema : tsTableColumnSchemas) { final TsTableColumnCategory category = columnSchema.getColumnCategory(); - // Skip TIME and ATTRIBUTE columns (only include TAG and FIELD) - if (category == TsTableColumnCategory.TIME || category == TsTableColumnCategory.ATTRIBUTE) { + // Skip ATTRIBUTE columns (only include TIME, TAG and FIELD) + if (category == TsTableColumnCategory.ATTRIBUTE) { continue; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsTable.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsTable.java index 1b5a9d4ac34aa..61b00d060bd63 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsTable.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/TsTable.java @@ -32,7 +32,6 @@ import org.apache.iotdb.commons.utils.CommonDateTimeUtils; import com.google.common.collect.ImmutableList; -import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.utils.Pair; import org.apache.tsfile.utils.ReadWriteIOUtils; @@ -64,16 +63,13 @@ public class TsTable { public static final String TIME_COLUMN_NAME = "time"; public static final String COMMENT_KEY = "__comment"; - private static final TimeColumnSchema TIME_COLUMN_SCHEMA = - new TimeColumnSchema(TIME_COLUMN_NAME, TSDataType.TIMESTAMP); - public static final String TTL_PROPERTY = "ttl"; public static final Set TABLE_ALLOWED_PROPERTIES = Collections.singleton(TTL_PROPERTY); private static final String OBJECT_STRING_ERROR = "When there are object fields, the %s %s shall not be '.', '..' or contain './', '.\\'."; protected String tableName; - private final Map columnSchemaMap = new LinkedHashMap<>(); + private Map columnSchemaMap = new LinkedHashMap<>(); private final Map tagColumnIndexMap = new HashMap<>(); private final Map idColumnIndexMap = new HashMap<>(); @@ -101,7 +97,6 @@ public class TsTable { public TsTable(final String tableName) { this.tableName = tableName; - columnSchemaMap.put(TIME_COLUMN_NAME, TIME_COLUMN_SCHEMA); } // This interface is used by InformationSchema table, so time column is not necessary @@ -217,42 +212,60 @@ public void addColumnSchema(final TsTableColumnSchema columnSchema) { }); } + /** + * Renames a column in the table schema while strictly preserving its original ordinal position. + */ public void renameColumnSchema(final String oldName, final String newName) { executeWrite( () -> { - // Ensures idempotency - if (columnSchemaMap.containsKey(oldName)) { - final TsTableColumnSchema schema = columnSchemaMap.remove(oldName); - final Map oldProps = schema.getProps(); - oldProps.computeIfAbsent(TreeViewSchema.ORIGINAL_NAME, k -> schema.getColumnName()); - switch (schema.getColumnCategory()) { - case TAG: - columnSchemaMap.put( - newName, new TagColumnSchema(newName, schema.getDataType(), oldProps)); - break; - case FIELD: - columnSchemaMap.put( - newName, - new FieldColumnSchema( - newName, - schema.getDataType(), - ((FieldColumnSchema) schema).getEncoding(), - ((FieldColumnSchema) schema).getCompressor(), - oldProps)); - break; - case ATTRIBUTE: - columnSchemaMap.put( - newName, new AttributeColumnSchema(newName, schema.getDataType(), oldProps)); - break; - case TIME: - default: - // Do nothing - columnSchemaMap.put(oldName, schema); + if (!columnSchemaMap.containsKey(oldName)) { + return; + } + + // Capture the current strict order of current columns + List> snapshotOfColumns = + new ArrayList<>(columnSchemaMap.entrySet()); + columnSchemaMap.clear(); + + // Re-insert all entries in their original sequence, substituting the renamed column at + // its exact original index + for (Map.Entry entry : snapshotOfColumns) { + String currentKey = entry.getKey(); + TsTableColumnSchema currentColumnSchema = entry.getValue(); + + if (currentKey.equals(oldName)) { + TsTableColumnSchema newSchema = createRenamedSchema(currentColumnSchema, newName); + columnSchemaMap.put(newName, newSchema); + } else { + columnSchemaMap.put(currentKey, currentColumnSchema); } } }); } + private TsTableColumnSchema createRenamedSchema(TsTableColumnSchema oldSchema, String newName) { + Map oldProps = oldSchema.getProps(); + oldProps.computeIfAbsent(TreeViewSchema.ORIGINAL_NAME, k -> oldSchema.getColumnName()); + + switch (oldSchema.getColumnCategory()) { + case TAG: + return new TagColumnSchema(newName, oldSchema.getDataType(), oldProps); + case FIELD: + return new FieldColumnSchema( + newName, + oldSchema.getDataType(), + ((FieldColumnSchema) oldSchema).getEncoding(), + ((FieldColumnSchema) oldSchema).getCompressor(), + oldProps); + case ATTRIBUTE: + return new AttributeColumnSchema(newName, oldSchema.getDataType(), oldProps); + case TIME: + return new TimeColumnSchema(newName, oldSchema.getDataType(), oldProps); + default: + return oldSchema; + } + } + public void removeColumnSchema(final String columnName) { executeWrite( () -> { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/column/TsTableColumnCategory.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/column/TsTableColumnCategory.java index ef620f3673a67..4b171a89c5c3d 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/column/TsTableColumnCategory.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/column/TsTableColumnCategory.java @@ -82,6 +82,8 @@ public ColumnCategory toTsFileColumnType() { return ColumnCategory.ATTRIBUTE; case FIELD: return ColumnCategory.FIELD; + case TIME: + return ColumnCategory.TIME; default: throw new IllegalArgumentException("Unsupported column type in TsFile: " + this); } @@ -95,6 +97,8 @@ public static TsTableColumnCategory fromTsFileColumnCategory(ColumnCategory colu return TAG; case ATTRIBUTE: return ATTRIBUTE; + case TIME: + return TIME; default: throw new IllegalArgumentException("Unknown column type: " + columnType); } diff --git a/pom.xml b/pom.xml index 6696f783d804b..395b100d4e4c0 100644 --- a/pom.xml +++ b/pom.xml @@ -173,7 +173,7 @@ 0.14.1 1.9 1.5.6-3 - 2.2.1-260115-SNAPSHOT + 2.2.1-260120-SNAPSHOT