From 19343215f005904c0bd0ca41713c73f9671810f5 Mon Sep 17 00:00:00 2001 From: libo Date: Thu, 2 Apr 2026 15:09:59 +0800 Subject: [PATCH 1/2] Support some configurations about data partition base on database level --- .../apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 | 1 + .../org/apache/iotdb/db/qp/sql/SqlLexer.g4 | 4 + .../persistence/schema/ClusterSchemaInfo.java | 11 + .../thrift/ConfigNodeRPCServiceProcessor.java | 27 +- .../commons/partition/DataPartition.java | 11 +- .../commons/utils/TimePartitionUtils.java | 324 ++++++++++++++---- .../commons/utils/TimePartitionUtilsTest.java | 86 ++++- 7 files changed, 372 insertions(+), 92 deletions(-) diff --git a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 index c64f9338e3a8f..e04ec081bb3ff 100644 --- a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 +++ b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 @@ -118,6 +118,7 @@ databaseAttributeClause databaseAttributeKey : TTL | TIME_PARTITION_INTERVAL + | TIME_PARTITION_ORIGIN | SCHEMA_REGION_GROUP_NUM | DATA_REGION_GROUP_NUM ; diff --git a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 index 65dcd5810f335..c0a4dff3aca38 100644 --- a/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 +++ b/iotdb-core/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 @@ -1211,6 +1211,10 @@ TIME_PARTITION_INTERVAL : T I M E '_' P A R T I T I O N '_' I N T E R V A L ; +TIME_PARTITION_ORIGIN + : T I M E '_' P A R T I T I O N '_' O R I G I N + ; + SCHEMA_REGION_GROUP_NUM : S C H E M A '_' R E G I O N '_' G R O U P '_' N U M ; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java index e3ce7d7c29db1..86f3532b030d4 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java @@ -38,6 +38,7 @@ import org.apache.iotdb.commons.utils.PathUtils; import org.apache.iotdb.commons.utils.StatusUtils; import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.confignode.consensus.request.read.database.CountDatabasePlan; import org.apache.iotdb.confignode.consensus.request.read.database.GetDatabasePlan; import org.apache.iotdb.confignode.consensus.request.read.table.DescTablePlan; @@ -196,6 +197,10 @@ public TSStatus createDatabase(final DatabaseSchemaPlan plan) { final TDatabaseSchema databaseSchema = plan.getSchema(); final PartialPath partialPathName = getQualifiedDatabasePartialPath(databaseSchema.getName()); + // Update TimePartitionUtils cache with database-specific time partition settings + TimePartitionUtils.updateDatabaseTimePartitionConfig( + databaseSchema.getName(), databaseSchema); + final ConfigMTree mTree = databaseSchema.isIsTableModel() ? tableModelMTree : treeModelMTree; mTree.setStorageGroup(partialPathName); @@ -280,6 +285,9 @@ public TSStatus alterDatabase(final DatabaseSchemaPlan plan) { .getAsMNode() .setDatabaseSchema(currentSchema); + // Update TimePartitionUtils cache with new time partition settings + TimePartitionUtils.updateDatabaseTimePartitionConfig(currentSchema.getName(), currentSchema); + result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode()); } catch (final MetadataException e) { LOGGER.error(ERROR_NAME, e); @@ -305,6 +313,9 @@ public TSStatus deleteDatabase(final DeleteDatabasePlan plan) { (isTableModel ? tableModelMTree : treeModelMTree) .deleteDatabase(getQualifiedDatabasePartialPath(plan.getName())); + // Remove database-specific time partition configuration from cache + TimePartitionUtils.removeDatabaseTimePartitionConfig(plan.getName()); + result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode()); } catch (final MetadataException e) { LOGGER.warn("Database not exist", e); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java index 4d01f3770c218..e58c7d59dfc4f 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java @@ -446,19 +446,20 @@ public TSStatus alterDatabase(final TDatabaseSchema databaseSchema) { "Failed to alter database. Doesn't support ALTER DataReplicationFactor yet."); } - if (databaseSchema.isSetTimePartitionOrigin()) { - errorResp = - new TSStatus(TSStatusCode.DATABASE_CONFIG_ERROR.getStatusCode()) - .setMessage( - "Failed to alter database. Doesn't support ALTER TimePartitionOrigin yet."); - } - - if (databaseSchema.isSetTimePartitionInterval()) { - errorResp = - new TSStatus(TSStatusCode.DATABASE_CONFIG_ERROR.getStatusCode()) - .setMessage( - "Failed to alter database. Doesn't support ALTER TimePartitionInterval yet."); - } + // Time partition settings are now supported for database-level configuration + // if (databaseSchema.isSetTimePartitionOrigin()) { + // errorResp = + // new TSStatus(TSStatusCode.DATABASE_CONFIG_ERROR.getStatusCode()) + // .setMessage( + // "Failed to alter database. Doesn't support ALTER TimePartitionOrigin yet."); + // } + + // if (databaseSchema.isSetTimePartitionInterval()) { + // errorResp = + // new TSStatus(TSStatusCode.DATABASE_CONFIG_ERROR.getStatusCode()) + // .setMessage( + // "Failed to alter database. Doesn't support ALTER TimePartitionInterval yet."); + // } if (errorResp != null) { LOGGER.warn("Execute AlterDatabase: {} with result: {}", databaseSchema, errorResp); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java index 100c40eddcc23..db81266e5dfe2 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartition.java @@ -109,7 +109,10 @@ public List> getTimePartitionRange( dataPartitionMap.get(storageGroup).get(seriesPartitionSlot); List timePartitionSlotList = map.keySet().stream() - .filter(key -> TimePartitionUtils.satisfyPartitionStartTime(timeFilter, key.startTime)) + .filter( + key -> + TimePartitionUtils.satisfyPartitionStartTime( + timeFilter, key.startTime, storageGroup)) .sorted(Comparator.comparingLong(TTimePartitionSlot::getStartTime)) .collect(toList()); @@ -152,7 +155,8 @@ public List getDataRegionReplicaSetWithTimeFilter( return regionReplicaSetMap.entrySet().stream() .filter( entry -> - TimePartitionUtils.satisfyPartitionStartTime(timeFilter, entry.getKey().startTime)) + TimePartitionUtils.satisfyPartitionStartTime( + timeFilter, entry.getKey().startTime, storageGroup)) .flatMap(entry -> entry.getValue().stream()) .distinct() .collect(toList()); @@ -175,7 +179,8 @@ public List getDataRegionReplicaSetWithTimeFilter( return dataPartitionMap.get(database).get(seriesPartitionSlot).entrySet().stream() .filter( entry -> - TimePartitionUtils.satisfyPartitionStartTime(timeFilter, entry.getKey().startTime)) + TimePartitionUtils.satisfyPartitionStartTime( + timeFilter, entry.getKey().startTime, database)) .flatMap(entry -> entry.getValue().stream()) .distinct() .collect(toList()); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/TimePartitionUtils.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/TimePartitionUtils.java index 250a347d1496b..12275a68f4653 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/TimePartitionUtils.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/TimePartitionUtils.java @@ -20,10 +20,13 @@ import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema; import org.apache.tsfile.read.filter.basic.Filter; import java.math.BigInteger; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; public class TimePartitionUtils { @@ -38,6 +41,10 @@ public class TimePartitionUtils { private static long timePartitionInterval = CommonDescriptor.getInstance().getConfig().getTimePartitionInterval(); + // Database-specific time partition settings cache + private static final Map databaseConfigCache = + new ConcurrentHashMap<>(); + private static final BigInteger bigTimePartitionOrigin = BigInteger.valueOf(timePartitionOrigin); private static final BigInteger bigTimePartitionInterval = BigInteger.valueOf(timePartitionInterval); @@ -46,8 +53,12 @@ public class TimePartitionUtils { private static final long timePartitionUpperBoundWithoutOverflow; static { - long minPartition = getTimePartitionIdWithoutOverflow(Long.MIN_VALUE); - long maxPartition = getTimePartitionIdWithoutOverflow(Long.MAX_VALUE); + long minPartition = + getTimePartitionIdWithoutOverflow( + Long.MIN_VALUE, timePartitionOrigin, timePartitionInterval); + long maxPartition = + getTimePartitionIdWithoutOverflow( + Long.MAX_VALUE, timePartitionOrigin, timePartitionInterval); BigInteger minPartitionStartTime = BigInteger.valueOf(minPartition) .multiply(bigTimePartitionInterval) @@ -71,111 +82,290 @@ public class TimePartitionUtils { } } - public static TTimePartitionSlot getTimePartitionSlot(long time) { - TTimePartitionSlot timePartitionSlot = new TTimePartitionSlot(); - timePartitionSlot.setStartTime(getTimePartitionLowerBound(time)); - return timePartitionSlot; - } + // Database-specific time partition configuration class + public static class DatabaseTimePartitionConfig { + private final long timePartitionOrigin; + private final long timePartitionInterval; + private final BigInteger bigTimePartitionOrigin; + private final BigInteger bigTimePartitionInterval; + private final boolean originMayCauseOverflow; + private final long timePartitionLowerBoundWithoutOverflow; + private final long timePartitionUpperBoundWithoutOverflow; - public static long getTimePartitionInterval() { - return timePartitionInterval; - } + public DatabaseTimePartitionConfig(long timePartitionOrigin, long timePartitionInterval) { + this.timePartitionOrigin = timePartitionOrigin; + this.timePartitionInterval = timePartitionInterval; + this.bigTimePartitionOrigin = BigInteger.valueOf(timePartitionOrigin); + this.bigTimePartitionInterval = BigInteger.valueOf(timePartitionInterval); + this.originMayCauseOverflow = (timePartitionOrigin != 0); - public static long getTimePartitionLowerBound(long time) { - if (time < timePartitionLowerBoundWithoutOverflow) { - return Long.MIN_VALUE; + // Calculate bounds for overflow handling + long minPartition = + getTimePartitionIdWithoutOverflow( + Long.MIN_VALUE, timePartitionOrigin, timePartitionInterval); + long maxPartition = + getTimePartitionIdWithoutOverflow( + Long.MAX_VALUE, timePartitionOrigin, timePartitionInterval); + BigInteger minPartitionStartTime = + BigInteger.valueOf(minPartition) + .multiply(this.bigTimePartitionInterval) + .add(this.bigTimePartitionOrigin); + BigInteger maxPartitionEndTime = + BigInteger.valueOf(maxPartition) + .multiply(this.bigTimePartitionInterval) + .add(this.bigTimePartitionInterval) + .add(this.bigTimePartitionOrigin); + if (minPartitionStartTime.compareTo(BigInteger.valueOf(Long.MIN_VALUE)) < 0) { + this.timePartitionLowerBoundWithoutOverflow = + minPartitionStartTime.add(this.bigTimePartitionInterval).longValue(); + } else { + this.timePartitionLowerBoundWithoutOverflow = minPartitionStartTime.longValue(); + } + if (maxPartitionEndTime.compareTo(BigInteger.valueOf(Long.MAX_VALUE)) > 0) { + this.timePartitionUpperBoundWithoutOverflow = + maxPartitionEndTime.subtract(this.bigTimePartitionInterval).longValue(); + } else { + this.timePartitionUpperBoundWithoutOverflow = maxPartitionEndTime.longValue(); + } } - if (originMayCauseOverflow) { - return BigInteger.valueOf(getTimePartitionIdWithoutOverflow(time)) - .multiply(bigTimePartitionInterval) - .add(bigTimePartitionOrigin) - .longValue(); - } else { - return getTimePartitionId(time) * timePartitionInterval + timePartitionOrigin; + + // Getters for database-specific configuration + public long getTimePartitionOrigin() { + return timePartitionOrigin; } - } - public static long getTimePartitionUpperBound(long time) { - if (time >= timePartitionUpperBoundWithoutOverflow) { - return Long.MAX_VALUE; + public long getTimePartitionInterval() { + return timePartitionInterval; + } + + public boolean isOriginMayCauseOverflow() { + return originMayCauseOverflow; + } + + public long getTimePartitionLowerBoundWithoutOverflow() { + return timePartitionLowerBoundWithoutOverflow; + } + + public long getTimePartitionUpperBoundWithoutOverflow() { + return timePartitionUpperBoundWithoutOverflow; + } + + public BigInteger getBigTimePartitionOrigin() { + return bigTimePartitionOrigin; + } + + public BigInteger getBigTimePartitionInterval() { + return bigTimePartitionInterval; } - long lowerBound = getTimePartitionLowerBound(time); - return lowerBound == Long.MIN_VALUE - ? timePartitionLowerBoundWithoutOverflow - : lowerBound + timePartitionInterval; } - public static long getTimePartitionId(long time) { - time -= timePartitionOrigin; - return time > 0 || time % timePartitionInterval == 0 - ? time / timePartitionInterval - : time / timePartitionInterval - 1; + // Update or add database-specific time partition configuration + public static void updateDatabaseTimePartitionConfig(String database, TDatabaseSchema schema) { + long interval = + schema.isSetTimePartitionInterval() + ? schema.getTimePartitionInterval() + : timePartitionInterval; + long origin = + schema.isSetTimePartitionOrigin() ? schema.getTimePartitionOrigin() : timePartitionOrigin; + databaseConfigCache.put(database, new DatabaseTimePartitionConfig(origin, interval)); } - public static long getTimePartitionIdWithoutOverflow(long time) { - BigInteger bigTime = BigInteger.valueOf(time).subtract(bigTimePartitionOrigin); - BigInteger partitionId = - bigTime.compareTo(BigInteger.ZERO) > 0 - || bigTime.remainder(bigTimePartitionInterval).equals(BigInteger.ZERO) - ? bigTime.divide(bigTimePartitionInterval) - : bigTime.divide(bigTimePartitionInterval).subtract(BigInteger.ONE); - return partitionId.longValue(); + // Remove database-specific time partition configuration + public static void removeDatabaseTimePartitionConfig(String database) { + databaseConfigCache.remove(database); } - public static long getStartTimeByPartitionId(long partitionId) { - return (partitionId * timePartitionInterval) + timePartitionOrigin; + // Get database-specific configuration, fallback to global if not found + private static DatabaseTimePartitionConfig getDatabaseConfig(String database) { + DatabaseTimePartitionConfig config = databaseConfigCache.get(database); + return config != null + ? config + : new DatabaseTimePartitionConfig(timePartitionOrigin, timePartitionInterval); } - public static boolean satisfyPartitionId(long startTime, long endTime, long partitionId) { + // Database-specific time partition methods + public static TTimePartitionSlot getTimePartitionSlot(long time, String database) { + DatabaseTimePartitionConfig config = getDatabaseConfig(database); + TTimePartitionSlot timePartitionSlot = new TTimePartitionSlot(); + timePartitionSlot.setStartTime(getTimePartitionLowerBoundInternal(time, config)); + return timePartitionSlot; + } + + public static long getTimePartitionInterval(String database) { + return getDatabaseConfig(database).getTimePartitionInterval(); + } + + public static long getTimePartitionLowerBound(long time, String database) { + return getTimePartitionLowerBoundInternal(time, getDatabaseConfig(database)); + } + + public static long getTimePartitionUpperBound(long time, String database) { + return getTimePartitionUpperBoundInternal(time, getDatabaseConfig(database)); + } + + public static long getTimePartitionId(long time, String database) { + DatabaseTimePartitionConfig config = getDatabaseConfig(database); + time -= config.getTimePartitionOrigin(); + return time > 0 || time % config.getTimePartitionInterval() == 0 + ? time / config.getTimePartitionInterval() + : time / config.getTimePartitionInterval() - 1; + } + + public static long getStartTimeByPartitionId(long partitionId, String database) { + DatabaseTimePartitionConfig config = getDatabaseConfig(database); + return (partitionId * config.getTimePartitionInterval()) + config.getTimePartitionOrigin(); + } + + public static boolean satisfyPartitionId( + long startTime, long endTime, long partitionId, String database) { + DatabaseTimePartitionConfig config = getDatabaseConfig(database); long startPartition = - originMayCauseOverflow - ? getTimePartitionIdWithoutOverflow(startTime) - : getTimePartitionId(startTime); + config.isOriginMayCauseOverflow() + ? getTimePartitionIdWithoutOverflow( + startTime, config.getTimePartitionOrigin(), config.getTimePartitionInterval()) + : getTimePartitionId( + startTime, config.getTimePartitionOrigin(), config.getTimePartitionInterval()); long endPartition = - originMayCauseOverflow - ? getTimePartitionIdWithoutOverflow(endTime) - : getTimePartitionId(endTime); + config.isOriginMayCauseOverflow() + ? getTimePartitionIdWithoutOverflow( + endTime, config.getTimePartitionOrigin(), config.getTimePartitionInterval()) + : getTimePartitionId( + endTime, config.getTimePartitionOrigin(), config.getTimePartitionInterval()); return startPartition <= partitionId && endPartition >= partitionId; } - public static boolean satisfyPartitionStartTime(Filter timeFilter, long partitionStartTime) { + public static boolean satisfyPartitionStartTime( + Filter timeFilter, long partitionStartTime, String database) { if (timeFilter == null) { return true; } - + DatabaseTimePartitionConfig config = getDatabaseConfig(database); long partitionEndTime = - partitionStartTime >= timePartitionLowerBoundWithoutOverflow + partitionStartTime >= config.getTimePartitionLowerBoundWithoutOverflow() ? Long.MAX_VALUE - : (partitionStartTime + timePartitionInterval - 1); + : (partitionStartTime + config.getTimePartitionInterval() - 1); return timeFilter.satisfyStartEndTime(partitionStartTime, partitionEndTime); } - public static boolean satisfyTimePartition(Filter timeFilter, long partitionId) { + public static boolean satisfyTimePartition(Filter timeFilter, long partitionId, String database) { + DatabaseTimePartitionConfig config = getDatabaseConfig(database); long partitionStartTime; - if (originMayCauseOverflow) { + if (config.isOriginMayCauseOverflow()) { partitionStartTime = BigInteger.valueOf(partitionId) - .multiply(bigTimePartitionInterval) - .add(bigTimePartitionOrigin) + .multiply(config.getBigTimePartitionInterval()) + .add(config.getBigTimePartitionOrigin()) .longValue(); } else { - partitionStartTime = partitionId * timePartitionInterval + timePartitionOrigin; + partitionStartTime = + partitionId * config.getTimePartitionInterval() + config.getTimePartitionOrigin(); } - return satisfyPartitionStartTime(timeFilter, partitionStartTime); - } - - public static void setTimePartitionInterval(long timePartitionInterval) { - TimePartitionUtils.timePartitionInterval = timePartitionInterval; + return satisfyPartitionStartTime(timeFilter, partitionStartTime, database); } - public static long getEstimateTimePartitionSize(long startTime, long endTime) { + public static long getEstimateTimePartitionSize(long startTime, long endTime, String database) { + DatabaseTimePartitionConfig config = getDatabaseConfig(database); if (endTime > 0 && startTime < 0) { return BigInteger.valueOf(endTime) .subtract(BigInteger.valueOf(startTime)) - .divide(bigTimePartitionInterval) + .divide(config.getBigTimePartitionInterval()) .longValue() + 1; } - return (endTime - startTime) / timePartitionInterval + 1; + return (endTime - startTime) / config.getTimePartitionInterval() + 1; + } + + // Helper methods for database-specific calculations + private static long getTimePartitionLowerBoundInternal( + long time, DatabaseTimePartitionConfig config) { + if (time < config.getTimePartitionLowerBoundWithoutOverflow()) { + return Long.MIN_VALUE; + } + if (config.isOriginMayCauseOverflow()) { + return BigInteger.valueOf( + getTimePartitionIdWithoutOverflow( + time, config.getTimePartitionOrigin(), config.getTimePartitionInterval())) + .multiply(config.getBigTimePartitionInterval()) + .add(config.getBigTimePartitionOrigin()) + .longValue(); + } else { + return getTimePartitionId( + time, config.getTimePartitionOrigin(), config.getTimePartitionInterval()) + * config.getTimePartitionInterval() + + config.getTimePartitionOrigin(); + } + } + + private static long getTimePartitionUpperBoundInternal( + long time, DatabaseTimePartitionConfig config) { + if (time >= config.getTimePartitionUpperBoundWithoutOverflow()) { + return Long.MAX_VALUE; + } + long lowerBound = getTimePartitionLowerBoundInternal(time, config); + return lowerBound == Long.MIN_VALUE + ? config.getTimePartitionLowerBoundWithoutOverflow() + : lowerBound + config.getTimePartitionInterval(); + } + + private static long getTimePartitionId(long time, long origin, long interval) { + time -= origin; + return time > 0 || time % interval == 0 ? time / interval : time / interval - 1; + } + + private static long getTimePartitionIdWithoutOverflow(long time, long origin, long interval) { + BigInteger bigTime = BigInteger.valueOf(time).subtract(BigInteger.valueOf(origin)); + BigInteger bigInterval = BigInteger.valueOf(interval); + BigInteger partitionId = + bigTime.compareTo(BigInteger.ZERO) > 0 + || bigTime.remainder(bigInterval).equals(BigInteger.ZERO) + ? bigTime.divide(bigInterval) + : bigTime.divide(bigInterval).subtract(BigInteger.ONE); + return partitionId.longValue(); + } + + // Original global methods for backward compatibility + public static long getTimePartitionInterval() { + return timePartitionInterval; + } + + public static void setTimePartitionInterval(long timePartitionInterval) { + TimePartitionUtils.timePartitionInterval = timePartitionInterval; + } + + // Backward compatibility methods that use global configuration + public static TTimePartitionSlot getTimePartitionSlot(long time) { + return getTimePartitionSlot(time, null); + } + + public static long getTimePartitionLowerBound(long time) { + return getTimePartitionLowerBound(time, null); + } + + public static long getTimePartitionUpperBound(long time) { + return getTimePartitionUpperBound(time, null); + } + + public static long getTimePartitionId(long time) { + return getTimePartitionId(time, null); + } + + public static long getStartTimeByPartitionId(long partitionId) { + return getStartTimeByPartitionId(partitionId, null); + } + + public static boolean satisfyPartitionId(long startTime, long endTime, long partitionId) { + return satisfyPartitionId(startTime, endTime, partitionId, null); + } + + public static boolean satisfyPartitionStartTime(Filter timeFilter, long partitionStartTime) { + return satisfyPartitionStartTime(timeFilter, partitionStartTime, null); + } + + public static boolean satisfyTimePartition(Filter timeFilter, long partitionId) { + return satisfyTimePartition(timeFilter, partitionId, null); + } + + public static long getEstimateTimePartitionSize(long startTime, long endTime) { + return getEstimateTimePartitionSize(startTime, endTime, null); } } diff --git a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/TimePartitionUtilsTest.java b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/TimePartitionUtilsTest.java index ea0eeda45d28f..83b52b3bde8cd 100644 --- a/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/TimePartitionUtilsTest.java +++ b/iotdb-core/node-commons/src/test/java/org/apache/iotdb/commons/utils/TimePartitionUtilsTest.java @@ -21,6 +21,7 @@ import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; import org.apache.iotdb.commons.conf.CommonDescriptor; +import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema; import org.junit.Assert; import org.junit.Before; @@ -47,7 +48,7 @@ public void testGetTimePartitionSlot_StartOfInterval() { TTimePartitionSlot expectedSlot = new TTimePartitionSlot(); expectedSlot.setStartTime(TEST_TIME_PARTITION_ORIGIN); - TTimePartitionSlot actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime); + TTimePartitionSlot actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime, "global"); assertEquals(expectedSlot.getStartTime(), actualSlot.getStartTime()); } @@ -57,7 +58,7 @@ public void testGetTimePartitionSlot_MiddleOfInterval() { TTimePartitionSlot expectedSlot = new TTimePartitionSlot(); expectedSlot.setStartTime(TEST_TIME_PARTITION_ORIGIN); - TTimePartitionSlot actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime); + TTimePartitionSlot actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime, "global"); assertEquals(expectedSlot.getStartTime(), actualSlot.getStartTime()); } @@ -67,7 +68,7 @@ public void testGetTimePartitionSlot_EndOfInterval() { TTimePartitionSlot expectedSlot = new TTimePartitionSlot(); expectedSlot.setStartTime(TEST_TIME_PARTITION_ORIGIN); - TTimePartitionSlot actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime); + TTimePartitionSlot actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime, "global"); assertEquals(expectedSlot.getStartTime(), actualSlot.getStartTime()); } @@ -77,7 +78,7 @@ public void testGetTimePartitionSlot_NegativeTime() { TTimePartitionSlot expectedSlot = new TTimePartitionSlot(); expectedSlot.setStartTime(TEST_TIME_PARTITION_ORIGIN - TEST_TIME_PARTITION_INTERVAL); - TTimePartitionSlot actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime); + TTimePartitionSlot actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime, "global"); assertEquals(expectedSlot.getStartTime(), actualSlot.getStartTime()); } @@ -87,22 +88,89 @@ public void testGetTimePartitionSlot_NegativeBoundaryTime() { TTimePartitionSlot expectedSlot = new TTimePartitionSlot(); expectedSlot.setStartTime(TEST_TIME_PARTITION_ORIGIN - TEST_TIME_PARTITION_INTERVAL); - TTimePartitionSlot actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime); + TTimePartitionSlot actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime, "global"); assertEquals(expectedSlot.getStartTime(), actualSlot.getStartTime()); } @Test public void testOverflow() { long testTime = Long.MIN_VALUE; - TTimePartitionSlot actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime); + TTimePartitionSlot actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime, "global"); Assert.assertTrue(actualSlot.getStartTime() < 0); testTime += 1; - long lowerBound = TimePartitionUtils.getTimePartitionLowerBound(testTime); + long lowerBound = TimePartitionUtils.getTimePartitionLowerBound(testTime, "global"); assertEquals(Long.MIN_VALUE, lowerBound); testTime = Long.MAX_VALUE; - actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime); + actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime, "global"); Assert.assertTrue(actualSlot.getStartTime() > 0); - long upperBound = TimePartitionUtils.getTimePartitionUpperBound(testTime); + long upperBound = TimePartitionUtils.getTimePartitionUpperBound(testTime, "global"); assertEquals(Long.MAX_VALUE, upperBound); } + + @Test + public void testDatabaseLevelTimePartition() { + // Create a database schema with custom time partition settings + TDatabaseSchema schema = new TDatabaseSchema(); + schema.setName("test.db"); + schema.setTimePartitionInterval(7200000L); // 2 hours + schema.setTimePartitionOrigin(2000L); + + // Update database configuration + TimePartitionUtils.updateDatabaseTimePartitionConfig("test.db", schema); + + // Test with custom database settings + long testTime = 2000L; + TTimePartitionSlot expectedSlot = new TTimePartitionSlot(); + expectedSlot.setStartTime(2000L); + + TTimePartitionSlot actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime, "test.db"); + assertEquals(expectedSlot.getStartTime(), actualSlot.getStartTime()); + + // Test with different time in the same partition + testTime = 2000L + 3600000L; // 1 hour later + actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime, "test.db"); + assertEquals(expectedSlot.getStartTime(), actualSlot.getStartTime()); + + // Test with time in next partition + testTime = 2000L + 7200000L; // 2 hours later + expectedSlot.setStartTime(2000L + 7200000L); + actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime, "test.db"); + assertEquals(expectedSlot.getStartTime(), actualSlot.getStartTime()); + } + + @Test + public void testDatabaseLevelTimePartitionFallbackToGlobal() { + // Test with database that doesn't have custom settings + long testTime = TEST_TIME_PARTITION_ORIGIN; + TTimePartitionSlot expectedSlot = new TTimePartitionSlot(); + expectedSlot.setStartTime(TEST_TIME_PARTITION_ORIGIN); + + TTimePartitionSlot actualSlot = + TimePartitionUtils.getTimePartitionSlot(testTime, "nonexistent.db"); + assertEquals(expectedSlot.getStartTime(), actualSlot.getStartTime()); + } + + @Test + public void testRemoveDatabaseTimePartitionConfig() { + // Create and update database configuration + TDatabaseSchema schema = new TDatabaseSchema(); + schema.setName("test.db"); + schema.setTimePartitionInterval(7200000L); + schema.setTimePartitionOrigin(2000L); + TimePartitionUtils.updateDatabaseTimePartitionConfig("test.db", schema); + + // Verify custom configuration is used + long testTime = 2000L; + TTimePartitionSlot actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime, "test.db"); + assertEquals(2000L, actualSlot.getStartTime()); + + // Remove database configuration + TimePartitionUtils.removeDatabaseTimePartitionConfig("test.db"); + + // Verify fallback to global settings + TTimePartitionSlot expectedSlot = new TTimePartitionSlot(); + expectedSlot.setStartTime(TEST_TIME_PARTITION_ORIGIN); + actualSlot = TimePartitionUtils.getTimePartitionSlot(testTime, "test.db"); + assertEquals(expectedSlot.getStartTime(), actualSlot.getStartTime()); + } } From 7f5aa4b5d9787cef1006f6a2196f5c5e63dafb90 Mon Sep 17 00:00:00 2001 From: libo Date: Fri, 3 Apr 2026 18:43:28 +0800 Subject: [PATCH 2/2] Switch all original old method calls to the method of obtaining configurations at the database granularity --- .../consensus/request/ConfigPhysicalPlan.java | 4 + .../request/ConfigPhysicalPlanType.java | 1 + .../database/SetTimePartitionOriginPlan.java | 86 +++++++++++++++++++ .../confignode/manager/ConfigManager.java | 11 +++ .../iotdb/confignode/manager/IManager.java | 3 + .../manager/schema/ClusterSchemaManager.java | 12 +++ .../executor/ConfigPlanExecutor.java | 3 + .../partition/DatabasePartitionTable.java | 2 +- .../persistence/partition/PartitionInfo.java | 2 +- .../persistence/schema/ClusterSchemaInfo.java | 27 ++++++ ...PartitionTableIntegrityCheckProcedure.java | 2 +- .../thrift/ConfigNodeRPCServiceProcessor.java | 8 ++ .../request/ConfigPhysicalPlanSerDeTest.java | 17 +++- .../DataPartitionTableGenerator.java | 23 +++-- .../ReplicateProgressDataNodeManager.java | 6 +- ...icalDataRegionTsFileAndDeletionSource.java | 4 +- .../PipeRealtimeDataRegionSource.java | 21 +++-- .../db/protocol/client/ConfigNodeClient.java | 7 ++ .../plan/analyze/AnalyzeUtils.java | 23 ++--- .../plan/analyze/AnalyzeVisitor.java | 38 ++++---- .../plan/analyze/TemplatedAnalyze.java | 3 +- .../executor/ClusterConfigTaskExecutor.java | 6 +- .../config/metadata/DatabaseSchemaTask.java | 3 + .../queryengine/plan/parser/ASTVisitor.java | 3 + .../planner/distribution/SourceRewriter.java | 11 ++- .../plan/planner/plan/node/PlanNodeId.java | 8 ++ .../plan/node/load/LoadSingleTsFileNode.java | 8 +- .../plan/node/write/InsertRowNode.java | 7 +- .../plan/node/write/InsertRowsNode.java | 3 +- .../node/write/InsertRowsOfOneDeviceNode.java | 3 +- .../plan/node/write/InsertTabletNode.java | 29 ++++--- .../node/write/RelationalInsertRowsNode.java | 3 +- .../TableDistributedPlanGenerator.java | 7 +- .../PushPredicateIntoTableScan.java | 2 +- .../scheduler/load/LoadTsFileScheduler.java | 2 +- .../statement/crud/InsertRowStatement.java | 4 +- .../crud/InsertRowsOfOneDeviceStatement.java | 10 ++- .../statement/crud/InsertTabletStatement.java | 17 ++-- .../metadata/DatabaseSchemaStatement.java | 11 +++ .../iotdb/db/storageengine/StorageEngine.java | 8 -- .../storageengine/dataregion/DataRegion.java | 54 ++++++------ .../dataregion/snapshot/SnapshotTaker.java | 5 +- .../dataregion/tsfile/TsFileManager.java | 26 +++--- .../load/splitter/AlignedChunkData.java | 14 +-- .../BatchedAlignedValueChunkData.java | 5 +- .../load/splitter/ChunkData.java | 3 +- .../load/splitter/NonAlignedChunkData.java | 7 +- .../load/splitter/TsFileSplitter.java | 65 ++++++++------ .../plan/analyze/QueryTimePartitionTest.java | 29 ++++--- .../node/write/WritePlanNodeSplitTest.java | 55 ++++++++---- .../db/storageengine/StorageEngineTest.java | 11 ++- ...tchedCompactionWithTsFileSplitterTest.java | 2 +- .../commons/partition/DataPartitionTable.java | 4 +- .../partition/SeriesPartitionTable.java | 16 ++-- .../utils/ThriftCommonsSerDeUtils.java | 3 +- .../commons/utils/TimePartitionUtils.java | 23 +++++ .../src/main/thrift/confignode.thrift | 8 ++ 57 files changed, 558 insertions(+), 220 deletions(-) create mode 100644 iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/SetTimePartitionOriginPlan.java diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java index 7fd7cd029119a..3a9a181ad7b99 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlan.java @@ -43,6 +43,7 @@ import org.apache.iotdb.confignode.consensus.request.write.database.SetSchemaReplicationFactorPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTimePartitionIntervalPlan; +import org.apache.iotdb.confignode.consensus.request.write.database.SetTimePartitionOriginPlan; import org.apache.iotdb.confignode.consensus.request.write.datanode.RegisterDataNodePlan; import org.apache.iotdb.confignode.consensus.request.write.datanode.RemoveDataNodePlan; import org.apache.iotdb.confignode.consensus.request.write.datanode.UpdateDataNodePlan; @@ -232,6 +233,9 @@ public static ConfigPhysicalPlan create(final ByteBuffer buffer) throws IOExcept case SetTimePartitionInterval: plan = new SetTimePartitionIntervalPlan(); break; + case SetTimePartitionOrigin: + plan = new SetTimePartitionOriginPlan(); + break; case AdjustMaxRegionGroupNum: plan = new AdjustMaxRegionGroupNumPlan(); break; diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java index 371435c9175bb..7953699b2c87d 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanType.java @@ -48,6 +48,7 @@ public enum ConfigPhysicalPlanType { SetSchemaReplicationFactor((short) 202), SetDataReplicationFactor((short) 203), SetTimePartitionInterval((short) 204), + SetTimePartitionOrigin((short) 212), AdjustMaxRegionGroupNum((short) 205), DeleteDatabase((short) 206), PreDeleteDatabase((short) 207), diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/SetTimePartitionOriginPlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/SetTimePartitionOriginPlan.java new file mode 100644 index 0000000000000..e9b2cff123a91 --- /dev/null +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/database/SetTimePartitionOriginPlan.java @@ -0,0 +1,86 @@ +/* + * 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.iotdb.confignode.consensus.request.write.database; + +import org.apache.iotdb.commons.utils.BasicStructureSerDeUtil; +import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlan; +import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Objects; + +public class SetTimePartitionOriginPlan extends ConfigPhysicalPlan { + + private String storageGroup; + + private long timePartitionOrigin; + + public SetTimePartitionOriginPlan() { + super(ConfigPhysicalPlanType.SetTimePartitionOrigin); + } + + public SetTimePartitionOriginPlan(String storageGroup, long timePartitionOrigin) { + this(); + this.storageGroup = storageGroup; + this.timePartitionOrigin = timePartitionOrigin; + } + + public String getDatabase() { + return storageGroup; + } + + public long getTimePartitionOrigin() { + return timePartitionOrigin; + } + + @Override + protected void serializeImpl(DataOutputStream stream) throws IOException { + stream.writeShort(getType().getPlanType()); + + BasicStructureSerDeUtil.write(storageGroup, stream); + stream.writeLong(timePartitionOrigin); + } + + @Override + protected void deserializeImpl(ByteBuffer buffer) throws IOException { + storageGroup = BasicStructureSerDeUtil.readString(buffer); + timePartitionOrigin = buffer.getLong(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SetTimePartitionOriginPlan that = (SetTimePartitionOriginPlan) o; + return timePartitionOrigin == that.timePartitionOrigin + && storageGroup.equals(that.storageGroup); + } + + @Override + public int hashCode() { + return Objects.hash(storageGroup, timePartitionOrigin); + } +} diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java index f455edb26b8b1..e73b2f3e87f2c 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java @@ -91,6 +91,7 @@ import org.apache.iotdb.confignode.consensus.request.write.database.SetSchemaReplicationFactorPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTimePartitionIntervalPlan; +import org.apache.iotdb.confignode.consensus.request.write.database.SetTimePartitionOriginPlan; import org.apache.iotdb.confignode.consensus.request.write.datanode.RemoveDataNodePlan; import org.apache.iotdb.confignode.consensus.request.write.template.CreateSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.response.ainode.AINodeRegisterResp; @@ -720,6 +721,16 @@ public TSStatus setTimePartitionInterval( } } + @Override + public TSStatus setTimePartitionOrigin(SetTimePartitionOriginPlan setTimePartitionOriginPlan) { + TSStatus status = confirmLeader(); + if (status.getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + return clusterSchemaManager.setTimePartitionOrigin(setTimePartitionOriginPlan); + } else { + return status; + } + } + @Override public DataSet countMatchedDatabases(CountDatabasePlan countDatabasePlan) { TSStatus status = confirmLeader(); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java index 02c82164595df..bc2070420ce72 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/IManager.java @@ -51,6 +51,7 @@ import org.apache.iotdb.confignode.consensus.request.write.database.SetSchemaReplicationFactorPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTimePartitionIntervalPlan; +import org.apache.iotdb.confignode.consensus.request.write.database.SetTimePartitionOriginPlan; import org.apache.iotdb.confignode.consensus.request.write.datanode.RemoveDataNodePlan; import org.apache.iotdb.confignode.manager.consensus.ConsensusManager; import org.apache.iotdb.confignode.manager.cq.CQManager; @@ -386,6 +387,8 @@ public interface IManager { TSStatus setTimePartitionInterval(SetTimePartitionIntervalPlan configPhysicalPlan); + TSStatus setTimePartitionOrigin(SetTimePartitionOriginPlan configPhysicalPlan); + /** * Count Databases. * diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java index 1ea8528213bce..2da605e2ed465 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/manager/schema/ClusterSchemaManager.java @@ -64,6 +64,7 @@ import org.apache.iotdb.confignode.consensus.request.write.database.SetDataReplicationFactorPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetSchemaReplicationFactorPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTimePartitionIntervalPlan; +import org.apache.iotdb.confignode.consensus.request.write.database.SetTimePartitionOriginPlan; import org.apache.iotdb.confignode.consensus.request.write.pipe.payload.PipeEnrichedPlan; import org.apache.iotdb.confignode.consensus.request.write.table.SetTableColumnCommentPlan; import org.apache.iotdb.confignode.consensus.request.write.table.SetTableCommentPlan; @@ -470,6 +471,17 @@ public TSStatus setTimePartitionInterval( } } + public TSStatus setTimePartitionOrigin(SetTimePartitionOriginPlan setTimePartitionOriginPlan) { + try { + return getConsensusManager().write(setTimePartitionOriginPlan); + } catch (ConsensusException e) { + LOGGER.warn(CONSENSUS_WRITE_ERROR, e); + TSStatus result = new TSStatus(TSStatusCode.EXECUTE_STATEMENT_ERROR.getStatusCode()); + result.setMessage(e.getMessage()); + return result; + } + } + /** * Only leader use this interface. Adjust the maxSchemaRegionGroupNum and maxDataRegionGroupNum of * each Database based on existing cluster resources diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java index 8016690d17c9a..79278034cef21 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/executor/ConfigPlanExecutor.java @@ -76,6 +76,7 @@ import org.apache.iotdb.confignode.consensus.request.write.database.SetSchemaReplicationFactorPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTimePartitionIntervalPlan; +import org.apache.iotdb.confignode.consensus.request.write.database.SetTimePartitionOriginPlan; import org.apache.iotdb.confignode.consensus.request.write.datanode.RegisterDataNodePlan; import org.apache.iotdb.confignode.consensus.request.write.datanode.RemoveDataNodePlan; import org.apache.iotdb.confignode.consensus.request.write.datanode.UpdateDataNodePlan; @@ -435,6 +436,8 @@ public TSStatus executeNonQueryPlan(ConfigPhysicalPlan physicalPlan) case SetTimePartitionInterval: return clusterSchemaInfo.setTimePartitionInterval( (SetTimePartitionIntervalPlan) physicalPlan); + case SetTimePartitionOrigin: + return clusterSchemaInfo.setTimePartitionOrigin((SetTimePartitionOriginPlan) physicalPlan); case CreateRegionGroups: return partitionInfo.createRegionGroups((CreateRegionGroupsPlan) physicalPlan); case OfferRegionMaintainTasks: diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/DatabasePartitionTable.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/DatabasePartitionTable.java index 1d2d776c69ba8..7c3b8a42fed89 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/DatabasePartitionTable.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/DatabasePartitionTable.java @@ -620,7 +620,7 @@ public Map getLastDataAllotTable() { */ public void autoCleanPartitionTable(long TTL, TTimePartitionSlot currentTimeSlot) { long[] removedTimePartitionSlots = - dataPartitionTable.autoCleanPartitionTable(TTL, currentTimeSlot).stream() + dataPartitionTable.autoCleanPartitionTable(TTL, currentTimeSlot, databaseName).stream() .map(TTimePartitionSlot::getStartTime) .collect(Collectors.toList()) .stream() diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java index b907527416bda..36436742307fe 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/partition/PartitionInfo.java @@ -521,7 +521,7 @@ public TSStatus autoCleanPartitionTable(AutoCleanPartitionTablePlan plan) { if (isDatabaseExisted(database) && 0 < ttl && ttl < Long.MAX_VALUE) { databasePartitionTables .get(database) - .autoCleanPartitionTable(ttl, plan.getCurrentTimeSlot()); + .autoCleanPartitionTable(ttl, plan.getCurrentTimeSlotMap()); } }); return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java index 86f3532b030d4..1417f5ae5e44a 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfo.java @@ -54,6 +54,7 @@ import org.apache.iotdb.confignode.consensus.request.write.database.SetDataReplicationFactorPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetSchemaReplicationFactorPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTimePartitionIntervalPlan; +import org.apache.iotdb.confignode.consensus.request.write.database.SetTimePartitionOriginPlan; import org.apache.iotdb.confignode.consensus.request.write.table.AddTableColumnPlan; import org.apache.iotdb.confignode.consensus.request.write.table.AlterColumnDataTypePlan; import org.apache.iotdb.confignode.consensus.request.write.table.CommitCreateTablePlan; @@ -493,6 +494,32 @@ public TSStatus setTimePartitionInterval(final SetTimePartitionIntervalPlan plan return result; } + public TSStatus setTimePartitionOrigin(final SetTimePartitionOriginPlan plan) { + final TSStatus result = new TSStatus(); + databaseReadWriteLock.writeLock().lock(); + try { + final ConfigMTree mTree = + PathUtils.isTableModelDatabase(plan.getDatabase()) ? tableModelMTree : treeModelMTree; + final PartialPath path = getQualifiedDatabasePartialPath(plan.getDatabase()); + if (mTree.isDatabaseAlreadySet(path)) { + mTree + .getDatabaseNodeByDatabasePath(path) + .getAsMNode() + .getDatabaseSchema() + .setTimePartitionOrigin(plan.getTimePartitionOrigin()); + result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode()); + } else { + result.setCode(TSStatusCode.DATABASE_NOT_EXIST.getStatusCode()); + } + } catch (final MetadataException e) { + LOGGER.error(ERROR_NAME, e); + result.setCode(TSStatusCode.DATABASE_NOT_EXIST.getStatusCode()).setMessage(ERROR_NAME); + } finally { + databaseReadWriteLock.writeLock().unlock(); + } + return result; + } + /** * Adjust the maximum RegionGroup count of each Database. * diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/partition/DataPartitionTableIntegrityCheckProcedure.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/partition/DataPartitionTableIntegrityCheckProcedure.java index 0970c3ac05770..ad5daad45cf76 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/partition/DataPartitionTableIntegrityCheckProcedure.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/impl/partition/DataPartitionTableIntegrityCheckProcedure.java @@ -352,7 +352,7 @@ private Flow analyzeMissingPartitions(final ConfigNodeProcedureEnv env) { } if (localEarliestSlotStartTime - > TimePartitionUtils.getStartTimeByPartitionId(earliestTimeslot)) { + > TimePartitionUtils.getStartTimeByPartitionId(earliestTimeslot, database)) { databasesWithLostDataPartition.add(database); LOG.warn( "[DataPartitionIntegrity] Database {} has lost timeslot {} in its data table partition, and this issue needs to be repaired", diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java index e58c7d59dfc4f..52f706fe99183 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java @@ -71,6 +71,7 @@ import org.apache.iotdb.confignode.consensus.request.write.database.SetSchemaReplicationFactorPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTTLPlan; import org.apache.iotdb.confignode.consensus.request.write.database.SetTimePartitionIntervalPlan; +import org.apache.iotdb.confignode.consensus.request.write.database.SetTimePartitionOriginPlan; import org.apache.iotdb.confignode.consensus.request.write.datanode.RemoveDataNodePlan; import org.apache.iotdb.confignode.consensus.response.ainode.AINodeConfigurationResp; import org.apache.iotdb.confignode.consensus.response.ainode.AINodeRegisterResp; @@ -194,6 +195,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaReplicationFactorReq; import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaTemplateReq; import org.apache.iotdb.confignode.rpc.thrift.TSetTimePartitionIntervalReq; +import org.apache.iotdb.confignode.rpc.thrift.TSetTimePartitionOriginReq; import org.apache.iotdb.confignode.rpc.thrift.TShowAINodesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowCQResp; import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp; @@ -514,6 +516,12 @@ public TSStatus setTimePartitionInterval(final TSetTimePartitionIntervalReq req) new SetTimePartitionIntervalPlan(req.getDatabase(), req.getTimePartitionInterval())); } + @Override + public TSStatus setTimePartitionOrigin(final TSetTimePartitionOriginReq req) throws TException { + return configManager.setTimePartitionOrigin( + new SetTimePartitionOriginPlan(req.getDatabase(), req.getTimePartitionOrigin())); + } + @Override public TCountDatabaseResp countMatchedDatabases(final TGetDatabaseReq req) { final PathPatternTree scope = diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java index 109e3c0d337d2..086a0bac931ab 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/consensus/request/ConfigPhysicalPlanSerDeTest.java @@ -486,10 +486,21 @@ public void AutoCleanPartitionTablePlan() throws IOException { databaseTTLMap.put("root.db1", -1L); // NULL_TTL databaseTTLMap.put("root.db2", 3600L * 1000 * 24); // 1d_TTL databaseTTLMap.put("root.db3", 3600L * 1000 * 24 * 30); // 1m_TTL - TTimePartitionSlot currentTimeSlot = - new TTimePartitionSlot(TimePartitionUtils.getTimePartitionSlot(System.currentTimeMillis())); + TTimePartitionSlot db1CurrentTimeSlot = + new TTimePartitionSlot( + TimePartitionUtils.getTimePartitionSlot(System.currentTimeMillis(), "root.db1")); + TTimePartitionSlot db2CurrentTimeSlot = + new TTimePartitionSlot( + TimePartitionUtils.getTimePartitionSlot(System.currentTimeMillis(), "root.db2")); + TTimePartitionSlot db3CurrentTimeSlot = + new TTimePartitionSlot( + TimePartitionUtils.getTimePartitionSlot(System.currentTimeMillis(), "root.db3")); + Map currentTimeSlotMap = new HashMap<>(); + currentTimeSlotMap.put("root.db1", db1CurrentTimeSlot); + currentTimeSlotMap.put("root.db2", db2CurrentTimeSlot); + currentTimeSlotMap.put("root.db3", db3CurrentTimeSlot); AutoCleanPartitionTablePlan req0 = - new AutoCleanPartitionTablePlan(databaseTTLMap, currentTimeSlot); + new AutoCleanPartitionTablePlan(databaseTTLMap, currentTimeSlotMap); AutoCleanPartitionTablePlan req1 = (AutoCleanPartitionTablePlan) ConfigPhysicalPlan.Factory.create(req0.serializeToByteBuffer()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/partition/DataPartitionTableGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/partition/DataPartitionTableGenerator.java index 1b43cb2bc5446..2202ae182ab60 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/partition/DataPartitionTableGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/partition/DataPartitionTableGenerator.java @@ -154,14 +154,16 @@ private void generateDataPartitionTableByMemory() { new ConcurrentHashMap<>(); tsFileManager.readLock(); - List seqTsFileList = tsFileManager.getTsFileList(true); - List unseqTsFileList = tsFileManager.getTsFileList(false); + List seqTsFileList = + tsFileManager.getTsFileList(true, databaseName); + List unseqTsFileList = + tsFileManager.getTsFileList(false, databaseName); tsFileManager.readUnlock(); constructDataPartitionMap( - seqTsFileList, seriesPartitionExecutor, dataPartitionMap); + seqTsFileList, seriesPartitionExecutor, dataPartitionMap, databaseName); constructDataPartitionMap( - unseqTsFileList, seriesPartitionExecutor, dataPartitionMap); + unseqTsFileList, seriesPartitionExecutor, dataPartitionMap, databaseName); if (dataPartitionMap.isEmpty()) { LOG.error("Failed to generate DataPartitionTable, dataPartitionMap is empty"); @@ -210,7 +212,8 @@ private void generateDataPartitionTableByMemory() { private void constructDataPartitionMap( List seqTsFileList, SeriesPartitionExecutor seriesPartitionExecutor, - Map dataPartitionMap) { + Map dataPartitionMap, + String database) { Set timeSlotIds = Collections.newSetFromMap(new ConcurrentHashMap<>()); for (TsFileResource tsFileResource : seqTsFileList) { @@ -227,10 +230,12 @@ private void constructDataPartitionMap( TSeriesPartitionSlot seriesSlotId = seriesPartitionExecutor.getSeriesPartitionSlot(deviceId); TTimePartitionSlot timePartitionSlot = - new TTimePartitionSlot(TimePartitionUtils.getStartTimeByPartitionId(timeSlotId)); + new TTimePartitionSlot( + TimePartitionUtils.getStartTimeByPartitionId(timeSlotId, database)); dataPartitionMap .computeIfAbsent( - seriesSlotId, empty -> newSeriesPartitionTable(consensusGroupId, timeSlotId)) + seriesSlotId, + empty -> newSeriesPartitionTable(consensusGroupId, timeSlotId, database)) .putDataPartition(timePartitionSlot, consensusGroupId); } if (!timeSlotIds.contains(timeSlotId)) { @@ -250,10 +255,10 @@ private void constructDataPartitionMap( } private static SeriesPartitionTable newSeriesPartitionTable( - TConsensusGroupId consensusGroupId, long timeSlotId) { + TConsensusGroupId consensusGroupId, long timeSlotId, String database) { SeriesPartitionTable seriesPartitionTable = new SeriesPartitionTable(); TTimePartitionSlot timePartitionSlot = - new TTimePartitionSlot(TimePartitionUtils.getStartTimeByPartitionId(timeSlotId)); + new TTimePartitionSlot(TimePartitionUtils.getStartTimeByPartitionId(timeSlotId, database)); seriesPartitionTable.putDataPartition(timePartitionSlot, consensusGroupId); return seriesPartitionTable; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ReplicateProgressDataNodeManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ReplicateProgressDataNodeManager.java index 653fbe5c81bb8..c7b05d6f08c9a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ReplicateProgressDataNodeManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/consensus/ReplicateProgressDataNodeManager.java @@ -102,13 +102,15 @@ private void recoverMaxProgressIndexFromDataRegion() { final TsFileManager tsFileManager = StorageEngine.getInstance().getDataRegion(dataRegionId).getTsFileManager(); + final String database = + StorageEngine.getInstance().getDataRegion(dataRegionId).getDatabaseName(); final List allProgressIndex = new ArrayList<>(); allProgressIndex.addAll( - tsFileManager.getTsFileList(true).stream() + tsFileManager.getTsFileList(true, database).stream() .map(TsFileResource::getMaxProgressIndex) .collect(Collectors.toList())); allProgressIndex.addAll( - tsFileManager.getTsFileList(false).stream() + tsFileManager.getTsFileList(false, database).stream() .map(TsFileResource::getMaxProgressIndex) .collect(Collectors.toList())); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionSource.java index 66f8d48ce2813..2b34b73ab9741 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionSource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/historical/PipeHistoricalDataRegionTsFileAndDeletionSource.java @@ -557,7 +557,7 @@ private void extractTsFiles( startIndex); final Map> sequenceTsFileResources2TableNames = - tsFileManager.getTsFileList(true).stream() + tsFileManager.getTsFileList(true, dataRegion.getDatabaseName()).stream() .peek(originalResourceList::add) .filter( resource -> @@ -590,7 +590,7 @@ && mayTsFileResourceOverlappedWithPattern(resource))) filteredTsFileResources2TableNames.putAll(sequenceTsFileResources2TableNames); final Map> unSequenceTsFileResources2TableNames = - tsFileManager.getTsFileList(false).stream() + tsFileManager.getTsFileList(false, dataRegion.getDatabaseName()).stream() .peek(originalResourceList::add) .filter( resource -> diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/PipeRealtimeDataRegionSource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/PipeRealtimeDataRegionSource.java index 1e47a48e500da..4a4cabfac3d6f 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/PipeRealtimeDataRegionSource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/pipe/source/dataregion/realtime/PipeRealtimeDataRegionSource.java @@ -232,8 +232,9 @@ public void customize( final DataRegion dataRegion = StorageEngine.getInstance().getDataRegion(new DataRegionId(environment.getRegionId())); + final String databaseName; if (dataRegion != null) { - final String databaseName = dataRegion.getDatabaseName(); + databaseName = dataRegion.getDatabaseName(); if (databaseName != null) { if (PathUtils.isTableModelDatabase(databaseName)) { isDbNameCoveredByPattern = tablePattern.coversDb(databaseName); @@ -241,16 +242,22 @@ public void customize( isDbNameCoveredByPattern = treePattern.coversDb(databaseName); } } + } else { + databaseName = null; } startTimePartitionIdLowerBound = - (realtimeDataExtractionStartTime % TimePartitionUtils.getTimePartitionInterval() == 0) - ? TimePartitionUtils.getTimePartitionId(realtimeDataExtractionStartTime) - : TimePartitionUtils.getTimePartitionId(realtimeDataExtractionStartTime) + 1; + (realtimeDataExtractionStartTime % TimePartitionUtils.getTimePartitionInterval(databaseName) + == 0) + ? TimePartitionUtils.getTimePartitionId(realtimeDataExtractionStartTime, databaseName) + : TimePartitionUtils.getTimePartitionId(realtimeDataExtractionStartTime, databaseName) + + 1; endTimePartitionIdUpperBound = - (realtimeDataExtractionEndTime % TimePartitionUtils.getTimePartitionInterval() == 0) - ? TimePartitionUtils.getTimePartitionId(realtimeDataExtractionEndTime) - : TimePartitionUtils.getTimePartitionId(realtimeDataExtractionEndTime) - 1; + (realtimeDataExtractionEndTime % TimePartitionUtils.getTimePartitionInterval(databaseName) + == 0) + ? TimePartitionUtils.getTimePartitionId(realtimeDataExtractionEndTime, databaseName) + : TimePartitionUtils.getTimePartitionId(realtimeDataExtractionEndTime, databaseName) + - 1; final boolean isDoubleLiving = parameters.getBooleanOrDefault( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java index e2c04caedfb20..1ba15c78d652e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/protocol/client/ConfigNodeClient.java @@ -152,6 +152,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaReplicationFactorReq; import org.apache.iotdb.confignode.rpc.thrift.TSetSchemaTemplateReq; import org.apache.iotdb.confignode.rpc.thrift.TSetTimePartitionIntervalReq; +import org.apache.iotdb.confignode.rpc.thrift.TSetTimePartitionOriginReq; import org.apache.iotdb.confignode.rpc.thrift.TShowAINodesResp; import org.apache.iotdb.confignode.rpc.thrift.TShowCQResp; import org.apache.iotdb.confignode.rpc.thrift.TShowClusterResp; @@ -647,6 +648,12 @@ public TSStatus setTimePartitionInterval(TSetTimePartitionIntervalReq req) throw () -> client.setTimePartitionInterval(req), status -> !updateConfigNodeLeader(status)); } + @Override + public TSStatus setTimePartitionOrigin(TSetTimePartitionOriginReq req) throws TException { + return executeRemoteCallWithRetry( + () -> client.setTimePartitionOrigin(req), status -> !updateConfigNodeLeader(status)); + } + @Override public TSchemaPartitionTableResp getSchemaPartitionTable(final TSchemaPartitionReq req) throws TException { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeUtils.java index 2e0b0b52fdc09..199ea04e6ec29 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeUtils.java @@ -128,15 +128,16 @@ public static String getDatabaseName( public static List computeTableDataPartitionParams( final InsertBaseStatement statement, final MPPQueryContext context) { + String database = getDatabaseName(statement, context); if (statement instanceof InsertTabletStatement) { final InsertTabletStatement insertTabletStatement = (InsertTabletStatement) statement; final Map> timePartitionSlotMap = new HashMap<>(); for (int i = 0; i < insertTabletStatement.getRowCount(); i++) { timePartitionSlotMap .computeIfAbsent(insertTabletStatement.getTableDeviceID(i), id -> new HashSet<>()) - .add(insertTabletStatement.getTimePartitionSlot(i)); + .add(insertTabletStatement.getTimePartitionSlot(i, database)); } - return computeDataPartitionParams(timePartitionSlotMap, getDatabaseName(statement, context)); + return computeDataPartitionParams(timePartitionSlotMap, database); } else if (statement instanceof InsertMultiTabletsStatement) { final InsertMultiTabletsStatement insertMultiTabletsStatement = (InsertMultiTabletsStatement) statement; @@ -146,17 +147,17 @@ public static List computeTableDataPartitionParams( for (int i = 0; i < insertTabletStatement.getRowCount(); i++) { timePartitionSlotMap .computeIfAbsent(insertTabletStatement.getTableDeviceID(i), id -> new HashSet<>()) - .add(insertTabletStatement.getTimePartitionSlot(i)); + .add(insertTabletStatement.getTimePartitionSlot(i, database)); } } - return computeDataPartitionParams(timePartitionSlotMap, getDatabaseName(statement, context)); + return computeDataPartitionParams(timePartitionSlotMap, database); } else if (statement instanceof InsertRowStatement) { final InsertRowStatement insertRowStatement = (InsertRowStatement) statement; return computeDataPartitionParams( Collections.singletonMap( insertRowStatement.getTableDeviceID(), - Collections.singleton(insertRowStatement.getTimePartitionSlot())), - getDatabaseName(statement, context)); + Collections.singleton(insertRowStatement.getTimePartitionSlot(database))), + database); } else if (statement instanceof InsertRowsStatement) { final InsertRowsStatement insertRowsStatement = (InsertRowsStatement) statement; final Map> timePartitionSlotMap = new HashMap<>(); @@ -164,9 +165,9 @@ public static List computeTableDataPartitionParams( insertRowsStatement.getInsertRowStatementList()) { timePartitionSlotMap .computeIfAbsent(insertRowStatement.getTableDeviceID(), id -> new HashSet<>()) - .add(insertRowStatement.getTimePartitionSlot()); + .add(insertRowStatement.getTimePartitionSlot(database)); } - return computeDataPartitionParams(timePartitionSlotMap, getDatabaseName(statement, context)); + return computeDataPartitionParams(timePartitionSlotMap, database); } throw new UnsupportedOperationException("computeDataPartitionParams for " + statement); } @@ -187,7 +188,8 @@ public static List computeTreeDataPartitionParams( dataPartitionQueryParamMap.computeIfAbsent( insertTabletStatement.getDevicePath().getIDeviceIDAsFullDevice(), k -> new HashSet<>()); - timePartitionSlotSet.addAll(insertTabletStatement.getTimePartitionSlots()); + timePartitionSlotSet.addAll( + insertTabletStatement.getTimePartitionSlots(getDatabaseName(statement, context))); } return computeDataPartitionParams( dataPartitionQueryParamMap, getDatabaseName(statement, context)); @@ -200,7 +202,8 @@ public static List computeTreeDataPartitionParams( dataPartitionQueryParamMap.computeIfAbsent( insertRowStatement.getDevicePath().getIDeviceIDAsFullDevice(), k -> new HashSet<>()); - timePartitionSlotSet.add(insertRowStatement.getTimePartitionSlot()); + timePartitionSlotSet.add( + insertRowStatement.getTimePartitionSlot(getDatabaseName(statement, context))); } return computeDataPartitionParams( dataPartitionQueryParamMap, getDatabaseName(statement, context)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java index dacdc69ba4596..f81aceda62c36 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/AnalyzeVisitor.java @@ -2130,7 +2130,8 @@ private DataPartition fetchDataPartitionByDevices( long startTime = System.nanoTime(); try { Pair, Pair> res = - getTimePartitionSlotList(context.getGlobalTimeFilter(), context); + getTimePartitionSlotList( + context.getGlobalTimeFilter(), context, context.getDatabaseName().orElse(null)); // there is no satisfied time range if (res.left.isEmpty() && Boolean.FALSE.equals(res.right.left)) { return new DataPartition( @@ -2168,7 +2169,7 @@ private DataPartition fetchDataPartitionByDevices( */ @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning public static Pair, Pair> getTimePartitionSlotList( - Filter timeFilter, MPPQueryContext context) { + Filter timeFilter, MPPQueryContext context, String database) { if (timeFilter == null) { // (-oo, +oo) return new Pair<>(Collections.emptyList(), new Pair<>(true, true)); @@ -2193,11 +2194,15 @@ public static Pair, Pair> getTimePart if (timeRangeList.get(0).getMin() == Long.MIN_VALUE) { needLeftAll = true; - endTime = TimePartitionUtils.getTimePartitionUpperBound(timeRangeList.get(0).getMax()); - timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(timeRangeList.get(0).getMax()); + endTime = + TimePartitionUtils.getTimePartitionUpperBound(timeRangeList.get(0).getMax(), database); + timePartitionSlot = + TimePartitionUtils.getTimePartitionSlot(timeRangeList.get(0).getMax(), database); } else { - endTime = TimePartitionUtils.getTimePartitionUpperBound(timeRangeList.get(0).getMin()); - timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(timeRangeList.get(0).getMin()); + endTime = + TimePartitionUtils.getTimePartitionUpperBound(timeRangeList.get(0).getMin(), database); + timePartitionSlot = + TimePartitionUtils.getTimePartitionSlot(timeRangeList.get(0).getMin(), database); needLeftAll = false; } @@ -2211,30 +2216,30 @@ public static Pair, Pair> getTimePart List result = new ArrayList<>(); TimeRange currentTimeRange = timeRangeList.get(index); reserveMemoryForTimePartitionSlot( - currentTimeRange.getMax(), currentTimeRange.getMin(), context); + currentTimeRange.getMax(), currentTimeRange.getMin(), context, database); while (index < size) { long curLeft = timeRangeList.get(index).getMin(); long curRight = timeRangeList.get(index).getMax(); if (curLeft >= endTime) { result.add(timePartitionSlot); // next init - endTime = TimePartitionUtils.getTimePartitionUpperBound(curLeft); - timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(curLeft); + endTime = TimePartitionUtils.getTimePartitionUpperBound(curLeft, database); + timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(curLeft, database); } else if (curRight >= endTime) { result.add(timePartitionSlot); // next init timePartitionSlot = new TTimePartitionSlot(endTime); // beware of overflow endTime = - endTime + TimePartitionUtils.getTimePartitionInterval() > endTime - ? endTime + TimePartitionUtils.getTimePartitionInterval() + endTime + TimePartitionUtils.getTimePartitionInterval(database) > endTime + ? endTime + TimePartitionUtils.getTimePartitionInterval(database) : Long.MAX_VALUE; } else { index++; if (index < size) { currentTimeRange = timeRangeList.get(index); reserveMemoryForTimePartitionSlot( - currentTimeRange.getMax(), currentTimeRange.getMin(), context); + currentTimeRange.getMax(), currentTimeRange.getMin(), context, database); } } } @@ -2243,7 +2248,7 @@ public static Pair, Pair> getTimePart if (needRightAll) { TTimePartitionSlot lastTimePartitionSlot = TimePartitionUtils.getTimePartitionSlot( - timeRangeList.get(timeRangeList.size() - 1).getMin()); + timeRangeList.get(timeRangeList.size() - 1).getMin(), database); if (lastTimePartitionSlot.startTime != timePartitionSlot.startTime) { result.add(lastTimePartitionSlot); } @@ -2252,11 +2257,11 @@ public static Pair, Pair> getTimePart } private static void reserveMemoryForTimePartitionSlot( - long maxTime, long minTime, MPPQueryContext context) { + long maxTime, long minTime, MPPQueryContext context, String database) { if (maxTime == Long.MAX_VALUE || minTime == Long.MIN_VALUE) { return; } - long size = TimePartitionUtils.getEstimateTimePartitionSize(minTime, maxTime); + long size = TimePartitionUtils.getEstimateTimePartitionSize(minTime, maxTime, database); context.reserveMemoryForFrontEnd( RamUsageEstimator.shallowSizeOfInstance(TTimePartitionSlot.class) * size); } @@ -2774,7 +2779,8 @@ public Analysis visitInsertRow(InsertRowStatement insertRowStatement, MPPQueryCo dataPartitionQueryParam.setDeviceID( realInsertRowStatement.getDevicePath().getIDeviceIDAsFullDevice()); dataPartitionQueryParam.setTimePartitionSlotList( - Collections.singletonList(realInsertRowStatement.getTimePartitionSlot())); + Collections.singletonList( + realInsertRowStatement.getTimePartitionSlot(context.getDatabaseName().orElse(null)))); AnalyzeUtils.analyzeDataPartition( analysis, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/TemplatedAnalyze.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/TemplatedAnalyze.java index 21b8d9d8dc105..c3313e3ed33f1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/TemplatedAnalyze.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/TemplatedAnalyze.java @@ -392,8 +392,9 @@ private static DataPartition fetchDataPartitionByDevices( IPartitionFetcher partitionFetcher) { long startTime = System.nanoTime(); try { + String database = context.getDatabaseName().orElse(null); Pair, Pair> res = - getTimePartitionSlotList(context.getGlobalTimeFilter(), context); + getTimePartitionSlotList(context.getGlobalTimeFilter(), context, database); // there is no satisfied time range if (res.left.isEmpty() && Boolean.FALSE.equals(res.right.left)) { return new DataPartition( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java index cf404e8d57b76..292a9e1e4c719 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/executor/ClusterConfigTaskExecutor.java @@ -3357,9 +3357,11 @@ public SettableFuture getRegionId( tGetRegionIdReq.setDatabase(getRegionIdStatement.getDatabase()); } tGetRegionIdReq.setStartTimeSlot( - TimePartitionUtils.getTimePartitionSlot(getRegionIdStatement.getStartTimeStamp())); + TimePartitionUtils.getTimePartitionSlot( + getRegionIdStatement.getStartTimeStamp(), getRegionIdStatement.getDatabase())); tGetRegionIdReq.setEndTimeSlot( - TimePartitionUtils.getTimePartitionSlot(getRegionIdStatement.getEndTimeStamp())); + TimePartitionUtils.getTimePartitionSlot( + getRegionIdStatement.getEndTimeStamp(), getRegionIdStatement.getDatabase())); resp = configNodeClient.getRegionId(tGetRegionIdReq); if (resp.getStatus().getCode() != TSStatusCode.SUCCESS_STATUS.getStatusCode()) { future.setException(new IoTDBException(resp.getStatus())); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/DatabaseSchemaTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/DatabaseSchemaTask.java index ab9dee095aaab..fa21a93301fc3 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/DatabaseSchemaTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/DatabaseSchemaTask.java @@ -66,6 +66,9 @@ public static TDatabaseSchema constructDatabaseSchema( if (databaseSchemaStatement.getTimePartitionInterval() != null) { databaseSchema.setTimePartitionInterval(databaseSchemaStatement.getTimePartitionInterval()); } + if (databaseSchemaStatement.getTimePartitionOrigin() != null) { + databaseSchema.setTimePartitionOrigin(databaseSchemaStatement.getTimePartitionOrigin()); + } if (databaseSchemaStatement.getSchemaRegionGroupNum() != null) { databaseSchema.setMinSchemaRegionGroupNum(databaseSchemaStatement.getSchemaRegionGroupNum()); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java index 421ead09c8988..bf115774dcec4 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/parser/ASTVisitor.java @@ -2967,6 +2967,9 @@ private void parseDatabaseAttributesClause( } else if (attributeKey.TIME_PARTITION_INTERVAL() != null) { final long timePartitionInterval = Long.parseLong(attribute.INTEGER_LITERAL().getText()); databaseSchemaStatement.setTimePartitionInterval(timePartitionInterval); + } else if (attributeKey.TIME_PARTITION_ORIGIN() != null) { + final long timePartitionOrigin = Long.parseLong(attribute.INTEGER_LITERAL().getText()); + databaseSchemaStatement.setTimePartitionOrigin(timePartitionOrigin); } else if (attributeKey.SCHEMA_REGION_GROUP_NUM() != null) { final int schemaRegionGroupNum = Integer.parseInt(attribute.INTEGER_LITERAL().getText()); databaseSchemaStatement.setSchemaRegionGroupNum(schemaRegionGroupNum); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SourceRewriter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SourceRewriter.java index 4e69469a497bb..43990b4171876 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SourceRewriter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/distribution/SourceRewriter.java @@ -1452,14 +1452,18 @@ private List getDeviceReplicaSets( tSeriesPartitionSlot.slotId, k -> getDataRegionReplicaSetWithTimeFilter( - finalSeriesPartitionMap, tSeriesPartitionSlot, timeFilter)); + finalSeriesPartitionMap, + tSeriesPartitionSlot, + timeFilter, + analysis.getDatabaseName())); } public List getDataRegionReplicaSetWithTimeFilter( Map>> seriesPartitionMap, TSeriesPartitionSlot tSeriesPartitionSlot, - Filter timeFilter) { + Filter timeFilter, + String database) { Map> regionReplicaSetMap = seriesPartitionMap.getOrDefault(tSeriesPartitionSlot, Collections.emptyMap()); if (regionReplicaSetMap.isEmpty()) { @@ -1469,7 +1473,8 @@ public List getDataRegionReplicaSetWithTimeFilter( Set uniqueValues = new HashSet<>(); for (Map.Entry> entry : regionReplicaSetMap.entrySet()) { - if (!TimePartitionUtils.satisfyPartitionStartTime(timeFilter, entry.getKey().startTime)) { + if (!TimePartitionUtils.satisfyPartitionStartTime( + timeFilter, entry.getKey().startTime, database)) { continue; } for (TRegionReplicaSet tRegionReplicaSet : entry.getValue()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeId.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeId.java index fd842616f7cd7..9a12817356a22 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeId.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/PlanNodeId.java @@ -38,6 +38,10 @@ public PlanNodeId(String id) { this.id = id; } + public PlanNodeId(String id, String database) { + this.id = database != null ? database + "_" + id : id; + } + public String getId() { return this.id; } @@ -68,6 +72,10 @@ public static PlanNodeId deserialize(InputStream stream) throws IOException { return new PlanNodeId(ReadWriteIOUtils.readString(stream)); } + public static PlanNodeId deserialize(InputStream stream, String database) throws IOException { + return new PlanNodeId(ReadWriteIOUtils.readString(stream), database); + } + public void serialize(ByteBuffer byteBuffer) { ReadWriteIOUtils.write(id, byteBuffer); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/load/LoadSingleTsFileNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/load/LoadSingleTsFileNode.java index c8170a4880a08..268751127f3ec 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/load/LoadSingleTsFileNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/load/LoadSingleTsFileNode.java @@ -103,10 +103,14 @@ public boolean needDecodeTsFile( // iterating the index, must present slotList.add( new Pair<>( - o, TimePartitionUtils.getTimePartitionSlot(resource.getStartTime(o).get()))); + o, + TimePartitionUtils.getTimePartitionSlot( + resource.getStartTime(o).get(), database))); slotList.add( new Pair<>( - o, TimePartitionUtils.getTimePartitionSlot(resource.getEndTime(o).get()))); + o, + TimePartitionUtils.getTimePartitionSlot( + resource.getEndTime(o).get(), database))); }); if (slotList.isEmpty()) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowNode.java index a2bd6cb1a00fc..d6cf24e4b1e03 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowNode.java @@ -124,7 +124,8 @@ public InsertRowNode( @Override public List splitByPartition(IAnalysis analysis) { - TTimePartitionSlot timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(time); + TTimePartitionSlot timePartitionSlot = + TimePartitionUtils.getTimePartitionSlot(time, analysis.getDatabaseName()); this.dataRegionReplicaSet = analysis .getDataPartitionInfo() @@ -225,8 +226,8 @@ public void setNeedInferType(boolean needInferType) { } @TestOnly - public List getTimePartitionSlots() { - return Collections.singletonList(TimePartitionUtils.getTimePartitionSlot(time)); + public List getTimePartitionSlots(String database) { + return Collections.singletonList(TimePartitionUtils.getTimePartitionSlot(time, database)); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsNode.java index 7392b7612705e..89385ccad687e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsNode.java @@ -277,7 +277,8 @@ public List splitByPartition(IAnalysis analysis) { .getDataPartitionInfo() .getDataRegionReplicaSetForWriting( insertRowNode.targetPath.getIDeviceIDAsFullDevice(), - TimePartitionUtils.getTimePartitionSlot(insertRowNode.getTime()), + TimePartitionUtils.getTimePartitionSlot( + insertRowNode.getTime(), analysis.getDatabaseName()), null); // Collect redirectInfo redirectInfo.add(dataRegionReplicaSet.getDataNodeLocations().get(0).getClientRpcEndPoint()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsOfOneDeviceNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsOfOneDeviceNode.java index f1e28d32b104d..274261179dc80 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsOfOneDeviceNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertRowsOfOneDeviceNode.java @@ -168,7 +168,8 @@ public List splitByPartition(IAnalysis analysis) { for (int i = 0; i < insertRowNodeList.size(); i++) { InsertRowNode insertRowNode = insertRowNodeList.get(i); TTimePartitionSlot timePartitionSlot = - TimePartitionUtils.getTimePartitionSlot(insertRowNode.getTime()); + TimePartitionUtils.getTimePartitionSlot( + insertRowNode.getTime(), analysis.getDatabaseName()); TRegionReplicaSet dataRegionReplicaSet = analysis .getDataPartitionInfo() diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java index 39683e5d9f94e..6c5797b352b3b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertTabletNode.java @@ -235,15 +235,18 @@ public List splitByPartition(IAnalysis analysis) { return Collections.emptyList(); } - final Map deviceIDSplitInfoMap = collectSplitRanges(); + final Map deviceIDSplitInfoMap = + collectSplitRanges(analysis.getDatabaseName()); final Map> splitMap = splitByReplicaSet(deviceIDSplitInfoMap, analysis); return doSplit(splitMap); } - private Map collectSplitRanges() { - long upperBoundOfTimePartition = TimePartitionUtils.getTimePartitionUpperBound(times[0]); - TTimePartitionSlot timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(times[0]); + private Map collectSplitRanges(String database) { + long upperBoundOfTimePartition = + TimePartitionUtils.getTimePartitionUpperBound(times[0], database); + TTimePartitionSlot timePartitionSlot = + TimePartitionUtils.getTimePartitionSlot(times[0], database); int startLoc = 0; // included IDeviceID currDeviceId = getDeviceID(0); @@ -261,8 +264,9 @@ private Map collectSplitRanges() { splitInfo.timePartitionSlots.add(timePartitionSlot); // next init startLoc = i; - upperBoundOfTimePartition = TimePartitionUtils.getTimePartitionUpperBound(times[i]); - timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(times[i]); + upperBoundOfTimePartition = + TimePartitionUtils.getTimePartitionUpperBound(times[i], database); + timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(times[i], database); currDeviceId = nextDeviceId; } } @@ -381,16 +385,19 @@ protected WritePlanNode generateOneSplit(Map.Entry getTimePartitionSlots() { + public List getTimePartitionSlots(String database) { List result = new ArrayList<>(); - long upperBoundOfTimePartition = TimePartitionUtils.getTimePartitionUpperBound(times[0]); - TTimePartitionSlot timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(times[0]); + long upperBoundOfTimePartition = + TimePartitionUtils.getTimePartitionUpperBound(times[0], database); + TTimePartitionSlot timePartitionSlot = + TimePartitionUtils.getTimePartitionSlot(times[0], database); for (int i = 1; i < times.length; i++) { // times are sorted in session API. if (times[i] >= upperBoundOfTimePartition) { result.add(timePartitionSlot); // next init - upperBoundOfTimePartition = TimePartitionUtils.getTimePartitionUpperBound(times[i]); - timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(times[i]); + upperBoundOfTimePartition = + TimePartitionUtils.getTimePartitionUpperBound(times[i], database); + timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(times[i], database); } } result.add(timePartitionSlot); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertRowsNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertRowsNode.java index 594ccf50471f9..e935cf733e718 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertRowsNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertRowsNode.java @@ -173,7 +173,8 @@ public List splitByPartition(IAnalysis analysis) { .getDataPartitionInfo() .getDataRegionReplicaSetForWriting( insertRowNode.getDeviceID(), - TimePartitionUtils.getTimePartitionSlot(insertRowNode.getTime()), + TimePartitionUtils.getTimePartitionSlot( + insertRowNode.getTime(), analysis.getDatabaseName()), analysis.getDatabaseName()); // Collect redirectInfo diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanGenerator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanGenerator.java index 79ea52597bcee..26c3c39432919 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanGenerator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/distribute/TableDistributedPlanGenerator.java @@ -1591,6 +1591,7 @@ private List getDeviceReplicaSets( IDeviceID deviceId, Filter timeFilter, Map> cachedSeriesSlotWithRegions) { + String database = dataPartition.getDatabaseNameByDevice(deviceId); // given seriesPartitionSlot has already been calculated final TSeriesPartitionSlot seriesPartitionSlot = dataPartition.calculateDeviceGroupId(deviceId); @@ -1610,7 +1611,8 @@ private List getDeviceReplicaSets( } if (timeSlotMap.size() == 1) { TTimePartitionSlot timePartitionSlot = timeSlotMap.keySet().iterator().next(); - if (TimePartitionUtils.satisfyPartitionStartTime(timeFilter, timePartitionSlot.startTime)) { + if (TimePartitionUtils.satisfyPartitionStartTime( + timeFilter, timePartitionSlot.startTime, database)) { cachedSeriesSlotWithRegions.put( seriesPartitionSlot.getSlotId(), timeSlotMap.values().iterator().next()); return timeSlotMap.values().iterator().next(); @@ -1623,7 +1625,8 @@ private List getDeviceReplicaSets( Set resultSet = new HashSet<>(); for (Map.Entry> entry : timeSlotMap.entrySet()) { TTimePartitionSlot timePartitionSlot = entry.getKey(); - if (TimePartitionUtils.satisfyPartitionStartTime(timeFilter, timePartitionSlot.startTime)) { + if (TimePartitionUtils.satisfyPartitionStartTime( + timeFilter, timePartitionSlot.startTime, database)) { resultSet.addAll(entry.getValue()); } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java index 7bd4530afd194..e14c5ea67b0d1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java @@ -1308,7 +1308,7 @@ private DataPartition fetchDataPartitionByDevices( final List deviceEntries, final Filter globalTimeFilter) { final Pair, Pair> res = - getTimePartitionSlotList(globalTimeFilter, queryContext); + getTimePartitionSlotList(globalTimeFilter, queryContext, database); // there is no satisfied time range if (res.left.isEmpty() && Boolean.FALSE.equals(res.right.left)) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java index bc8446fd3dae3..b434c773a3474 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/scheduler/load/LoadTsFileScheduler.java @@ -319,7 +319,7 @@ private boolean firstPhase(LoadSingleTsFileNode node) { try { new TsFileSplitter( node.getTsFileResource().getTsFile(), tsFileDataManager::addOrSendTsFileData) - .splitTsFileByDataPartition(); + .splitTsFileByDataPartition(node.getDatabase()); if (!tsFileDataManager.sendAllTsFileData()) { return false; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowStatement.java index 1cb5abed66ef5..9b3f498b88cfb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowStatement.java @@ -175,8 +175,8 @@ public void fillValues(ByteBuffer buffer) throws QueryProcessException { } } - public TTimePartitionSlot getTimePartitionSlot() { - return TimePartitionUtils.getTimePartitionSlot(time); + public TTimePartitionSlot getTimePartitionSlot(String database) { + return TimePartitionUtils.getTimePartitionSlot(time, database); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowsOfOneDeviceStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowsOfOneDeviceStatement.java index 3336695ff0e86..79edacd21d8ad 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowsOfOneDeviceStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertRowsOfOneDeviceStatement.java @@ -92,7 +92,8 @@ public List getTimePartitionSlots() { Set timePartitionSlotSet = new HashSet<>(); for (InsertRowStatement insertRowStatement : insertRowStatementList) { timePartitionSlotSet.add( - TimePartitionUtils.getTimePartitionSlot(insertRowStatement.getTime())); + TimePartitionUtils.getTimePartitionSlot( + insertRowStatement.getTime(), getDatabaseName(insertRowStatement))); } return new ArrayList<>(timePartitionSlotSet); } @@ -222,4 +223,11 @@ public Optional getDatabaseName() { protected void subRemoveAttributeColumns(List columnsToKeep) { insertRowStatementList.forEach(InsertRowStatement::removeAttributeColumns); } + + public static String getDatabaseName(final InsertBaseStatement statement) { + if (statement.getDatabaseName().isPresent()) { + return statement.getDatabaseName().get(); + } + return null; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java index 12369d81bfd30..ef387f519c8ec 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/crud/InsertTabletStatement.java @@ -245,24 +245,27 @@ public boolean isEmpty() { || columns.length == 0; } - public List getTimePartitionSlots() { + public List getTimePartitionSlots(String database) { List result = new ArrayList<>(); - long upperBoundOfTimePartition = TimePartitionUtils.getTimePartitionUpperBound(times[0]); - TTimePartitionSlot timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(times[0]); + long upperBoundOfTimePartition = + TimePartitionUtils.getTimePartitionUpperBound(times[0], database); + TTimePartitionSlot timePartitionSlot = + TimePartitionUtils.getTimePartitionSlot(times[0], database); for (int i = 1; i < times.length; i++) { // times are sorted in session API. if (times[i] >= upperBoundOfTimePartition) { result.add(timePartitionSlot); // next init - upperBoundOfTimePartition = TimePartitionUtils.getTimePartitionUpperBound(times[i]); - timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(times[i]); + upperBoundOfTimePartition = + TimePartitionUtils.getTimePartitionUpperBound(times[i], database); + timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(times[i], database); } } result.add(timePartitionSlot); return result; } - public TTimePartitionSlot getTimePartitionSlot(int i) { - return TimePartitionUtils.getTimePartitionSlot(times[i]); + public TTimePartitionSlot getTimePartitionSlot(int i, String database) { + return TimePartitionUtils.getTimePartitionSlot(times[i], database); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/DatabaseSchemaStatement.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/DatabaseSchemaStatement.java index 53ff33e2437cb..42094aeb937cb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/DatabaseSchemaStatement.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/statement/metadata/DatabaseSchemaStatement.java @@ -36,6 +36,7 @@ public class DatabaseSchemaStatement extends Statement implements IConfigStateme private PartialPath databasePath; private Long ttl = null; private Long timePartitionInterval = null; + private Long timePartitionOrigin = null; private Integer schemaRegionGroupNum = null; private Integer dataRegionGroupNum = null; private boolean enablePrintExceptionLog = true; @@ -94,6 +95,14 @@ public void setTimePartitionInterval(final Long timePartitionInterval) { this.timePartitionInterval = timePartitionInterval; } + public Long getTimePartitionOrigin() { + return timePartitionOrigin; + } + + public void setTimePartitionOrigin(Long timePartitionOrigin) { + this.timePartitionOrigin = timePartitionOrigin; + } + public Integer getSchemaRegionGroupNum() { return schemaRegionGroupNum; } @@ -152,6 +161,8 @@ public String toString() { + dataReplicationFactor + ", timePartitionInterval=" + timePartitionInterval + + ", timePartitionOrigin=" + + timePartitionOrigin + ", schemaRegionGroupNum=" + schemaRegionGroupNum + ", dataRegionGroupNum=" diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java index 2a1c21de04057..35c6cd6b9e35b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/StorageEngine.java @@ -44,7 +44,6 @@ import org.apache.iotdb.commons.utils.PathUtils; import org.apache.iotdb.commons.utils.StatusUtils; import org.apache.iotdb.commons.utils.TestOnly; -import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.consensus.ConsensusFactory; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; @@ -174,11 +173,6 @@ public static StorageEngine getInstance() { return InstanceHolder.INSTANCE; } - private static void initTimePartition() { - TimePartitionUtils.setTimePartitionInterval( - CommonDescriptor.getInstance().getConfig().getTimePartitionInterval()); - } - /** block insertion if the insertion is rejected by memory control */ public static void blockInsertionIfReject() throws WriteProcessRejectException { long startTime = System.currentTimeMillis(); @@ -299,8 +293,6 @@ public Map> getLocalDataRegionInfo() { @Override public void start() throws StartupException { recoverDataRegionNum = 0; - // build time Interval to divide time partition - initTimePartition(); // create systemDir try { FileUtils.forceMkdir(SystemFileFactory.INSTANCE.getFile(systemDir)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java index eb37cb1d5d21b..17306204d61ea 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/DataRegion.java @@ -757,11 +757,11 @@ private void recover() throws DataRegionException { dataRegionRecoveryContext.recoverPerformers) { recoverUnsealedTsFileCallBack(recoverPerformer); } - for (TsFileResource resource : tsFileManager.getTsFileList(true)) { + for (TsFileResource resource : tsFileManager.getTsFileList(true, databaseName)) { long partitionNum = resource.getTimePartition(); updatePartitionFileVersion(partitionNum, resource.getVersion()); } - for (TsFileResource resource : tsFileManager.getTsFileList(false)) { + for (TsFileResource resource : tsFileManager.getTsFileList(false, databaseName)) { long partitionNum = resource.getTimePartition(); updatePartitionFileVersion(partitionNum, resource.getVersion()); } @@ -2069,14 +2069,14 @@ public void deleteDALFolderAndClose() { /** close all tsfile resource */ public void closeAllResources() { - for (TsFileResource tsFileResource : tsFileManager.getTsFileList(false)) { + for (TsFileResource tsFileResource : tsFileManager.getTsFileList(false, databaseName)) { try { tsFileResource.close(); } catch (IOException e) { logger.error("Cannot close a TsFileResource {}", tsFileResource, e); } } - for (TsFileResource tsFileResource : tsFileManager.getTsFileList(true)) { + for (TsFileResource tsFileResource : tsFileManager.getTsFileList(true, databaseName)) { try { tsFileResource.close(); } catch (IOException e) { @@ -2097,8 +2097,8 @@ public void syncDeleteDataFiles() throws TsFileProcessorException { // normally, mergingModification is just need to be closed by after a merge task is finished. // we close it here just for IT test. closeAllResources(); - List tsFileResourceList = tsFileManager.getTsFileList(true); - tsFileResourceList.addAll(tsFileManager.getTsFileList(false)); + List tsFileResourceList = tsFileManager.getTsFileList(true, databaseName); + tsFileResourceList.addAll(tsFileManager.getTsFileList(false, databaseName)); tsFileResourceList.forEach( x -> { FileMetrics.getInstance().deleteTsFile(x.isSeq(), Collections.singletonList(x)); @@ -2401,7 +2401,7 @@ public QueryDataSource query( throws QueryProcessException { Pair, List> pair = - tsFileManager.getAllTsFileListForQuery(timePartitions, globalTimeFilter); + tsFileManager.getAllTsFileListForQuery(timePartitions, globalTimeFilter, databaseName); List seqTsFileResouceList = pair.left; List unSeqTsFileResouceList = pair.right; @@ -2573,7 +2573,7 @@ public IQueryDataSource queryForSeriesRegionScan( List timePartitions, long waitForLockTimeInMs) { Pair, List> pair = - tsFileManager.getAllTsFileListForQuery(timePartitions, globalTimeFilter); + tsFileManager.getAllTsFileListForQuery(timePartitions, globalTimeFilter, databaseName); List seqTsFileResouceList = pair.left; List unSeqTsFileResouceList = pair.right; @@ -2598,7 +2598,7 @@ public IQueryDataSource queryForSeriesRegionScan( List unSeqFileScanHandles = getFileHandleListForQuery( - tsFileManager.getTsFileList(false, timePartitions, globalTimeFilter), + tsFileManager.getTsFileList(false, timePartitions, globalTimeFilter, databaseName), pathList, queryContext, globalTimeFilter, @@ -2651,7 +2651,7 @@ public IQueryDataSource queryForDeviceRegionScan( long waitForLockTimeInMs) { Pair, List> pair = - tsFileManager.getAllTsFileListForQuery(timePartitions, globalTimeFilter); + tsFileManager.getAllTsFileListForQuery(timePartitions, globalTimeFilter, databaseName); List seqTsFileResouceList = pair.left; List unSeqTsFileResouceList = pair.right; @@ -2676,7 +2676,7 @@ public IQueryDataSource queryForDeviceRegionScan( List unSeqFileScanHandles = getFileHandleListForQuery( - tsFileManager.getTsFileList(false, timePartitions, globalTimeFilter), + tsFileManager.getTsFileList(false, timePartitions, globalTimeFilter, databaseName), devicePathToAligned, queryContext, globalTimeFilter, @@ -2822,8 +2822,9 @@ private void getTwoKindsOfTsFiles( List unsealedResource, long startTime, long endTime) { - List tsFileResources = tsFileManager.getTsFileList(true, startTime, endTime); - tsFileResources.addAll(tsFileManager.getTsFileList(false, startTime, endTime)); + List tsFileResources = + tsFileManager.getTsFileList(true, startTime, endTime, databaseName); + tsFileResources.addAll(tsFileManager.getTsFileList(false, startTime, endTime, databaseName)); tsFileResources.stream().filter(TsFileResource::isClosed).forEach(sealedResource::add); tsFileResources.stream() .filter(resource -> !resource.isClosed()) @@ -3063,12 +3064,14 @@ private List logDeletionInWAL(RelationalDeleteDataNode deleteD long startTime = modEntry.getStartTime(); long endTime = modEntry.getEndTime(); for (Map.Entry entry : workSequenceTsFileProcessors.entrySet()) { - if (TimePartitionUtils.satisfyPartitionId(startTime, endTime, entry.getKey())) { + if (TimePartitionUtils.satisfyPartitionId( + startTime, endTime, entry.getKey(), deleteDataNode.getDatabaseName())) { involvedProcessors.add(entry.getValue()); } } for (Map.Entry entry : workUnsequenceTsFileProcessors.entrySet()) { - if (TimePartitionUtils.satisfyPartitionId(startTime, endTime, entry.getKey())) { + if (TimePartitionUtils.satisfyPartitionId( + startTime, endTime, entry.getKey(), deleteDataNode.getDatabaseName())) { involvedProcessors.add(entry.getValue()); } } @@ -3104,13 +3107,13 @@ private List logDeletionInWAL( new DeleteDataNode(new PlanNodeId(""), Collections.singletonList(path), startTime, endTime); deleteDataNode.setSearchIndex(searchIndex); for (Map.Entry entry : workSequenceTsFileProcessors.entrySet()) { - if (TimePartitionUtils.satisfyPartitionId(startTime, endTime, entry.getKey())) { + if (TimePartitionUtils.satisfyPartitionId(startTime, endTime, entry.getKey(), databaseName)) { WALFlushListener walFlushListener = entry.getValue().logDeleteDataNodeInWAL(deleteDataNode); walFlushListeners.add(walFlushListener); } } for (Map.Entry entry : workUnsequenceTsFileProcessors.entrySet()) { - if (TimePartitionUtils.satisfyPartitionId(startTime, endTime, entry.getKey())) { + if (TimePartitionUtils.satisfyPartitionId(startTime, endTime, entry.getKey(), databaseName)) { WALFlushListener walFlushListener = entry.getValue().logDeleteDataNodeInWAL(deleteDataNode); walFlushListeners.add(walFlushListener); } @@ -4342,7 +4345,7 @@ private TsFileResource unloadTsFileInside(File fileToBeUnloaded) { writeLock("unloadTsFileInside"); TsFileResource unloadedTsFileResource = null; try { - Iterator sequenceIterator = tsFileManager.getIterator(true); + Iterator sequenceIterator = tsFileManager.getIterator(true, databaseName); while (sequenceIterator.hasNext()) { TsFileResource sequenceResource = sequenceIterator.next(); if (sequenceResource.getTsFile().getName().equals(fileToBeUnloaded.getName())) { @@ -4354,7 +4357,8 @@ private TsFileResource unloadTsFileInside(File fileToBeUnloaded) { } } if (unloadedTsFileResource == null) { - Iterator unsequenceIterator = tsFileManager.getIterator(false); + Iterator unsequenceIterator = + tsFileManager.getIterator(false, databaseName); while (unsequenceIterator.hasNext()) { TsFileResource unsequenceResource = unsequenceIterator.next(); if (unsequenceResource.getTsFile().getName().equals(fileToBeUnloaded.getName())) { @@ -4382,11 +4386,11 @@ public Collection getWorkUnsequenceTsFileProcessors() { } public List getSequenceFileList() { - return tsFileManager.getTsFileList(true); + return tsFileManager.getTsFileList(true, databaseName); } public List getUnSequenceFileList() { - return tsFileManager.getTsFileList(false); + return tsFileManager.getTsFileList(false, databaseName); } @Override @@ -4783,14 +4787,14 @@ public void addSettleFilesToList( List unseqResourcesToBeSettled, List tsFilePaths) { if (tsFilePaths.isEmpty()) { - for (TsFileResource resource : tsFileManager.getTsFileList(true)) { + for (TsFileResource resource : tsFileManager.getTsFileList(true, databaseName)) { if (!resource.isClosed()) { continue; } resource.setSettleTsFileCallBack(this::settleTsFileCallBack); seqResourcesToBeSettled.add(resource); } - for (TsFileResource resource : tsFileManager.getTsFileList(false)) { + for (TsFileResource resource : tsFileManager.getTsFileList(false, databaseName)) { if (!resource.isClosed()) { continue; } @@ -4808,7 +4812,7 @@ public void addSettleFilesToList( .getParentFile() .getParentFile() .getName())) { - for (TsFileResource resource : tsFileManager.getTsFileList(true)) { + for (TsFileResource resource : tsFileManager.getTsFileList(true, databaseName)) { if (resource.getTsFile().getAbsolutePath().equals(tsFilePath)) { resource.setSettleTsFileCallBack(this::settleTsFileCallBack); seqResourcesToBeSettled.add(resource); @@ -4816,7 +4820,7 @@ public void addSettleFilesToList( } } } else { - for (TsFileResource resource : tsFileManager.getTsFileList(false)) { + for (TsFileResource resource : tsFileManager.getTsFileList(false, databaseName)) { if (resource.getTsFile().getAbsolutePath().equals(tsFilePath)) { unseqResourcesToBeSettled.add(resource); break; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/snapshot/SnapshotTaker.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/snapshot/SnapshotTaker.java index f4313827c9ab3..21346b364ed9e 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/snapshot/SnapshotTaker.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/snapshot/SnapshotTaker.java @@ -194,11 +194,12 @@ public static boolean clearSnapshotOfDataRegion(DataRegion dataRegion) { } private void readLockTheFile() { + String database = dataRegion.getDatabaseName(); TsFileManager manager = dataRegion.getTsFileManager(); manager.readLock(); try { - seqFiles = manager.getTsFileList(true); - unseqFiles = manager.getTsFileList(false); + seqFiles = manager.getTsFileList(true, database); + unseqFiles = manager.getTsFileList(false, database); for (TsFileResource resource : seqFiles) { resource.readLock(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java index 86890370c313b..851f934249fee 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileManager.java @@ -65,8 +65,13 @@ public TsFileManager(String storageGroupName, String dataRegionId, String dataRe this.dataRegionId = dataRegionId; } + // @todo public List getTsFileList(boolean sequence) { - return getTsFileList(sequence, null, null); + return getTsFileList(sequence, null, null, ""); + } + + public List getTsFileList(boolean sequence, String database) { + return getTsFileList(sequence, null, null, database); } /** @@ -74,7 +79,7 @@ public List getTsFileList(boolean sequence) { * @param timePartitions {@code null} for all time partitions, empty for zero time partitions */ public List getTsFileList( - boolean sequence, List timePartitions, Filter timeFilter) { + boolean sequence, List timePartitions, Filter timeFilter, String database) { // the iteration of ConcurrentSkipListMap is not concurrent secure // so we must add read lock here readLock(); @@ -83,7 +88,7 @@ public List getTsFileList( Map chosenMap = sequence ? sequenceFiles : unsequenceFiles; if (timePartitions == null) { for (Map.Entry entry : chosenMap.entrySet()) { - if (TimePartitionUtils.satisfyTimePartition(timeFilter, entry.getKey())) { + if (TimePartitionUtils.satisfyTimePartition(timeFilter, entry.getKey(), database)) { allResources.addAll(entry.getValue().getArrayList()); } } @@ -107,17 +112,17 @@ public List getTsFileList( * @return left is seq resource list, right is unSeq resource list */ public Pair, List> getAllTsFileListForQuery( - List timePartitions, Filter timeFilter) { + List timePartitions, Filter timeFilter, String database) { List seq = new ArrayList<>(); List unSeq = new ArrayList<>(); if (timePartitions == null) { for (Map.Entry entry : sequenceFiles.entrySet()) { - if (TimePartitionUtils.satisfyTimePartition(timeFilter, entry.getKey())) { + if (TimePartitionUtils.satisfyTimePartition(timeFilter, entry.getKey(), database)) { seq.addAll(entry.getValue().getArrayList()); } } for (Map.Entry entry : unsequenceFiles.entrySet()) { - if (TimePartitionUtils.satisfyTimePartition(timeFilter, entry.getKey())) { + if (TimePartitionUtils.satisfyTimePartition(timeFilter, entry.getKey(), database)) { unSeq.addAll(entry.getValue().getArrayList()); } } @@ -158,7 +163,8 @@ public Pair, List> getTsFileListSnapshot( } } - public List getTsFileList(boolean sequence, long startTime, long endTime) { + public List getTsFileList( + boolean sequence, long startTime, long endTime, String database) { // the iteration of ConcurrentSkipListMap is not concurrent secure // so we must add read lock here readLock(); @@ -166,7 +172,7 @@ public List getTsFileList(boolean sequence, long startTime, long List allResources = new ArrayList<>(); Map chosenMap = sequence ? sequenceFiles : unsequenceFiles; for (Map.Entry entry : chosenMap.entrySet()) { - if (TimePartitionUtils.satisfyPartitionId(startTime, endTime, entry.getKey())) { + if (TimePartitionUtils.satisfyPartitionId(startTime, endTime, entry.getKey(), database)) { allResources.addAll(entry.getValue().getArrayList()); } } @@ -194,10 +200,10 @@ public TsFileResourceList getOrCreateUnsequenceListByTimePartition(long timePart } } - public Iterator getIterator(boolean sequence) { + public Iterator getIterator(boolean sequence, String database) { readLock(); try { - return getTsFileList(sequence).iterator(); + return getTsFileList(sequence, database).iterator(); } finally { readUnlock(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/AlignedChunkData.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/AlignedChunkData.java index 72268168258ee..188f47fcaff38 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/AlignedChunkData.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/AlignedChunkData.java @@ -207,13 +207,14 @@ public void writeEntirePage(final PageHeader pageHeader, final ByteBuffer pageDa } @Override - public void writeDecodePage(final long[] times, final Object[] values, final int satisfiedLength) + public void writeDecodePage( + final long[] times, final Object[] values, final int satisfiedLength, String database) throws IOException { pageNumbers.set(pageNumbers.size() - 1, pageNumbers.get(pageNumbers.size() - 1) + 1); satisfiedLengthQueue.offer(satisfiedLength); final long startTime = timePartitionSlot.getStartTime(); // beware of overflow - long endTime = startTime + TimePartitionUtils.getTimePartitionInterval() - 1; + long endTime = startTime + TimePartitionUtils.getTimePartitionInterval(database) - 1; if (endTime <= startTime) { endTime = Long.MAX_VALUE; } @@ -232,12 +233,15 @@ public void writeDecodePage(final long[] times, final Object[] values, final int } public void writeDecodeValuePage( - final long[] times, final TsPrimitiveType[] values, final TSDataType dataType) + final long[] times, + final TsPrimitiveType[] values, + final TSDataType dataType, + String database) throws IOException { pageNumbers.set(pageNumbers.size() - 1, pageNumbers.get(pageNumbers.size() - 1) + 1); final long startTime = timePartitionSlot.getStartTime(); // beware of overflow - long endTime = startTime + TimePartitionUtils.getTimePartitionInterval() - 1; + long endTime = startTime + TimePartitionUtils.getTimePartitionInterval(database) - 1; if (endTime <= startTime) { endTime = Long.MAX_VALUE; } @@ -448,7 +452,7 @@ private void buildChunk( public static AlignedChunkData deserialize(final InputStream stream) throws IOException, PageException { final TTimePartitionSlot timePartitionSlot = - TimePartitionUtils.getTimePartitionSlot(ReadWriteIOUtils.readLong(stream)); + new TTimePartitionSlot(ReadWriteIOUtils.readLong(stream)); final boolean isStringArrayDeviceID = ReadWriteIOUtils.readBool(stream); final IDeviceID device = isStringArrayDeviceID diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/BatchedAlignedValueChunkData.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/BatchedAlignedValueChunkData.java index 6e8d3850b8e24..70bc0fdd4769a 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/BatchedAlignedValueChunkData.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/BatchedAlignedValueChunkData.java @@ -60,12 +60,13 @@ public BatchedAlignedValueChunkData(IDeviceID device, TTimePartitionSlot timePar } @Override - public void writeDecodeValuePage(long[] times, TsPrimitiveType[] values, TSDataType dataType) + public void writeDecodeValuePage( + long[] times, TsPrimitiveType[] values, TSDataType dataType, String database) throws IOException { pageNumbers.set(pageNumbers.size() - 1, pageNumbers.get(pageNumbers.size() - 1) + 1); final long startTime = timePartitionSlot.getStartTime(); // beware of overflow - long endTime = startTime + TimePartitionUtils.getTimePartitionInterval() - 1; + long endTime = startTime + TimePartitionUtils.getTimePartitionInterval(database) - 1; if (endTime <= startTime) { endTime = Long.MAX_VALUE; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/ChunkData.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/ChunkData.java index 7cc5db029950e..a7c9bbf6693b8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/ChunkData.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/ChunkData.java @@ -46,7 +46,8 @@ public interface ChunkData extends TsFileData { void writeEntirePage(PageHeader pageHeader, ByteBuffer pageData) throws IOException; - void writeDecodePage(long[] times, Object[] values, int satisfiedLength) throws IOException; + void writeDecodePage(long[] times, Object[] values, int satisfiedLength, String database) + throws IOException; void writeToFileWriter(TsFileIOWriter writer) throws IOException, PageException; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/NonAlignedChunkData.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/NonAlignedChunkData.java index 2310b9cb95c3e..36a5193e9f855 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/NonAlignedChunkData.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/NonAlignedChunkData.java @@ -161,12 +161,13 @@ public void writeEntirePage(final PageHeader pageHeader, final ByteBuffer pageDa } @Override - public void writeDecodePage(final long[] times, final Object[] values, final int satisfiedLength) + public void writeDecodePage( + final long[] times, final Object[] values, final int satisfiedLength, String database) throws IOException { pageNumber += 1; final long startTime = timePartitionSlot.getStartTime(); // beware of overflow - long endTime = startTime + TimePartitionUtils.getTimePartitionInterval() - 1; + long endTime = startTime + TimePartitionUtils.getTimePartitionInterval(database) - 1; if (endTime <= startTime) { endTime = Long.MAX_VALUE; } @@ -284,7 +285,7 @@ private void buildChunkWriter(final InputStream stream) throws IOException, Page public static NonAlignedChunkData deserialize(final InputStream stream) throws IOException, PageException { final TTimePartitionSlot timePartitionSlot = - TimePartitionUtils.getTimePartitionSlot(ReadWriteIOUtils.readLong(stream)); + new TTimePartitionSlot(ReadWriteIOUtils.readLong(stream)); final boolean isStringArrayDeviceID = ReadWriteIOUtils.readBool(stream); final IDeviceID device = isStringArrayDeviceID diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/TsFileSplitter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/TsFileSplitter.java index 5a75f4fb8e085..ba8156cee5968 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/TsFileSplitter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/load/splitter/TsFileSplitter.java @@ -93,7 +93,7 @@ public TsFileSplitter(File tsFile, TsFileDataConsumer consumer) { } @SuppressWarnings({"squid:S3776", "squid:S6541"}) - public void splitTsFileByDataPartition() + public void splitTsFileByDataPartition(String database) throws IOException, LoadFileException, IllegalStateException { try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFile.getAbsolutePath())) { getAllModification(deletions); @@ -117,14 +117,14 @@ public void splitTsFileByDataPartition() case MetaMarker.TIME_CHUNK_HEADER: case MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER: case MetaMarker.ONLY_ONE_PAGE_TIME_CHUNK_HEADER: - processTimeChunkOrNonAlignedChunk(reader, marker); + processTimeChunkOrNonAlignedChunk(reader, marker, database); if (isAligned) { storeTimeChunkContext(); } break; case MetaMarker.VALUE_CHUNK_HEADER: case MetaMarker.ONLY_ONE_PAGE_VALUE_CHUNK_HEADER: - processValueChunk(reader, marker); + processValueChunk(reader, marker, database); break; case MetaMarker.CHUNK_GROUP_HEADER: ChunkGroupHeader chunkGroupHeader = reader.readChunkGroupHeader(); @@ -148,7 +148,8 @@ public void splitTsFileByDataPartition() } } - private void processTimeChunkOrNonAlignedChunk(TsFileSequenceReader reader, byte marker) + private void processTimeChunkOrNonAlignedChunk( + TsFileSequenceReader reader, byte marker, String database) throws IOException, LoadFileException { long chunkOffset = reader.position(); timeChunkIndexOfCurrentValueColumn = pageIndex2TimesList.size(); @@ -175,11 +176,11 @@ private void processTimeChunkOrNonAlignedChunk(TsFileSequenceReader reader, byte return; } TTimePartitionSlot timePartitionSlot = - TimePartitionUtils.getTimePartitionSlot(chunkMetadata.getStartTime()); + TimePartitionUtils.getTimePartitionSlot(chunkMetadata.getStartTime(), database); ChunkData chunkData = ChunkData.createChunkData(isAligned, curDevice, header, timePartitionSlot); - if (!needDecodeChunk(chunkMetadata)) { + if (!needDecodeChunk(chunkMetadata, database)) { chunkData.setNotDecode(); chunkData.writeEntireChunk(reader.readChunk(-1, header.getDataSize()), chunkMetadata); if (isAligned) { @@ -193,7 +194,8 @@ private void processTimeChunkOrNonAlignedChunk(TsFileSequenceReader reader, byte return; } - decodeAndWriteTimeChunkOrNonAlignedChunk(reader, header, chunkMetadata, chunkOffset, chunkData); + decodeAndWriteTimeChunkOrNonAlignedChunk( + reader, header, chunkMetadata, chunkOffset, chunkData, database); } private void decodeAndWriteTimeChunkOrNonAlignedChunk( @@ -201,7 +203,8 @@ private void decodeAndWriteTimeChunkOrNonAlignedChunk( ChunkHeader header, IChunkMetadata chunkMetadata, long chunkOffset, - ChunkData chunkData) + ChunkData chunkData, + String database) throws IOException, LoadFileException { String measurementId = header.getMeasurementID(); TTimePartitionSlot timePartitionSlot = chunkData.getTimePartitionSlot(); @@ -222,13 +225,14 @@ private void decodeAndWriteTimeChunkOrNonAlignedChunk( reader.readPageHeader( header.getDataType(), (header.getChunkType() & 0x3F) == MetaMarker.CHUNK_HEADER); long pageDataSize = pageHeader.getSerializedPageSize(); - if (!needDecodePage(pageHeader, chunkMetadata)) { // an entire page + // an entire page + if (!needDecodePage(pageHeader, chunkMetadata, database)) { long startTime = pageHeader.getStatistics() == null ? chunkMetadata.getStartTime() : pageHeader.getStartTime(); TTimePartitionSlot pageTimePartitionSlot = - TimePartitionUtils.getTimePartitionSlot(startTime); + TimePartitionUtils.getTimePartitionSlot(startTime, database); if (!timePartitionSlot.equals(pageTimePartitionSlot)) { if (!isAligned) { consumeChunkData(measurementId, chunkOffset, chunkData); @@ -254,14 +258,15 @@ private void decodeAndWriteTimeChunkOrNonAlignedChunk( int satisfiedLength = 0; long endTime = - timePartitionSlot.getStartTime() + TimePartitionUtils.getTimePartitionInterval(); + timePartitionSlot.getStartTime() + + TimePartitionUtils.getTimePartitionInterval(database); // beware of overflow if (endTime <= timePartitionSlot.getStartTime()) { endTime = Long.MAX_VALUE; } for (int i = 0; i < times.length; i++) { if (times[i] >= endTime) { - chunkData.writeDecodePage(times, values, satisfiedLength); + chunkData.writeDecodePage(times, values, satisfiedLength, database); if (isAligned) { pageIndex2ChunkData .computeIfAbsent(pageIndex, o -> new ArrayList<>()) @@ -270,18 +275,20 @@ private void decodeAndWriteTimeChunkOrNonAlignedChunk( consumeChunkData(measurementId, chunkOffset, chunkData); } - timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(times[i]); + timePartitionSlot = TimePartitionUtils.getTimePartitionSlot(times[i], database); satisfiedLength = 0; - endTime = - timePartitionSlot.getStartTime() + TimePartitionUtils.getTimePartitionInterval(); - if (endTime <= timePartitionSlot.getStartTime()) { - endTime = Long.MAX_VALUE; + long endTime2 = + timePartitionSlot.getStartTime() + + TimePartitionUtils.getTimePartitionInterval(database); + if (endTime2 <= timePartitionSlot.getStartTime()) { + endTime2 = Long.MAX_VALUE; } + endTime = endTime2; chunkData = ChunkData.createChunkData(isAligned, curDevice, header, timePartitionSlot); } satisfiedLength += 1; } - chunkData.writeDecodePage(times, values, satisfiedLength); + chunkData.writeDecodePage(times, values, satisfiedLength, database); if (isAligned) { pageIndex2ChunkData .computeIfAbsent(pageIndex, o -> new ArrayList<>()) @@ -298,7 +305,7 @@ private void decodeAndWriteTimeChunkOrNonAlignedChunk( } } - private void processValueChunk(TsFileSequenceReader reader, byte marker) + private void processValueChunk(TsFileSequenceReader reader, byte marker, String database) throws IOException, LoadFileException { long chunkOffset = reader.position(); IChunkMetadata chunkMetadata = offset2ChunkMetadata.get(chunkOffset - Byte.BYTES); @@ -347,7 +354,8 @@ private void processValueChunk(TsFileSequenceReader reader, byte marker) long[] times = pageIndex2Times.get(pageIndex); TsPrimitiveType[] values = decodeValuePage(reader, header, pageHeader, times, valueDecoder); for (AlignedChunkData alignedChunkData : alignedChunkDataList) { - alignedChunkData.writeDecodeValuePage(times, values, header.getDataType()); + alignedChunkData.setNotDecode(); + alignedChunkData.writeDecodeValuePage(times, values, header.getDataType(), database); } } long pageDataSize = pageHeader.getSerializedPageSize(); @@ -486,18 +494,19 @@ private void consumeChunkData(String measurement, long offset, ChunkData chunkDa } } - private boolean needDecodeChunk(IChunkMetadata chunkMetadata) { - return !TimePartitionUtils.getTimePartitionSlot(chunkMetadata.getStartTime()) - .equals(TimePartitionUtils.getTimePartitionSlot(chunkMetadata.getEndTime())); + private boolean needDecodeChunk(IChunkMetadata chunkMetadata, String database) { + return !TimePartitionUtils.getTimePartitionSlot(chunkMetadata.getStartTime(), database) + .equals(TimePartitionUtils.getTimePartitionSlot(chunkMetadata.getEndTime(), database)); } - private boolean needDecodePage(PageHeader pageHeader, IChunkMetadata chunkMetadata) { + private boolean needDecodePage( + PageHeader pageHeader, IChunkMetadata chunkMetadata, String database) { if (pageHeader.getStatistics() == null) { - return !TimePartitionUtils.getTimePartitionSlot(chunkMetadata.getStartTime()) - .equals(TimePartitionUtils.getTimePartitionSlot(chunkMetadata.getEndTime())); + return !TimePartitionUtils.getTimePartitionSlot(chunkMetadata.getStartTime(), database) + .equals(TimePartitionUtils.getTimePartitionSlot(chunkMetadata.getEndTime(), database)); } - return !TimePartitionUtils.getTimePartitionSlot(pageHeader.getStartTime()) - .equals(TimePartitionUtils.getTimePartitionSlot(pageHeader.getEndTime())); + return !TimePartitionUtils.getTimePartitionSlot(pageHeader.getStartTime(), database) + .equals(TimePartitionUtils.getTimePartitionSlot(pageHeader.getEndTime(), database)); } private Pair decodePage( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/QueryTimePartitionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/QueryTimePartitionTest.java index 1e85a1461ee87..3810ef79c3a81 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/QueryTimePartitionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/QueryTimePartitionTest.java @@ -360,13 +360,15 @@ public void testGetTimePartitionSlotList() { // time >= 10 and time <= 9 Pair, Pair> res = getTimePartitionSlotList( - FilterFactory.and(TimeFilterApi.gtEq(10), TimeFilterApi.ltEq(9)), context); + FilterFactory.and(TimeFilterApi.gtEq(10), TimeFilterApi.ltEq(9)), + context, + context.getDatabase()); assertTrue(res.left.isEmpty()); assertFalse(res.right.left); assertFalse(res.right.right); // time >= 10 - res = getTimePartitionSlotList(TimeFilterApi.gtEq(10), context); + res = getTimePartitionSlotList(TimeFilterApi.gtEq(10), context, context.getDatabase()); assertEquals(1, res.left.size()); List expected = Collections.singletonList(new TTimePartitionSlot(0)); assertEquals(expected.size(), res.left.size()); @@ -377,7 +379,7 @@ public void testGetTimePartitionSlotList() { assertTrue(res.right.right); // time < 20 - res = getTimePartitionSlotList(TimeFilterApi.lt(20), context); + res = getTimePartitionSlotList(TimeFilterApi.lt(20), context, context.getDatabase()); assertEquals(1, res.left.size()); expected = Collections.singletonList(new TTimePartitionSlot(0)); assertEquals(expected.size(), res.left.size()); @@ -390,7 +392,9 @@ public void testGetTimePartitionSlotList() { // time > 10 and time <= 20 res = getTimePartitionSlotList( - FilterFactory.and(TimeFilterApi.gt(10), TimeFilterApi.ltEq(20)), context); + FilterFactory.and(TimeFilterApi.gt(10), TimeFilterApi.ltEq(20)), + context, + context.getDatabase()); expected = Collections.singletonList(new TTimePartitionSlot(0)); assertEquals(expected.size(), res.left.size()); for (int i = 0; i < expected.size(); i++) { @@ -407,7 +411,8 @@ public void testGetTimePartitionSlotList() { TimeFilterApi.gt(0), TimeFilterApi.ltEq( CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() * 3 + 1)), - context); + context, + context.getDatabase()); expected = Arrays.asList( new TTimePartitionSlot(0), @@ -433,7 +438,8 @@ public void testGetTimePartitionSlotList() { CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() - 1), TimeFilterApi.lt( CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() + 1)), - context); + context, + context.getDatabase()); expected = Arrays.asList( new TTimePartitionSlot(0), @@ -453,7 +459,8 @@ public void testGetTimePartitionSlotList() { TimeFilterApi.between( CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() - 1, CommonDescriptor.getInstance().getConfig().getTimePartitionInterval()), - context); + context, + context.getDatabase()); expected = Arrays.asList( new TTimePartitionSlot(0), @@ -475,7 +482,8 @@ public void testGetTimePartitionSlotList() { CommonDescriptor.getInstance().getConfig().getTimePartitionInterval()), TimeFilterApi.ltEq( CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() + 1)), - context); + context, + context.getDatabase()); expected = Collections.singletonList( new TTimePartitionSlot( @@ -494,7 +502,8 @@ public void testGetTimePartitionSlotList() { TimeFilterApi.between( CommonDescriptor.getInstance().getConfig().getTimePartitionInterval(), CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() + 1), - context); + context, + context.getDatabase()); expected = Collections.singletonList( new TTimePartitionSlot( @@ -556,7 +565,7 @@ public void testGetTimePartitionSlotList() { CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() * 5 + 10))); - res = getTimePartitionSlotList(orFilter4, context); + res = getTimePartitionSlotList(orFilter4, context, context.getDatabase()); expected = Arrays.asList( new TTimePartitionSlot(0), diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/WritePlanNodeSplitTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/WritePlanNodeSplitTest.java index 8d23bc848b9eb..f8544816b1d3c 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/WritePlanNodeSplitTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/planner/node/write/WritePlanNodeSplitTest.java @@ -34,6 +34,7 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.schema.table.column.TsTableColumnCategory; import org.apache.iotdb.commons.utils.TimePartitionUtils; +import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.queryengine.plan.analyze.Analysis; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeId; @@ -79,7 +80,10 @@ public void setUp() { prevTimePartitionInterval = CommonDescriptor.getInstance().getConfig().getTimePartitionInterval(); CommonDescriptor.getInstance().getConfig().setTimePartitionInterval(100); - TimePartitionUtils.setTimePartitionInterval(100); + TDatabaseSchema databaseSchema = new TDatabaseSchema(); + databaseSchema.setTimePartitionInterval(100); + TimePartitionUtils.updateDatabaseTimePartitionConfig("root.sg1", databaseSchema); + TimePartitionUtils.updateDatabaseTimePartitionConfig("root.sg2", databaseSchema); executorClassName = IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionExecutorClass(); seriesSlotPartitionNum = IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionSlotNum(); @@ -109,13 +113,14 @@ private void initDataPartitionMap() { for (int i = 0; i < seriesSlotPartitionNum; i++) { Map> timePartitionSlotMap = new HashMap<>(); for (int t = -2; t < 5; t++) { + @SuppressWarnings("deprecation") long startTime = t * TimePartitionUtils.getTimePartitionInterval() + 1; timePartitionSlotMap.put( - TimePartitionUtils.getTimePartitionSlot(startTime), + TimePartitionUtils.getTimePartitionSlot(startTime, "root.sg1"), Collections.singletonList( new TRegionReplicaSet( new TConsensusGroupId( - TConsensusGroupType.DataRegion, getRegionIdByTime(startTime)), + TConsensusGroupType.DataRegion, getRegionIdByTime(startTime, "root.sg1")), locationList))); } @@ -130,7 +135,7 @@ TConsensusGroupType.DataRegion, getRegionIdByTime(startTime)), Map> timePartitionSlotMap = new HashMap<>(); for (int t = 0; t < 5; t++) { timePartitionSlotMap.put( - new TTimePartitionSlot(t * TimePartitionUtils.getTimePartitionInterval()), + new TTimePartitionSlot(t * TimePartitionUtils.getTimePartitionInterval("root.sg2")), Collections.singletonList( new TRegionReplicaSet( new TConsensusGroupId(TConsensusGroupType.DataRegion, 99), locationList))); @@ -154,8 +159,9 @@ private void initSchemaPartitionMap() { schemaPartitionMap.put("root.sg1", seriesPartitionSlotMap); } - private int getRegionIdByTime(long startTime) { - return (int) (4 - ((startTime - 1) / TimePartitionUtils.getTimePartitionInterval())); + private int getRegionIdByTime(long startTime, String database) { + long interval = TimePartitionUtils.getTimePartitionInterval(database); + return (int) (4 - ((startTime - 1) / interval)); } protected DataPartition getDataPartition( @@ -209,7 +215,8 @@ public void testSplitInsertTablet() throws IllegalPathException { DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); dataPartitionQueryParam.setDeviceID( insertTabletNode.getTargetPath().getIDeviceIDAsFullDevice()); - dataPartitionQueryParam.setTimePartitionSlotList(insertTabletNode.getTimePartitionSlots()); + dataPartitionQueryParam.setTimePartitionSlotList( + insertTabletNode.getTimePartitionSlots("root.sg1")); DataPartition dataPartition = getDataPartition(Collections.singletonList(dataPartitionQueryParam)); @@ -223,7 +230,7 @@ public void testSplitInsertTablet() throws IllegalPathException { InsertTabletNode tabletNode = (InsertTabletNode) insertNode; Assert.assertEquals(tabletNode.getTimes().length, 2); TConsensusGroupId regionId = tabletNode.getDataRegionReplicaSet().getRegionId(); - Assert.assertEquals(getRegionIdByTime(tabletNode.getMinTime()), regionId.getId()); + Assert.assertEquals(getRegionIdByTime(tabletNode.getMinTime(), "root.sg1"), regionId.getId()); } insertTabletNode = new InsertTabletNode(new PlanNodeId("plan node 2")); @@ -236,7 +243,8 @@ public void testSplitInsertTablet() throws IllegalPathException { dataPartitionQueryParam = new DataPartitionQueryParam(); dataPartitionQueryParam.setDeviceID( insertTabletNode.getTargetPath().getIDeviceIDAsFullDevice()); - dataPartitionQueryParam.setTimePartitionSlotList(insertTabletNode.getTimePartitionSlots()); + dataPartitionQueryParam.setTimePartitionSlotList( + insertTabletNode.getTimePartitionSlots("root.sg2")); dataPartition = getDataPartition(Collections.singletonList(dataPartitionQueryParam)); analysis = new Analysis(); @@ -276,13 +284,13 @@ public void testSplitRelationalInsertTablet() throws IllegalPathException { DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); dataPartitionQueryParam.setDeviceID(relationalInsertTabletNode.getDeviceID(0)); dataPartitionQueryParam.setTimePartitionSlotList( - relationalInsertTabletNode.getTimePartitionSlots()); + relationalInsertTabletNode.getTimePartitionSlots("root.sg1")); dataPartitionQueryParamList.add(dataPartitionQueryParam); dataPartitionQueryParam = new DataPartitionQueryParam(); dataPartitionQueryParam.setDeviceID(relationalInsertTabletNode.getDeviceID(1)); dataPartitionQueryParam.setTimePartitionSlotList( - relationalInsertTabletNode.getTimePartitionSlots()); + relationalInsertTabletNode.getTimePartitionSlots("root.sg1")); dataPartitionQueryParamList.add(dataPartitionQueryParam); DataPartition dataPartition = getDataPartition(dataPartitionQueryParamList); @@ -299,7 +307,7 @@ public void testSplitRelationalInsertTablet() throws IllegalPathException { // keep the time order after split Assert.assertTrue(tabletNode.getTimes()[0] < tabletNode.getTimes()[1]); TConsensusGroupId regionId = tabletNode.getDataRegionReplicaSet().getRegionId(); - Assert.assertEquals(getRegionIdByTime(tabletNode.getMinTime()), regionId.getId()); + Assert.assertEquals(getRegionIdByTime(tabletNode.getMinTime(), "root.sg1"), regionId.getId()); } } @@ -333,7 +341,10 @@ public void testInsertMultiTablets() throws IllegalPathException { DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); dataPartitionQueryParam.setDeviceID( insertTabletNode.getTargetPath().getIDeviceIDAsFullDevice()); - dataPartitionQueryParam.setTimePartitionSlotList(insertTabletNode.getTimePartitionSlots()); + String fullPath = insertTabletNode.getTargetPath().getFullPath(); + dataPartitionQueryParam.setTimePartitionSlotList( + insertTabletNode.getTimePartitionSlots( + fullPath.substring(0, fullPath.indexOf(".", fullPath.indexOf(".") + 1)))); dataPartitionQueryParams.add(dataPartitionQueryParam); } @@ -354,7 +365,9 @@ public void testInsertRowsNode() throws IllegalPathException { for (int i = 0; i < 7; i++) { InsertRowNode insertRowNode = new InsertRowNode(new PlanNodeId("plan node 3")); insertRowNode.setTargetPath(new PartialPath(String.format("root.sg1.d%d", i))); - insertRowNode.setTime((i - 2) * TimePartitionUtils.getTimePartitionInterval()); + @SuppressWarnings("deprecation") + long interval = TimePartitionUtils.getTimePartitionInterval(); + insertRowNode.setTime((i - 2) * interval); insertRowsNode.addOneInsertRowNode(insertRowNode, 2 * i); insertRowNode = new InsertRowNode(new PlanNodeId("plan node 3")); @@ -367,7 +380,8 @@ public void testInsertRowsNode() throws IllegalPathException { for (InsertRowNode insertRowNode : insertRowsNode.getInsertRowNodeList()) { DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); dataPartitionQueryParam.setDeviceID(insertRowNode.getTargetPath().getIDeviceIDAsFullDevice()); - dataPartitionQueryParam.setTimePartitionSlotList(insertRowNode.getTimePartitionSlots()); + dataPartitionQueryParam.setTimePartitionSlotList( + insertRowNode.getTimePartitionSlots("root.sg2")); dataPartitionQueryParams.add(dataPartitionQueryParam); } @@ -406,7 +420,8 @@ public void testInsertRowsOfOneDeviceNode() throws IllegalPathException { for (InsertRowNode insertRowNode : insertRowsOfOneDeviceNode.getInsertRowNodeList()) { DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); dataPartitionQueryParam.setDeviceID(insertRowNode.getTargetPath().getIDeviceIDAsFullDevice()); - dataPartitionQueryParam.setTimePartitionSlotList(insertRowNode.getTimePartitionSlots()); + dataPartitionQueryParam.setTimePartitionSlotList( + insertRowNode.getTimePartitionSlots("root.sg1")); dataPartitionQueryParams.add(dataPartitionQueryParam); } @@ -443,7 +458,8 @@ public void testInsertRowsOfOneDeviceNode() throws IllegalPathException { for (InsertRowNode insertRowNode : insertRowsOfOneDeviceNode.getInsertRowNodeList()) { DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); dataPartitionQueryParam.setDeviceID(insertRowNode.getTargetPath().getIDeviceIDAsFullDevice()); - dataPartitionQueryParam.setTimePartitionSlotList(insertRowNode.getTimePartitionSlots()); + dataPartitionQueryParam.setTimePartitionSlotList( + insertRowNode.getTimePartitionSlots("root.sg2")); dataPartitionQueryParams.add(dataPartitionQueryParam); } @@ -462,7 +478,10 @@ public void testInsertRowsOfOneDeviceNode() throws IllegalPathException { @After public void tearDown() { - TimePartitionUtils.setTimePartitionInterval(prevTimePartitionInterval); + TDatabaseSchema databaseSchema = new TDatabaseSchema(); + databaseSchema.setTimePartitionInterval(prevTimePartitionInterval); + TimePartitionUtils.updateDatabaseTimePartitionConfig("root.sg1", databaseSchema); + TimePartitionUtils.updateDatabaseTimePartitionConfig("root.sg2", databaseSchema); CommonDescriptor.getInstance().getConfig().setTimePartitionInterval(prevTimePartitionInterval); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/StorageEngineTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/StorageEngineTest.java index 4c4ac20e9aeba..855b7407d0d00 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/StorageEngineTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/StorageEngineTest.java @@ -74,14 +74,17 @@ public void testGetAllDataRegionIds() throws Exception { @Test public void testGetTimePartitionId() { + @SuppressWarnings("deprecation") long timePartitionInterval = TimePartitionUtils.getTimePartitionInterval(); - Assert.assertEquals(-2, TimePartitionUtils.getTimePartitionId(-timePartitionInterval - 1)); - Assert.assertEquals(-1, TimePartitionUtils.getTimePartitionId(-timePartitionInterval)); + @SuppressWarnings("deprecation") + long interval = TimePartitionUtils.getTimePartitionInterval(); + Assert.assertEquals(-2, TimePartitionUtils.getTimePartitionId(-interval - 1)); + Assert.assertEquals(-1, TimePartitionUtils.getTimePartitionId(-interval)); Assert.assertEquals(-1, TimePartitionUtils.getTimePartitionId(-1)); Assert.assertEquals(0, TimePartitionUtils.getTimePartitionId(0)); Assert.assertEquals(0, TimePartitionUtils.getTimePartitionId(1)); - Assert.assertEquals(0, TimePartitionUtils.getTimePartitionId(timePartitionInterval / 2)); - Assert.assertEquals(1, TimePartitionUtils.getTimePartitionId(timePartitionInterval * 2 - 1)); + Assert.assertEquals(0, TimePartitionUtils.getTimePartitionId(interval / 2)); + Assert.assertEquals(1, TimePartitionUtils.getTimePartitionId(interval * 2 - 1)); Assert.assertEquals(2, TimePartitionUtils.getTimePartitionId(timePartitionInterval * 2 + 1)); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/BatchedCompactionWithTsFileSplitterTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/BatchedCompactionWithTsFileSplitterTest.java index 272d9e6ae5ca7..f07588eee8e21 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/BatchedCompactionWithTsFileSplitterTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/BatchedCompactionWithTsFileSplitterTest.java @@ -302,7 +302,7 @@ private void consumeChunkDataAndValidate(TsFileResource resource) } return true; }); - splitter.splitTsFileByDataPartition(); + splitter.splitTsFileByDataPartition(resource.getDatabaseName()); List splitResources = new ArrayList<>(); for (Map.Entry entry : writerMap.entrySet()) { TestLoadTsFileIOWriter writer = entry.getValue(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionTable.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionTable.java index d154f1813e1b7..c70a22615e3f3 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionTable.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionTable.java @@ -273,12 +273,12 @@ public Map getLastDataAllotTable() { * @param currentTimeSlot The current TimeSlot */ public Set autoCleanPartitionTable( - long TTL, TTimePartitionSlot currentTimeSlot) { + long TTL, TTimePartitionSlot currentTimeSlot, String database) { Set removedTimePartitionSlots = new HashSet<>(); dataPartitionMap.forEach( (seriesPartitionSlot, seriesPartitionTable) -> removedTimePartitionSlots.addAll( - seriesPartitionTable.autoCleanPartitionTable(TTL, currentTimeSlot))); + seriesPartitionTable.autoCleanPartitionTable(TTL, currentTimeSlot, database))); return removedTimePartitionSlots; } diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/SeriesPartitionTable.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/SeriesPartitionTable.java index da8952051e514..9c6d32549d61a 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/SeriesPartitionTable.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/SeriesPartitionTable.java @@ -52,13 +52,6 @@ public class SeriesPartitionTable { - // should only be used in CN scope, in DN scope should directly use - // TimePartitionUtils.getTimePartitionInterval() - private static final long TIME_PARTITION_INTERVAL = - CommonDateTimeUtils.convertMilliTimeWithPrecision( - TimePartitionUtils.getTimePartitionInterval(), - CommonDescriptor.getInstance().getConfig().getTimestampPrecision()); - private final ConcurrentSkipListMap> seriesPartitionMap; @@ -262,14 +255,19 @@ public TConsensusGroupId getLastConsensusGroupId() { * @param currentTimeSlot The current TimeSlot */ public List autoCleanPartitionTable( - long TTL, TTimePartitionSlot currentTimeSlot) { + long TTL, TTimePartitionSlot currentTimeSlot, String database) { + // should only be used in CN scope, in DN scope should directly use + final long timePartitionInterval = + CommonDateTimeUtils.convertMilliTimeWithPrecision( + TimePartitionUtils.getTimePartitionInterval(database), + CommonDescriptor.getInstance().getConfig().getTimestampPrecision()); List removedTimePartitions = new ArrayList<>(); Iterator>> iterator = seriesPartitionMap.entrySet().iterator(); while (iterator.hasNext()) { Map.Entry> entry = iterator.next(); TTimePartitionSlot timePartitionSlot = entry.getKey(); - if (timePartitionSlot.getStartTime() + TIME_PARTITION_INTERVAL + TTL + if (timePartitionSlot.getStartTime() + timePartitionInterval + TTL <= currentTimeSlot.getStartTime()) { removedTimePartitions.add(timePartitionSlot); iterator.remove(); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtils.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtils.java index ee5e0677f0ba3..059c438bea8b0 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtils.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/ThriftCommonsSerDeUtils.java @@ -44,6 +44,7 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; +import java.util.Map; import static org.apache.iotdb.rpc.TConfigurationConst.defaultTConfiguration; @@ -226,7 +227,7 @@ public static TTimeSlotList deserializeTTimePartitionSlotList(ByteBuffer buffer) } public static void serializeTTimePartitionSlot( - TTimePartitionSlot timePartitionSlot, DataOutputStream stream) { + Map timePartitionSlot, DataOutputStream stream) { try { timePartitionSlot.write(generateWriteProtocol(stream)); } catch (TException e) { diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/TimePartitionUtils.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/TimePartitionUtils.java index 12275a68f4653..798345511ab21 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/TimePartitionUtils.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/utils/TimePartitionUtils.java @@ -177,6 +177,9 @@ public static void removeDatabaseTimePartitionConfig(String database) { // Get database-specific configuration, fallback to global if not found private static DatabaseTimePartitionConfig getDatabaseConfig(String database) { + if (database == null) { + return new DatabaseTimePartitionConfig(timePartitionOrigin, timePartitionInterval); + } DatabaseTimePartitionConfig config = databaseConfigCache.get(database); return config != null ? config @@ -195,6 +198,10 @@ public static long getTimePartitionInterval(String database) { return getDatabaseConfig(database).getTimePartitionInterval(); } + public static long getTimePartitionOrigin(String database) { + return getDatabaseConfig(database).getTimePartitionOrigin(); + } + public static long getTimePartitionLowerBound(long time, String database) { return getTimePartitionLowerBoundInternal(time, getDatabaseConfig(database)); } @@ -324,47 +331,63 @@ private static long getTimePartitionIdWithoutOverflow(long time, long origin, lo } // Original global methods for backward compatibility + @Deprecated public static long getTimePartitionInterval() { return timePartitionInterval; } + @Deprecated + public static long getTimePartitionOrigin() { + return timePartitionOrigin; + } + + @Deprecated public static void setTimePartitionInterval(long timePartitionInterval) { TimePartitionUtils.timePartitionInterval = timePartitionInterval; } // Backward compatibility methods that use global configuration + @Deprecated public static TTimePartitionSlot getTimePartitionSlot(long time) { return getTimePartitionSlot(time, null); } + @Deprecated public static long getTimePartitionLowerBound(long time) { return getTimePartitionLowerBound(time, null); } + @Deprecated public static long getTimePartitionUpperBound(long time) { return getTimePartitionUpperBound(time, null); } + @Deprecated public static long getTimePartitionId(long time) { return getTimePartitionId(time, null); } + @Deprecated public static long getStartTimeByPartitionId(long partitionId) { return getStartTimeByPartitionId(partitionId, null); } + @Deprecated public static boolean satisfyPartitionId(long startTime, long endTime, long partitionId) { return satisfyPartitionId(startTime, endTime, partitionId, null); } + @Deprecated public static boolean satisfyPartitionStartTime(Filter timeFilter, long partitionStartTime) { return satisfyPartitionStartTime(timeFilter, partitionStartTime, null); } + @Deprecated public static boolean satisfyTimePartition(Filter timeFilter, long partitionId) { return satisfyTimePartition(timeFilter, partitionId, null); } + @Deprecated public static long getEstimateTimePartitionSize(long startTime, long endTime) { return getEstimateTimePartitionSize(startTime, endTime, null); } diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift index 92312ee81a307..da84931c2bb27 100644 --- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift +++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift @@ -200,6 +200,11 @@ struct TSetTimePartitionIntervalReq { 2: required i64 timePartitionInterval } +struct TSetTimePartitionOriginReq { + 1: required string database + 2: required i64 timePartitionOrigin +} + struct TCountDatabaseResp { 1: required common.TSStatus status 2: optional i32 count @@ -1416,6 +1421,9 @@ service IConfigNodeRPCService { /** Update the specific Database's PartitionInterval */ common.TSStatus setTimePartitionInterval(TSetTimePartitionIntervalReq req) + /** Update the specific Database's PartitionOrigin */ + common.TSStatus setTimePartitionOrigin(TSetTimePartitionOriginReq req) + /** Count the matched Databases */ TCountDatabaseResp countMatchedDatabases(TGetDatabaseReq req)