From 19343215f005904c0bd0ca41713c73f9671810f5 Mon Sep 17 00:00:00 2001 From: libo Date: Thu, 2 Apr 2026 15:09:59 +0800 Subject: [PATCH 01/15] 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 02/15] 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) From cbfb3812d5058b3e0d10579f5569efb17d25cd8f Mon Sep 17 00:00:00 2001 From: libo Date: Sat, 4 Apr 2026 18:04:53 +0800 Subject: [PATCH 03/15] Adjust to call the method that supports obtaining data region configurations at the database granularity --- .../it/partition/IoTDBTimePartitionIT.java | 2 +- .../AutoCleanPartitionTablePlan.java | 19 +- .../persistence/partition/PartitionInfo.java | 3 +- .../plan/analyze/AnalyzeUtils.java | 3 +- .../executor/ClusterConfigTaskExecutor.java | 2 +- .../config/metadata/GetTimeSlotListTask.java | 12 +- .../planner/distribution/SourceRewriter.java | 8 +- .../storageengine/dataregion/DataRegion.java | 16 +- .../dataregion/tsfile/TsFileManager.java | 5 - .../dataregion/tsfile/TsFileResource.java | 7 +- .../timeindex/ArrayDeviceTimeIndex.java | 21 +- .../tsfile/timeindex/FileTimeIndex.java | 13 +- .../tsfile/timeindex/ITimeIndex.java | 10 +- .../db/tools/TsFileSplitByPartitionTool.java | 47 +-- .../tools/settle/TsFileAndModSettleTool.java | 3 +- .../node/write/WritePlanNodeSplitTest.java | 6 +- .../db/storageengine/StorageEngineTest.java | 23 +- .../dataregion/DataRegionTest.java | 272 +----------------- .../db/storageengine/dataregion/TTLTest.java | 8 +- .../dataregion/TsFileManagerTest.java | 30 +- .../compaction/AbstractCompactionTest.java | 10 +- .../compaction/CompactionSchedulerTest.java | 110 +++---- ...pactionSchedulerWithFastPerformerTest.java | 110 +++---- .../CompactionWithMinTimestampTest.java | 2 +- .../FastAlignedCrossCompactionTest.java | 46 +-- ...tCompactionPerformerWithEmptyPageTest.java | 2 +- ...sistentCompressionTypeAndEncodingTest.java | 12 +- .../FastCrossCompactionPerformerTest.java | 4 +- .../FastInnerCompactionPerformerTest.java | 70 ++--- .../FastNonAlignedCrossCompactionTest.java | 46 +-- .../ReadChunkInnerCompactionTest.java | 18 +- .../ReadPointAlignedCrossCompactionTest.java | 46 +-- ...eadPointNonAlignedCrossCompactionTest.java | 46 +-- .../CompactionDataTypeAlterTableTest.java | 4 +- .../CompactionDataTypeAlterTest.java | 36 ++- ...DataTypeNotMatchAlterableDataTypeTest.java | 30 +- .../CompactionDataTypeNotMatchTest.java | 60 +++- ...actionWithFastPerformerValidationTest.java | 23 +- ...rtionCrossSpaceCompactionSelectorTest.java | 97 ++++--- .../InsertionCrossSpaceCompactionTest.java | 66 ++--- ...sSpaceCompactionWithFastPerformerTest.java | 4 +- ...eCompactionWithReadPointPerformerTest.java | 4 +- .../inner/InnerCompactionMoreDataTest.java | 2 +- .../inner/InnerCompactionSchedulerTest.java | 6 +- .../NewSizeTieredCompactionSelectorTest.java | 30 +- .../SizeTieredCompactionRecoverTest.java | 36 +-- .../RepairUnsortedFileCompactionTest.java | 86 +++--- .../RepairUnsortedFileSchedulerTest.java | 16 +- .../settle/SettleCompactionRecoverTest.java | 20 +- .../settle/SettleCompactionSelectorTest.java | 112 ++++---- .../settle/SettleCompactionTaskTest.java | 34 +-- .../CompactionWithAllNullRowsTest.java | 50 +++- .../TableModelCompactionWithTTLTest.java | 6 +- ...TableModelFastCompactionPerformerTest.java | 18 +- ...ModelReadChunkCompactionPerformerTest.java | 14 +- ...ModelReadPointCompactionPerformerTest.java | 16 +- .../CompactionTaskMemCostEstimatorTest.java | 6 +- .../utils/MultiTsFileDeviceIteratorTest.java | 162 +++++++---- .../snapshot/IoTDBSnapshotTest.java | 3 +- .../utils/ThriftCommonsSerDeUtils.java | 24 +- .../commons/utils/TimePartitionUtils.java | 62 ---- 61 files changed, 1014 insertions(+), 1045 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBTimePartitionIT.java b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBTimePartitionIT.java index 469d6974cb583..5161185e2442e 100644 --- a/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBTimePartitionIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/confignode/it/partition/IoTDBTimePartitionIT.java @@ -142,7 +142,7 @@ public void testTimePartition() throws Exception { } timestatmps.forEach( t -> { - long timePartitionId = TimePartitionUtils.getTimePartitionId(t); + long timePartitionId = TimePartitionUtils.getTimePartitionId(t, "root.sg1"); assertTrue(timePartitions.contains(timePartitionId)); }); } diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/partition/AutoCleanPartitionTablePlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/partition/AutoCleanPartitionTablePlan.java index 1b1bfbe38f029..9e5283f1ba6ed 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/partition/AutoCleanPartitionTablePlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/partition/AutoCleanPartitionTablePlan.java @@ -35,25 +35,25 @@ public class AutoCleanPartitionTablePlan extends ConfigPhysicalPlan { Map databaseTTLMap; - TTimePartitionSlot currentTimeSlot; + Map currentTimeSlotMap; public AutoCleanPartitionTablePlan() { super(ConfigPhysicalPlanType.AutoCleanPartitionTable); } public AutoCleanPartitionTablePlan( - Map databaseTTLMap, TTimePartitionSlot currentTimeSlot) { + Map databaseTTLMap, Map currentTimeSlotMap) { this(); this.databaseTTLMap = databaseTTLMap; - this.currentTimeSlot = currentTimeSlot; + this.currentTimeSlotMap = currentTimeSlotMap; } public Map getDatabaseTTLMap() { return databaseTTLMap; } - public TTimePartitionSlot getCurrentTimeSlot() { - return currentTimeSlot; + public Map getCurrentTimeSlotMap() { + return currentTimeSlotMap; } @Override @@ -64,7 +64,7 @@ protected void serializeImpl(DataOutputStream stream) throws IOException { BasicStructureSerDeUtil.write(entry.getKey(), stream); stream.writeLong(entry.getValue()); } - ThriftCommonsSerDeUtils.serializeTTimePartitionSlot(currentTimeSlot, stream); + ThriftCommonsSerDeUtils.serializeTTimePartitionSlot(currentTimeSlotMap, stream); } @Override @@ -76,7 +76,8 @@ protected void deserializeImpl(ByteBuffer buffer) throws IOException { long value = buffer.getLong(); databaseTTLMap.put(key, value); } - currentTimeSlot = ThriftCommonsSerDeUtils.deserializeTTimePartitionSlot(buffer); + currentTimeSlotMap = + ThriftCommonsSerDeUtils.deserializeTTimePartitionSlotMap(buffer, new TreeMap<>()); } @Override @@ -89,11 +90,11 @@ public boolean equals(Object o) { } AutoCleanPartitionTablePlan that = (AutoCleanPartitionTablePlan) o; return Objects.equals(databaseTTLMap, that.databaseTTLMap) - && Objects.equals(currentTimeSlot, that.currentTimeSlot); + && Objects.equals(currentTimeSlotMap, that.currentTimeSlotMap); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), databaseTTLMap, currentTimeSlot); + return Objects.hash(super.hashCode(), databaseTTLMap, currentTimeSlotMap); } } 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 36436742307fe..98ae75e22fe1f 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,8 @@ public TSStatus autoCleanPartitionTable(AutoCleanPartitionTablePlan plan) { if (isDatabaseExisted(database) && 0 < ttl && ttl < Long.MAX_VALUE) { databasePartitionTables .get(database) - .autoCleanPartitionTable(ttl, plan.getCurrentTimeSlotMap()); + .autoCleanPartitionTable( + ttl, plan.getCurrentTimeSlotMap().getOrDefault(database, null)); } }); return new TSStatus(TSStatusCode.SUCCESS_STATUS.getStatusCode()); 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 199ea04e6ec29..262873d101667 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 @@ -215,7 +215,8 @@ private static DataPartitionQueryParam getTreeDataPartitionQueryParam( InsertTabletStatement statement, MPPQueryContext context) { DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); dataPartitionQueryParam.setDeviceID(statement.getDevicePath().getIDeviceIDAsFullDevice()); - dataPartitionQueryParam.setTimePartitionSlotList(statement.getTimePartitionSlots()); + dataPartitionQueryParam.setTimePartitionSlotList( + statement.getTimePartitionSlots(getDatabaseName(statement, context))); dataPartitionQueryParam.setDatabaseName(getDatabaseName(statement, context)); return dataPartitionQueryParam; } 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 292a9e1e4c719..376e434c62e01 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 @@ -3428,7 +3428,7 @@ public SettableFuture getTimeSlotList( } catch (final Exception e) { future.setException(e); } - GetTimeSlotListTask.buildTSBlock(resp, future); + GetTimeSlotListTask.buildTSBlock(resp, future, getTimeSlotListStatement.getDatabase()); return future; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/GetTimeSlotListTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/GetTimeSlotListTask.java index 7bdd850bd0853..e6ecfe9b58438 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/GetTimeSlotListTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/GetTimeSlotListTask.java @@ -58,11 +58,13 @@ public ListenableFuture execute(IConfigTaskExecutor configTask return configTaskExecutor.getTimeSlotList(getTimeSlotListStatement); } - public static void buildTSBlockRow(TsBlockBuilder builder, TTimePartitionSlot timePartitionSlot) { + public static void buildTSBlockRow( + TsBlockBuilder builder, TTimePartitionSlot timePartitionSlot, String database) { builder.getTimeColumnBuilder().writeLong(0L); builder .getColumnBuilder(0) - .writeLong(TimePartitionUtils.getTimePartitionId(timePartitionSlot.getStartTime())); + .writeLong( + TimePartitionUtils.getTimePartitionId(timePartitionSlot.getStartTime(), database)); builder .getColumnBuilder(1) .writeBinary( @@ -73,14 +75,16 @@ public static void buildTSBlockRow(TsBlockBuilder builder, TTimePartitionSlot ti } public static void buildTSBlock( - TGetTimeSlotListResp getTimeSlotListResp, SettableFuture future) { + TGetTimeSlotListResp getTimeSlotListResp, + SettableFuture future, + String database) { List outputDataTypes = ColumnHeaderConstant.getTimeSlotListColumnHeaders.stream() .map(ColumnHeader::getColumnType) .collect(Collectors.toList()); TsBlockBuilder builder = new TsBlockBuilder(outputDataTypes); - getTimeSlotListResp.getTimeSlotList().forEach(e -> buildTSBlockRow(builder, e)); + getTimeSlotListResp.getTimeSlotList().forEach(e -> buildTSBlockRow(builder, e, database)); DatasetHeader datasetHeader = DatasetHeaderFactory.getGetTimeSlotListHeader(); future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS, builder.build(), datasetHeader)); 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 43990b4171876..ea630a1b0065a 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 @@ -1236,7 +1236,8 @@ private List splitInnerTimeJoinNode( InnerTimeJoinNode innerTimeJoinNode = (InnerTimeJoinNode) node.clone(); innerTimeJoinNode.setPlanNodeId(context.queryContext.getQueryId().genPlanNodeId()); - List timePartitionIds = convertToTimePartitionIds(oneRegion); + List timePartitionIds = + convertToTimePartitionIds(oneRegion, analysis.getDatabaseName()); innerTimeJoinNode.setTimePartitions(timePartitionIds); // region group id -> parent InnerTimeJoinNode @@ -1285,10 +1286,11 @@ private List splitInnerTimeJoinNode( return subInnerJoinNode; } - private List convertToTimePartitionIds(List timePartitionSlotList) { + private List convertToTimePartitionIds( + List timePartitionSlotList, String database) { List res = new ArrayList<>(timePartitionSlotList.size()); for (TTimePartitionSlot timePartitionSlot : timePartitionSlotList) { - res.add(TimePartitionUtils.getTimePartitionId(timePartitionSlot.startTime)); + res.add(TimePartitionUtils.getTimePartitionId(timePartitionSlot.startTime, database)); } return res; } 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 17306204d61ea..40bb4f839181d 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 @@ -1193,7 +1193,8 @@ public void insert(InsertRowNode insertRowNode) throws WriteProcessException { } // init map - long timePartitionId = TimePartitionUtils.getTimePartitionId(insertRowNode.getTime()); + long timePartitionId = + TimePartitionUtils.getTimePartitionId(insertRowNode.getTime(), databaseName); initFlushTimeMap(timePartitionId); boolean isSequence = @@ -1257,7 +1258,7 @@ private void split( int before = loc; long beforeTime = insertTabletNode.getTimes()[before]; // before time partition - long beforeTimePartition = TimePartitionUtils.getTimePartitionId(beforeTime); + long beforeTimePartition = TimePartitionUtils.getTimePartitionId(beforeTime, databaseName); // init flush time map initFlushTimeMap(beforeTimePartition); @@ -1265,7 +1266,7 @@ private void split( boolean isSequence = false; while (loc < endOffset) { long time = insertTabletNode.getTimes()[loc]; - final long timePartitionId = TimePartitionUtils.getTimePartitionId(time); + final long timePartitionId = TimePartitionUtils.getTimePartitionId(time, databaseName); long lastFlushTime; // judge if we should insert sequence @@ -1779,7 +1780,8 @@ private TsFileProcessor insertRowsWithTypeConsistencyCheck( tsFileProcessor.insertRows(subInsertRowsNode, infoForMetrics); } catch (DataTypeInconsistentException e) { InsertRowNode firstRow = subInsertRowsNode.getInsertRowNodeList().get(0); - long timePartitionId = TimePartitionUtils.getTimePartitionId(firstRow.getTime()); + long timePartitionId = + TimePartitionUtils.getTimePartitionId(firstRow.getTime(), databaseName); // flush both MemTables so that the new type can be inserted into a new MemTable TsFileProcessor workSequenceProcessor = workSequenceTsFileProcessors.get(timePartitionId); if (workSequenceProcessor != null) { @@ -4472,7 +4474,8 @@ public void insert(InsertRowsOfOneDeviceNode insertRowsOfOneDeviceNode) continue; } // init map - long timePartitionId = TimePartitionUtils.getTimePartitionId(insertRowNode.getTime()); + long timePartitionId = + TimePartitionUtils.getTimePartitionId(insertRowNode.getTime(), databaseName); if (config.isEnableSeparateData() && !lastFlushTimeMap.checkAndCreateFlushedTimePartition(timePartitionId, true)) { @@ -4596,7 +4599,8 @@ public void insert(InsertRowsNode insertRowsNode) continue; } // init map - timePartitionIds[i] = TimePartitionUtils.getTimePartitionId(insertRowNode.getTime()); + timePartitionIds[i] = + TimePartitionUtils.getTimePartitionId(insertRowNode.getTime(), databaseName); if (config.isEnableSeparateData() && !lastFlushTimeMap.checkAndCreateFlushedTimePartition(timePartitionIds[i], true)) { 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 851f934249fee..af761c065b5ba 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,11 +65,6 @@ public TsFileManager(String storageGroupName, String dataRegionId, String dataRe this.dataRegionId = dataRegionId; } - // @todo - public List getTsFileList(boolean sequence) { - return getTsFileList(sequence, null, null, ""); - } - public List getTsFileList(boolean sequence, String database) { return getTsFileList(sequence, null, null, database); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java index 3548a795404ad..99624430e8081 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/TsFileResource.java @@ -1124,12 +1124,7 @@ public long getTimePartition() { * @throws PartitionViolationException if the data of the file spans partitions or it is empty */ public long getTimePartitionWithCheck() throws PartitionViolationException { - return timeIndex.getTimePartitionWithCheck(file.toString()); - } - - /** Check whether the tsFile spans multiple time partitions. */ - public boolean isSpanMultiTimePartitions() { - return timeIndex.isSpanMultiTimePartitions(); + return timeIndex.getTimePartitionWithCheck(file.toString(), getDatabaseName()); } public void setExclusiveModFile(ModificationFile exclusiveModFile) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ArrayDeviceTimeIndex.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ArrayDeviceTimeIndex.java index caca8e9fdba44..f06b5d1e499eb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ArrayDeviceTimeIndex.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ArrayDeviceTimeIndex.java @@ -268,11 +268,11 @@ private long[] enLargeArray(long[] array, long defaultValue) { } @Override - public long getTimePartition(String tsFilePath) { + public long getTimePartition(String tsFilePath, String database) { try { if (deviceToIndex != null && !deviceToIndex.isEmpty()) { return TimePartitionUtils.getTimePartitionId( - startTimes[deviceToIndex.values().iterator().next()]); + startTimes[deviceToIndex.values().iterator().next()], database); } String[] filePathSplits = FilePathUtils.splitTsFilePath(tsFilePath); return Long.parseLong(filePathSplits[filePathSplits.length - 2]); @@ -282,30 +282,33 @@ public long getTimePartition(String tsFilePath) { } @Override - public long getTimePartitionWithCheck(String tsFilePath) throws PartitionViolationException { + public long getTimePartitionWithCheck(String tsFilePath, String database) + throws PartitionViolationException { try { - return getTimePartitionWithCheck(); + return getTimePartitionWithCheck(database); } catch (PartitionViolationException e) { throw new PartitionViolationException(tsFilePath); } } @Override - public boolean isSpanMultiTimePartitions() { + public boolean isSpanMultiTimePartitions(String database) { try { - getTimePartitionWithCheck(); + getTimePartitionWithCheck(database); return false; } catch (PartitionViolationException e) { return true; } } - private long getTimePartitionWithCheck() throws PartitionViolationException { + private long getTimePartitionWithCheck(String database) throws PartitionViolationException { Long partitionId = null; for (final int index : deviceToIndex.values()) { - final long startTimePartitionId = TimePartitionUtils.getTimePartitionId(startTimes[index]); - final long endTimePartitionId = TimePartitionUtils.getTimePartitionId(endTimes[index]); + final long startTimePartitionId = + TimePartitionUtils.getTimePartitionId(startTimes[index], database); + final long endTimePartitionId = + TimePartitionUtils.getTimePartitionId(endTimes[index], database); if (startTimePartitionId != endTimePartitionId) { throw new PartitionViolationException(); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndex.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndex.java index 2cf89a02626aa..3854c8c6622af 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndex.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndex.java @@ -191,9 +191,10 @@ public long getTimePartition(String tsFilePath) { } @Override - public long getTimePartitionWithCheck(String tsFilePath) throws PartitionViolationException { - final long startPartitionId = TimePartitionUtils.getTimePartitionId(startTime); - final long endPartitionId = TimePartitionUtils.getTimePartitionId(endTime); + public long getTimePartitionWithCheck(String tsFilePath, String database) + throws PartitionViolationException { + final long startPartitionId = TimePartitionUtils.getTimePartitionId(startTime, database); + final long endPartitionId = TimePartitionUtils.getTimePartitionId(endTime, database); if (startPartitionId == endPartitionId) { return startPartitionId; @@ -203,9 +204,9 @@ public long getTimePartitionWithCheck(String tsFilePath) throws PartitionViolati } @Override - public boolean isSpanMultiTimePartitions() { - return TimePartitionUtils.getTimePartitionId(startTime) - != TimePartitionUtils.getTimePartitionId(endTime); + public boolean isSpanMultiTimePartitions(String database) { + return TimePartitionUtils.getTimePartitionId(startTime, database) + != TimePartitionUtils.getTimePartitionId(endTime, database); } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ITimeIndex.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ITimeIndex.java index 114a207d75794..3d6b050ac816b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ITimeIndex.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/ITimeIndex.java @@ -102,25 +102,29 @@ ITimeIndex deserialize(InputStream inputStream, IDeviceID.Deserializer deseriali * get time partition * * @param tsFilePath tsFile absolute path + * @param database database name * @return partition */ - long getTimePartition(String tsFilePath); + long getTimePartition(String tsFilePath, String database); /** * get time partition with check. If data of tsFile spans partitions, an exception will be thrown * * @param tsFilePath tsFile path + * @param database database name * @return partition * @throws PartitionViolationException data of tsFile spans partitions */ - long getTimePartitionWithCheck(String tsFilePath) throws PartitionViolationException; + long getTimePartitionWithCheck(String tsFilePath, String database) + throws PartitionViolationException; /** * Check whether the tsFile spans multiple time partitions. * + * @param database database name * @return true if the tsFile spans multiple time partitions, otherwise false. */ - boolean isSpanMultiTimePartitions(); + boolean isSpanMultiTimePartitions(String database); /** * update start time diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/TsFileSplitByPartitionTool.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/TsFileSplitByPartitionTool.java index 38686bee04774..146bacd17c674 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/TsFileSplitByPartitionTool.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/TsFileSplitByPartitionTool.java @@ -119,7 +119,7 @@ public static void rewriteTsFile( throws IOException, WriteProcessException, IllegalPathException { try (TsFileSplitByPartitionTool rewriteTool = new TsFileSplitByPartitionTool(resourceToBeRewritten)) { - rewriteTool.parseAndRewriteFile(rewrittenResources); + rewriteTool.parseAndRewriteFile(rewrittenResources, resourceToBeRewritten.getDatabaseName()); } } @@ -134,7 +134,7 @@ public void close() throws IOException { * @throws IOException WriteProcessException */ @SuppressWarnings({"squid:S3776", "deprecation"}) // Suppress high Cognitive Complexity warning - public void parseAndRewriteFile(List rewrittenResources) + public void parseAndRewriteFile(List rewrittenResources, String database) throws IOException, WriteProcessException, IllegalPathException { // check if the TsFile has correct header if (!fileCheck()) { @@ -181,7 +181,8 @@ public void parseAndRewriteFile(List rewrittenResources) // a new Page PageHeader pageHeader = reader.readPageHeader(dataType, header.getChunkType() == MetaMarker.CHUNK_HEADER); - boolean needToDecode = checkIfNeedToDecode(measurementSchema, deviceId, pageHeader); + boolean needToDecode = + checkIfNeedToDecode(measurementSchema, deviceId, pageHeader, database); needToDecodeInfo.add(needToDecode); ByteBuffer pageData = !needToDecode @@ -198,7 +199,8 @@ public void parseAndRewriteFile(List rewrittenResources) pageHeadersInChunk, dataInChunk, needToDecodeInfo, - chunkHeaderOffset); + chunkHeaderOffset, + database); firstChunkInChunkGroup = false; break; case MetaMarker.OPERATION_INDEX_RANGE: @@ -248,7 +250,7 @@ public void parseAndRewriteFile(List rewrittenResources) * false. */ protected boolean checkIfNeedToDecode( - MeasurementSchema schema, IDeviceID deviceId, PageHeader pageHeader) { + MeasurementSchema schema, IDeviceID deviceId, PageHeader pageHeader, String database) { if (pageHeader.getStatistics() == null) { return true; } @@ -267,8 +269,8 @@ protected boolean checkIfNeedToDecode( } } } - return TimePartitionUtils.getTimePartitionId(pageHeader.getStartTime()) - != TimePartitionUtils.getTimePartitionId(pageHeader.getEndTime()); + return TimePartitionUtils.getTimePartitionId(pageHeader.getStartTime(), database) + != TimePartitionUtils.getTimePartitionId(pageHeader.getEndTime(), database); } /** @@ -283,17 +285,27 @@ protected void reWriteChunk( List pageHeadersInChunk, List pageDataInChunk, List needToDecodeInfoInChunk, - long chunkHeaderOffset) + long chunkHeaderOffset, + String database) throws IOException, PageException, IllegalPathException { valueDecoder = Decoder.getDecoderByType(schema.getEncodingType(), schema.getType()); Map partitionChunkWriterMap = new HashMap<>(); for (int i = 0; i < pageDataInChunk.size(); i++) { if (Boolean.TRUE.equals(needToDecodeInfoInChunk.get(i))) { decodeAndWritePage( - deviceId, schema, pageDataInChunk.get(i), partitionChunkWriterMap, chunkHeaderOffset); + deviceId, + schema, + pageDataInChunk.get(i), + partitionChunkWriterMap, + chunkHeaderOffset, + database); } else { writePage( - schema, pageHeadersInChunk.get(i), pageDataInChunk.get(i), partitionChunkWriterMap); + schema, + pageHeadersInChunk.get(i), + pageDataInChunk.get(i), + partitionChunkWriterMap, + database); } } for (Entry entry : partitionChunkWriterMap.entrySet()) { @@ -352,9 +364,10 @@ protected void writePage( MeasurementSchema schema, PageHeader pageHeader, ByteBuffer pageData, - Map partitionChunkWriterMap) + Map partitionChunkWriterMap, + String database) throws PageException { - long partitionId = TimePartitionUtils.getTimePartitionId(pageHeader.getStartTime()); + long partitionId = TimePartitionUtils.getTimePartitionId(pageHeader.getStartTime(), database); getOrDefaultTsFileIOWriter(oldTsFile, partitionId); ChunkWriterImpl chunkWriter = partitionChunkWriterMap.computeIfAbsent(partitionId, v -> new ChunkWriterImpl(schema)); @@ -366,7 +379,8 @@ protected void decodeAndWritePage( MeasurementSchema schema, ByteBuffer pageData, Map partitionChunkWriterMap, - long chunkHeaderOffset) + long chunkHeaderOffset, + String database) throws IOException, IllegalPathException { valueDecoder.reset(); PageReader pageReader = @@ -375,7 +389,7 @@ protected void decodeAndWritePage( List deleteIntervalList = getOldSortedDeleteIntervals(deviceId, schema); pageReader.setDeleteIntervalList(deleteIntervalList); BatchData batchData = pageReader.getAllSatisfiedPageData(); - rewritePageIntoFiles(batchData, schema, partitionChunkWriterMap); + rewritePageIntoFiles(batchData, schema, partitionChunkWriterMap, database); } private List getOldSortedDeleteIntervals( @@ -400,11 +414,12 @@ private List getOldSortedDeleteIntervals( protected void rewritePageIntoFiles( BatchData batchData, MeasurementSchema schema, - Map partitionChunkWriterMap) { + Map partitionChunkWriterMap, + String database) { while (batchData.hasCurrent()) { long time = batchData.currentTime(); Object value = batchData.currentValue(); - long partitionId = TimePartitionUtils.getTimePartitionId(time); + long partitionId = TimePartitionUtils.getTimePartitionId(time, database); ChunkWriterImpl chunkWriter = partitionChunkWriterMap.computeIfAbsent(partitionId, v -> new ChunkWriterImpl(schema)); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/settle/TsFileAndModSettleTool.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/settle/TsFileAndModSettleTool.java index 9f01e35e989cd..deb60f43068d8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/settle/TsFileAndModSettleTool.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/tools/settle/TsFileAndModSettleTool.java @@ -214,7 +214,8 @@ public void settleOneTsFileAndMod( } try (TsFileSplitByPartitionTool tsFileRewriteTool = new TsFileSplitByPartitionTool(resourceToBeSettled)) { - tsFileRewriteTool.parseAndRewriteFile(settledResources); + tsFileRewriteTool.parseAndRewriteFile( + settledResources, resourceToBeSettled.getDatabaseName()); } } 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 f8544816b1d3c..41885c5fc3871 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 @@ -113,8 +113,7 @@ 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; + long startTime = t * TimePartitionUtils.getTimePartitionInterval("root.sg1") + 1; timePartitionSlotMap.put( TimePartitionUtils.getTimePartitionSlot(startTime, "root.sg1"), Collections.singletonList( @@ -365,8 +364,7 @@ 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))); - @SuppressWarnings("deprecation") - long interval = TimePartitionUtils.getTimePartitionInterval(); + long interval = TimePartitionUtils.getTimePartitionInterval("root.sg1"); insertRowNode.setTime((i - 2) * interval); insertRowsNode.addOneInsertRowNode(insertRowNode, 2 * i); 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 855b7407d0d00..2817fa6d6868e 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,17 +74,16 @@ public void testGetAllDataRegionIds() throws Exception { @Test public void testGetTimePartitionId() { - @SuppressWarnings("deprecation") - long timePartitionInterval = TimePartitionUtils.getTimePartitionInterval(); - @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(interval / 2)); - Assert.assertEquals(1, TimePartitionUtils.getTimePartitionId(interval * 2 - 1)); - Assert.assertEquals(2, TimePartitionUtils.getTimePartitionId(timePartitionInterval * 2 + 1)); + long timePartitionInterval = TimePartitionUtils.getTimePartitionInterval("root.db"); + long interval = TimePartitionUtils.getTimePartitionInterval("root.db"); + Assert.assertEquals(-2, TimePartitionUtils.getTimePartitionId(-interval - 1, "root.db")); + Assert.assertEquals(-1, TimePartitionUtils.getTimePartitionId(-interval, "root.db")); + Assert.assertEquals(-1, TimePartitionUtils.getTimePartitionId(-1, "root.db")); + Assert.assertEquals(0, TimePartitionUtils.getTimePartitionId(0, "root.db")); + Assert.assertEquals(0, TimePartitionUtils.getTimePartitionId(1, "root.db")); + Assert.assertEquals(0, TimePartitionUtils.getTimePartitionId(interval / 2, "root.db")); + Assert.assertEquals(1, TimePartitionUtils.getTimePartitionId(interval * 2 - 1, "root.db")); + Assert.assertEquals( + 2, TimePartitionUtils.getTimePartitionId(timePartitionInterval * 2 + 1, "root.db")); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java index 0cb7143e708ab..993c87a4f3b1e 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java @@ -58,7 +58,6 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.constant.InnerUnsequenceCompactionSelector; import org.apache.iotdb.db.storageengine.dataregion.compaction.utils.CompactionConfigRestorer; import org.apache.iotdb.db.storageengine.dataregion.flush.FlushManager; -import org.apache.iotdb.db.storageengine.dataregion.flush.TsFileFlushPolicy; import org.apache.iotdb.db.storageengine.dataregion.memtable.ReadOnlyMemChunk; import org.apache.iotdb.db.storageengine.dataregion.memtable.TsFileProcessor; import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; @@ -106,6 +105,7 @@ public class DataRegionTest { private static final CommonConfig COMMON_CONFIG = CommonDescriptor.getInstance().getConfig(); private static final Logger logger = LoggerFactory.getLogger(DataRegionTest.class); + private String databaseName = "root.vehicle"; private String storageGroup = "root.vehicle.d0"; private String systemDir = TestConstant.OUTPUT_DATA_DIR.concat("info"); private String deviceId = "root.vehicle.d0"; @@ -1432,7 +1432,8 @@ public void testDeleteDataNotInFlushingMemtable() record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); dataRegion.insert(buildInsertRowNodeByTSRecord(record)); } - TsFileResource tsFileResource = dataRegion.getTsFileManager().getTsFileList(true).get(0); + TsFileResource tsFileResource = + dataRegion.getTsFileManager().getTsFileList(true, databaseName).get(0); TsFileProcessor tsFileProcessor = tsFileResource.getProcessor(); tsFileProcessor.getFlushingMemTable().addLast(tsFileProcessor.getWorkMemTable()); @@ -1462,265 +1463,8 @@ public void testDeleteDataInSeqFlushingMemtable() record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); dataRegion.insert(buildInsertRowNodeByTSRecord(record)); } - TsFileResource tsFileResource = dataRegion.getTsFileManager().getTsFileList(true).get(0); - TsFileProcessor tsFileProcessor = tsFileResource.getProcessor(); - tsFileProcessor.getFlushingMemTable().addLast(tsFileProcessor.getWorkMemTable()); - - MeasurementPath path = new MeasurementPath("root.vehicle.d0.s0"); - DeleteDataNode deleteDataNode1 = - new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 99); - deleteDataNode1.setSearchIndex(0); - MeasurementPath path2 = new MeasurementPath("root.vehicle.d200.s0"); - DeleteDataNode deleteDataNode2 = - new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path2), 50, 70); - deleteDataNode2.setSearchIndex(0); - // delete data which is not in flushing memtable - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode1); - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode2); - - DeleteDataNode deleteDataNode3 = - new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path), 50, 100); - deleteDataNode3.setSearchIndex(0); - DeleteDataNode deleteDataNode4 = - new DeleteDataNode(new PlanNodeId("4"), Collections.singletonList(path), 50, 150); - deleteDataNode4.setSearchIndex(0); - DeleteDataNode deleteDataNode5 = - new DeleteDataNode(new PlanNodeId("5"), Collections.singletonList(path), 100, 190); - deleteDataNode5.setSearchIndex(0); - // delete data which is in flushing memtable - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode3); - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode4); - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode4); - - dataRegion.syncCloseAllWorkingTsFileProcessors(); - assertTrue(tsFileResource.anyModFileExists()); - Assert.assertEquals(3, tsFileResource.getAllModEntries().size()); - } - - @Test - public void testDeleteDataInUnSeqFlushingMemtable() - throws IllegalPathException, WriteProcessException, IOException { - for (int j = 100; j < 200; j++) { - TSRecord record = new TSRecord(deviceId, j); - record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); - dataRegion.insert(buildInsertRowNodeByTSRecord(record)); - } - TsFileResource tsFileResource = dataRegion.getTsFileManager().getTsFileList(true).get(0); - - MeasurementPath path = new MeasurementPath("root.vehicle.d0.s0"); - DeleteDataNode deleteDataNode1 = - new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 99); - deleteDataNode1.setSearchIndex(0); - MeasurementPath path2 = new MeasurementPath("root.vehicle.d200.s0"); - DeleteDataNode deleteDataNode2 = - new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path2), 50, 70); - deleteDataNode2.setSearchIndex(0); - // delete data which is not in work memtable - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode1); - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode2); - - DeleteDataNode deleteDataNode3 = - new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path), 50, 100); - deleteDataNode3.setSearchIndex(0); - DeleteDataNode deleteDataNode4 = - new DeleteDataNode(new PlanNodeId("4"), Collections.singletonList(path), 50, 150); - deleteDataNode4.setSearchIndex(0); - DeleteDataNode deleteDataNode5 = - new DeleteDataNode(new PlanNodeId("5"), Collections.singletonList(path), 100, 190); - deleteDataNode5.setSearchIndex(0); - // delete data which is in work memtable - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode3); - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode4); - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode5); - - dataRegion.syncCloseAllWorkingTsFileProcessors(); - Assert.assertFalse(tsFileResource.anyModFileExists()); - - // insert unseq data points - for (int j = 50; j < 100; j++) { - TSRecord record = new TSRecord(deviceId, j); - record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); - dataRegion.insert(buildInsertRowNodeByTSRecord(record)); - } - - DeleteDataNode deleteDataNode6 = - new DeleteDataNode(new PlanNodeId("6"), Collections.singletonList(path), 200, 299); - deleteDataNode6.setSearchIndex(0); - // delete data which is not in work memtable - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode6); - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d200.s0"), deleteDataNode2); - - DeleteDataNode deleteDataNode7 = - new DeleteDataNode(new PlanNodeId("7"), Collections.singletonList(path), 80, 85); - deleteDataNode7.setSearchIndex(0); - // delete data which is in work memtable - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode7); - - Assert.assertFalse(tsFileResource.anyModFileExists()); - - tsFileResource = dataRegion.getTsFileManager().getTsFileList(false).get(0); - TsFileProcessor tsFileProcessor = tsFileResource.getProcessor(); - tsFileProcessor.getFlushingMemTable().addLast(tsFileProcessor.getWorkMemTable()); - - DeleteDataNode deleteDataNode8 = - new DeleteDataNode(new PlanNodeId("8"), Collections.singletonList(path), 0, 49); - deleteDataNode8.setSearchIndex(0); - DeleteDataNode deleteDataNode9 = - new DeleteDataNode(new PlanNodeId("9"), Collections.singletonList(path), 100, 200); - deleteDataNode9.setSearchIndex(0); - // delete data which is not in flushing memtable - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode8); - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode9); - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d200.s0"), deleteDataNode2); - - DeleteDataNode deleteDataNode10 = - new DeleteDataNode(new PlanNodeId("10"), Collections.singletonList(path2), 25, 50); - deleteDataNode10.setSearchIndex(0); - DeleteDataNode deleteDataNode11 = - new DeleteDataNode(new PlanNodeId("11"), Collections.singletonList(path2), 50, 80); - deleteDataNode11.setSearchIndex(0); - DeleteDataNode deleteDataNode12 = - new DeleteDataNode(new PlanNodeId("12"), Collections.singletonList(path2), 99, 150); - deleteDataNode12.setSearchIndex(0); - // delete data which is in flushing memtable - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode10); - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode11); - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode12); - - dataRegion.syncCloseAllWorkingTsFileProcessors(); - assertTrue(tsFileResource.anyModFileExists()); - Assert.assertEquals(3, tsFileResource.getAllModEntries().size()); - } - - @Test - public void testDeleteDataInSeqWorkingMemtable() - throws IllegalPathException, WriteProcessException, IOException { - for (int j = 100; j < 200; j++) { - TSRecord record = new TSRecord("root.vehicle.d0", j); - record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); - dataRegion.insert(buildInsertRowNodeByTSRecord(record)); - } - for (int j = 100; j < 200; j++) { - TSRecord record = new TSRecord("root.vehicle.d199", j); - record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); - dataRegion.insert(buildInsertRowNodeByTSRecord(record)); - } - TsFileResource tsFileResource = dataRegion.getTsFileManager().getTsFileList(true).get(0); - - MeasurementPath path = new MeasurementPath("root.vehicle.d0.s0"); - DeleteDataNode deleteDataNode1 = - new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 99); - deleteDataNode1.setSearchIndex(0); - MeasurementPath path2 = new MeasurementPath("root.vehicle.d200.s0"); - DeleteDataNode deleteDataNode2 = - new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path2), 50, 70); - deleteDataNode2.setSearchIndex(0); - // delete data which is not in working memtable - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode1); - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d200.s0"), deleteDataNode2); - - MeasurementPath path3 = new MeasurementPath("root.vehicle.d199.*"); - DeleteDataNode deleteDataNode3 = - new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path3), 50, 500); - deleteDataNode3.setSearchIndex(0); - // delete data which is in working memtable - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d199.*"), deleteDataNode3); - - dataRegion.syncCloseAllWorkingTsFileProcessors(); - Assert.assertFalse(tsFileResource.anyModFileExists()); - Assert.assertFalse( - tsFileResource - .getDevices() - .contains(IDeviceID.Factory.DEFAULT_FACTORY.create("root.vehicle.d199"))); - } - - @Test - public void testFlushingEmptyMemtable() - throws IllegalPathException, WriteProcessException, IOException { - for (int j = 100; j < 200; j++) { - TSRecord record = new TSRecord(deviceId, j); - record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); - dataRegion.insert(buildInsertRowNodeByTSRecord(record)); - } - TsFileResource tsFileResource = dataRegion.getTsFileManager().getTsFileList(true).get(0); - - MeasurementPath path = new MeasurementPath("root.vehicle.d0.s0"); - DeleteDataNode deleteDataNode = - new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 100, 200); - deleteDataNode.setSearchIndex(0); - // delete all data which is in flushing memtable - dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode); - - dataRegion.syncCloseAllWorkingTsFileProcessors(); - Assert.assertFalse(tsFileResource.getTsFile().exists()); - Assert.assertFalse(tsFileResource.anyModFileExists()); - Assert.assertFalse(dataRegion.getTsFileManager().contains(tsFileResource, true)); - Assert.assertFalse( - dataRegion.getWorkSequenceTsFileProcessors().contains(tsFileResource.getProcessor())); - } - - public static class DummyDataRegion extends DataRegion { - - public DummyDataRegion(String systemInfoDir, String storageGroupName) - throws DataRegionException { - super(systemInfoDir, "0", new TsFileFlushPolicy.DirectFlushPolicy(), storageGroupName); - } - } - - // -- test for deleting data directly - // -- delete data and file only when: - // 1. tsfile is closed - // 2. tsfile is not compating - // 3. tsfile's start time and end time must be a subinterval - // of the given time range. - - @Test - public void testDeleteDataDirectlySeqWriteModsOrDeleteFiles() - throws IllegalPathException, WriteProcessException, IOException { - for (int j = 100; j < 200; j++) { - TSRecord record = new TSRecord(deviceId, j); - record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); - dataRegion.insert(buildInsertRowNodeByTSRecord(record)); - } - - TsFileResource tsFileResource = dataRegion.getTsFileManager().getTsFileList(true).get(0); - // delete data in work mem, no mods. - MeasurementPath path = new MeasurementPath("root.vehicle.d0.**"); - DeleteDataNode deleteDataNode1 = - new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 100); - deleteDataNode1.setSearchIndex(0); - dataRegion.deleteDataDirectly(new MeasurementPath("root.vehicle.d0.**"), deleteDataNode1); - assertTrue(tsFileResource.getTsFile().exists()); - Assert.assertFalse(tsFileResource.anyModFileExists()); - - dataRegion.syncCloseAllWorkingTsFileProcessors(); - - // delete data in closed file, but time not match - DeleteDataNode deleteDataNode2 = - new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path), 100, 120); - deleteDataNode2.setSearchIndex(0); - dataRegion.deleteDataDirectly(new MeasurementPath("root.vehicle.d0.**"), deleteDataNode2); - assertTrue(tsFileResource.getTsFile().exists()); - assertTrue(tsFileResource.anyModFileExists()); - - // delete data in closed file, and time all match - DeleteDataNode deleteDataNode3 = - new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path), 100, 199); - deleteDataNode3.setSearchIndex(0); - dataRegion.deleteDataDirectly(new MeasurementPath("root.vehicle.d0.**"), deleteDataNode3); - Assert.assertFalse(tsFileResource.getTsFile().exists()); - Assert.assertFalse(tsFileResource.anyModFileExists()); - } - - @Test - public void testDeleteDataDirectlyUnseqWriteModsOrDeleteFiles() - throws IllegalPathException, WriteProcessException, IOException { - for (int j = 100; j < 200; j++) { - TSRecord record = new TSRecord(deviceId, j); - record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); - dataRegion.insert(buildInsertRowNodeByTSRecord(record)); - } - TsFileResource tsFileResourceSeq = dataRegion.getTsFileManager().getTsFileList(true).get(0); + TsFileResource tsFileResourceSeq = + dataRegion.getTsFileManager().getTsFileList(true, databaseName).get(0); dataRegion.syncCloseAllWorkingTsFileProcessors(); for (int j = 30; j < 100; j++) { TSRecord record = new TSRecord(deviceId, j); @@ -1729,7 +1473,8 @@ public void testDeleteDataDirectlyUnseqWriteModsOrDeleteFiles() } dataRegion.syncCloseWorkingTsFileProcessors(true); - TsFileResource tsFileResourceUnSeq = dataRegion.getTsFileManager().getTsFileList(false).get(0); + TsFileResource tsFileResourceUnSeq = + dataRegion.getTsFileManager().getTsFileList(false, databaseName).get(0); assertTrue(tsFileResourceSeq.getTsFile().exists()); assertTrue(tsFileResourceUnSeq.getTsFile().exists()); @@ -1785,7 +1530,8 @@ public void testFlushSpecifiedResource() record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); dataRegion.insert(buildInsertRowNodeByTSRecord(record)); } - TsFileResource tsFileResourceSeq = dataRegion.getTsFileManager().getTsFileList(true).get(0); + TsFileResource tsFileResourceSeq = + dataRegion.getTsFileManager().getTsFileList(true, databaseName).get(0); Future future = dataRegion.asyncCloseOneTsFileProcessor(tsFileResourceSeq); Future future2 = dataRegion.asyncCloseOneTsFileProcessor(tsFileResourceSeq); assertTrue(future == future2 || future2 instanceof CompletableFuture); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/TTLTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/TTLTest.java index afd7ed608eadb..e20fa6a0119f8 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/TTLTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/TTLTest.java @@ -383,8 +383,8 @@ public void testTTLRemoval() dataRegion.getTsFileManager(), timePartition, new CompactionScheduleContext(), true); } long totalWaitingTime = 0; - while (dataRegion.getTsFileManager().getTsFileList(true).size() - + dataRegion.getTsFileManager().getTsFileList(false).size() + while (dataRegion.getTsFileManager().getTsFileList(true, "root.TTL_SG1").size() + + dataRegion.getTsFileManager().getTsFileList(false, "root.TTL_SG1").size() != 0) { sleep(200); totalWaitingTime += 200; @@ -474,8 +474,8 @@ public void testTTLCleanFile() dataRegion.getTsFileManager(), timePartition, new CompactionScheduleContext(), true); } long totalWaitingTime = 0; - while (dataRegion.getTsFileManager().getTsFileList(true).size() - + dataRegion.getTsFileManager().getTsFileList(false).size() + while (dataRegion.getTsFileManager().getTsFileList(true, "root.TTL_SG1").size() + + dataRegion.getTsFileManager().getTsFileList(false, "root.TTL_SG1").size() != 0) { sleep(200); totalWaitingTime += 200; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/TsFileManagerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/TsFileManagerTest.java index 07989246ae4d4..6983300086b32 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/TsFileManagerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/TsFileManagerTest.java @@ -49,6 +49,7 @@ public class TsFileManagerTest { private TsFileManager tsFileManager; private List seqResources; private List unseqResources; + private static final String databaseName = "test"; @Before public void setUp() throws IOException, WriteProcessException, MetadataException { @@ -87,8 +88,8 @@ public void testAddRemoveAndIterator() { tsFileManager.add(tsFileResource, true); } tsFileManager.addAll(unseqResources, false); - assertEquals(5, tsFileManager.getTsFileList(true).size()); - assertEquals(4, tsFileManager.getTsFileList(false).size()); + assertEquals(5, tsFileManager.getTsFileList(true, databaseName).size()); + assertEquals(4, tsFileManager.getTsFileList(false, databaseName).size()); assertEquals(5, tsFileManager.size(true)); assertEquals(4, tsFileManager.size(false)); assertTrue(tsFileManager.contains(seqResources.get(0), true)); @@ -124,20 +125,20 @@ public void testAddRemoveAndIterator() { false)); assertFalse(tsFileManager.isEmpty(true)); assertFalse(tsFileManager.isEmpty(false)); - tsFileManager.remove(tsFileManager.getTsFileList(true).get(0), true); - tsFileManager.remove(tsFileManager.getTsFileList(false).get(0), false); - assertEquals(4, tsFileManager.getTsFileList(true).size()); - tsFileManager.removeAll(tsFileManager.getTsFileList(false), false); - assertEquals(0, tsFileManager.getTsFileList(false).size()); + tsFileManager.remove(tsFileManager.getTsFileList(true, databaseName).get(0), true); + tsFileManager.remove(tsFileManager.getTsFileList(false, databaseName).get(0), false); + assertEquals(4, tsFileManager.getTsFileList(true, databaseName).size()); + tsFileManager.removeAll(tsFileManager.getTsFileList(false, databaseName), false); + assertEquals(0, tsFileManager.getTsFileList(false, databaseName).size()); long count = 0; - Iterator iterator = tsFileManager.getIterator(true); + Iterator iterator = tsFileManager.getIterator(true, databaseName); while (iterator.hasNext()) { iterator.next(); count++; } assertEquals(4, count); - tsFileManager.removeAll(tsFileManager.getTsFileList(true), true); - assertEquals(0, tsFileManager.getTsFileList(true).size()); + tsFileManager.removeAll(tsFileManager.getTsFileList(true, databaseName), true); + assertEquals(0, tsFileManager.getTsFileList(true, databaseName).size()); assertTrue(tsFileManager.isEmpty(true)); assertTrue(tsFileManager.isEmpty(false)); tsFileManager.add( @@ -181,16 +182,16 @@ public void testIteratorRemove() { tsFileManager.add(tsFileResource, true); } tsFileManager.addAll(seqResources, false); - assertEquals(5, tsFileManager.getTsFileList(true).size()); + assertEquals(5, tsFileManager.getTsFileList(true, databaseName).size()); - Iterator tsFileResourceIterator = tsFileManager.getIterator(true); + Iterator tsFileResourceIterator = tsFileManager.getIterator(true, databaseName); tsFileResourceIterator.next(); try { tsFileResourceIterator.remove(); } catch (UnsupportedOperationException e) { // pass } - assertEquals(5, tsFileManager.getTsFileList(true).size()); + assertEquals(5, tsFileManager.getTsFileList(true, databaseName).size()); TsFileResource tsFileResource1 = new TsFileResource( @@ -232,7 +233,8 @@ public void testIteratorRemove() { + 0 + ".tsfile"))); tsFileManager.add(tsFileResource3, true); - Iterator tsFileResourceIterator2 = tsFileManager.getIterator(true); + Iterator tsFileResourceIterator2 = + tsFileManager.getIterator(true, databaseName); int count = 0; while (tsFileResourceIterator2.hasNext()) { count++; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/AbstractCompactionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/AbstractCompactionTest.java index 39b4d53ef906b..6346a0c20aeb8 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/AbstractCompactionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/AbstractCompactionTest.java @@ -556,7 +556,7 @@ private void removeFiles() throws IOException { protected void validateSeqFiles(boolean isSeq) { TsFileValidationTool.clearMap(true); List files = new ArrayList<>(); - for (TsFileResource resource : tsFileManager.getTsFileList(isSeq)) { + for (TsFileResource resource : tsFileManager.getTsFileList(isSeq, COMPACTION_TEST_SG)) { files.add(resource.getTsFile()); } TsFileValidationTool.findIncorrectFiles(files); @@ -574,8 +574,8 @@ protected Map> readSourceFiles( path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); while (tsBlockReader.hasNextBatch()) { TsBlock block = tsBlockReader.nextBatch(); @@ -601,8 +601,8 @@ protected void validateTargetDatas( entry.getKey(), FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); List timeseriesData = entry.getValue(); tmpSourceDatas.put(entry.getKey(), new ArrayList<>(timeseriesData)); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionSchedulerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionSchedulerTest.java index b1377dbbba6bb..a6274abceccba 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionSchedulerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionSchedulerTest.java @@ -188,7 +188,7 @@ public void test1() throws IOException, MetadataException, InterruptedException long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 1) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 1) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -204,7 +204,7 @@ public void test1() throws IOException, MetadataException, InterruptedException totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 1) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 1) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -220,7 +220,7 @@ public void test1() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -313,7 +313,7 @@ public void test2() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 1) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 1) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -328,7 +328,7 @@ public void test2() throws IOException, MetadataException, InterruptedException } CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(10); totalWaitingTime += 10; @@ -339,8 +339,8 @@ public void test2() throws IOException, MetadataException, InterruptedException if (totalWaitingTime % 10_000 == 0) { logger.warn( "sequence file num is {}, unsequence file num is {}", - tsFileManager.getTsFileList(true).size(), - tsFileManager.getTsFileList(false).size()); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size(), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); } if (totalWaitingTime % SCHEDULE_AGAIN_TIME == 0) { logger.warn("Has waited for {} s, Schedule again", totalWaitingTime / 1000); @@ -350,7 +350,7 @@ public void test2() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - // assertEquals(100, tsFileManager.getTsFileList(true).size()); + // assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -423,7 +423,7 @@ public void test3() throws IOException, MetadataException, InterruptedException } long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 1) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 1) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -438,7 +438,7 @@ public void test3() throws IOException, MetadataException, InterruptedException } } totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -448,7 +448,7 @@ public void test3() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(1, tsFileManager.getTsFileList(true).size()); + assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -520,7 +520,7 @@ public void test4() throws IOException, MetadataException, InterruptedException tsFileManager.add(tsFileResource, false); } long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -534,7 +534,7 @@ public void test4() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -605,7 +605,7 @@ public void test5() throws IOException, MetadataException, InterruptedException } long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 1) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 1) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -620,7 +620,7 @@ public void test5() throws IOException, MetadataException, InterruptedException } } totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 1) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 1) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -636,7 +636,7 @@ public void test5() throws IOException, MetadataException, InterruptedException } totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -719,12 +719,12 @@ public void test6() throws IOException, MetadataException, InterruptedException tsFileManager.add(tsFileResource, false); } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 1) { - assertEquals(100, tsFileManager.getTsFileList(true).size()); + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 1) { + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); try { Thread.sleep(100); totalWaitingTime += 100; @@ -741,8 +741,8 @@ public void test6() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { - assertEquals(100, tsFileManager.getTsFileList(true).size()); + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); try { Thread.sleep(100); totalWaitingTime += 100; @@ -756,7 +756,7 @@ public void test6() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -828,7 +828,7 @@ public void test7() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 1) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 1) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -845,7 +845,7 @@ public void test7() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -932,7 +932,7 @@ public void test8() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -946,7 +946,7 @@ public void test8() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -1020,7 +1020,7 @@ public void test9() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 50) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 50) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1037,7 +1037,7 @@ public void test9() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 25) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 25) { Thread.sleep(100); totalWaitingTime += 100; CompactionScheduler.scheduleCompaction(tsFileManager, 0); @@ -1046,7 +1046,7 @@ public void test9() throws IOException, MetadataException, InterruptedException fail(); } } - assertTrue(tsFileManager.getTsFileList(true).size() <= 25); + assertTrue(tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() <= 25); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -1124,7 +1124,7 @@ public void test10() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 50) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 50) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1138,11 +1138,11 @@ public void test10() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 25) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 25) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1232,7 +1232,7 @@ public void test11() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 50) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 50) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1249,7 +1249,7 @@ public void test11() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 25) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 25) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1336,7 +1336,7 @@ public void test12() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 98) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 98) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1353,11 +1353,11 @@ public void test12() throws IOException, MetadataException, InterruptedException fail(e.getMessage()); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 96) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 96) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1371,7 +1371,7 @@ public void test12() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -1442,16 +1442,16 @@ public void test14() throws IOException, MetadataException, InterruptedException tsFileManager.add(tsFileResource, false); } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); CompactionScheduler.scheduleCompaction(tsFileManager, 0); - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 99) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 99) { try { Thread.sleep(100); totalWaitingTime += 100; - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); CompactionScheduler.scheduleCompaction(tsFileManager, 0); if (totalWaitingTime > MAX_WAITING_TIME) { @@ -1462,14 +1462,14 @@ public void test14() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 98) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 98) { try { Thread.sleep(100); - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); totalWaitingTime += 100; CompactionScheduler.scheduleCompaction(tsFileManager, 0); @@ -1481,7 +1481,7 @@ public void test14() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -1555,7 +1555,7 @@ public void test15() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 99) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 99) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1572,7 +1572,7 @@ public void test15() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 98) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 98) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1586,7 +1586,7 @@ public void test15() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1672,7 +1672,7 @@ public void test16() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 98) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 98) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1686,11 +1686,11 @@ public void test16() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 96) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 96) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1704,7 +1704,7 @@ public void test16() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -1804,10 +1804,10 @@ public void testLargeFileInLowerLevel() throws Exception { CompactionScheduler.scheduleCompaction(tsFileManager, 0); Thread.sleep(100); long sleepTime = 0; - while (tsFileManager.getTsFileList(true).size() >= 2) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() >= 2) { CompactionScheduler.scheduleCompaction(tsFileManager, 0); tsFileManager.readLock(); - List resources = tsFileManager.getTsFileList(true); + List resources = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); int previousFileLevel = TsFileNameGenerator.getTsFileName(resources.get(0).getTsFile().getName()) .getInnerCompactionCnt(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionSchedulerWithFastPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionSchedulerWithFastPerformerTest.java index 747b2a67ea915..5c71a12858637 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionSchedulerWithFastPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionSchedulerWithFastPerformerTest.java @@ -185,7 +185,7 @@ public void test1() throws IOException, MetadataException, InterruptedException long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 1) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 1) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -201,7 +201,7 @@ public void test1() throws IOException, MetadataException, InterruptedException totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 1) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 1) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -217,7 +217,7 @@ public void test1() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -310,7 +310,7 @@ public void test2() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 1) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 1) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -325,7 +325,7 @@ public void test2() throws IOException, MetadataException, InterruptedException } CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(10); totalWaitingTime += 10; @@ -336,8 +336,8 @@ public void test2() throws IOException, MetadataException, InterruptedException if (totalWaitingTime % 10_000 == 0) { logger.warn( "sequence file num is {}, unsequence file num is {}", - tsFileManager.getTsFileList(true).size(), - tsFileManager.getTsFileList(false).size()); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size(), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); } if (totalWaitingTime % SCHEDULE_AGAIN_TIME == 0) { logger.warn("Has waited for {} s, Schedule again", totalWaitingTime / 1000); @@ -347,7 +347,7 @@ public void test2() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - // assertEquals(100, tsFileManager.getTsFileList(true).size()); + // assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -420,7 +420,7 @@ public void test3() throws IOException, MetadataException, InterruptedException } long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 1) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 1) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -435,7 +435,7 @@ public void test3() throws IOException, MetadataException, InterruptedException } } totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -445,7 +445,7 @@ public void test3() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(1, tsFileManager.getTsFileList(true).size()); + assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -517,7 +517,7 @@ public void test4() throws IOException, MetadataException, InterruptedException tsFileManager.add(tsFileResource, false); } long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -531,7 +531,7 @@ public void test4() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -602,7 +602,7 @@ public void test5() throws IOException, MetadataException, InterruptedException } long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 1) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 1) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -617,7 +617,7 @@ public void test5() throws IOException, MetadataException, InterruptedException } } totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 1) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 1) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -633,7 +633,7 @@ public void test5() throws IOException, MetadataException, InterruptedException } totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -716,12 +716,12 @@ public void test6() throws IOException, MetadataException, InterruptedException tsFileManager.add(tsFileResource, false); } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 1) { - assertEquals(100, tsFileManager.getTsFileList(true).size()); + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 1) { + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); try { Thread.sleep(100); totalWaitingTime += 100; @@ -738,8 +738,8 @@ public void test6() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { - assertEquals(100, tsFileManager.getTsFileList(true).size()); + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); try { Thread.sleep(100); totalWaitingTime += 100; @@ -753,7 +753,7 @@ public void test6() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -825,7 +825,7 @@ public void test7() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 1) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 1) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -842,7 +842,7 @@ public void test7() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -929,7 +929,7 @@ public void test8() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -943,7 +943,7 @@ public void test8() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -1017,7 +1017,7 @@ public void test9() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 50) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 50) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1034,7 +1034,7 @@ public void test9() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 25) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 25) { Thread.sleep(100); totalWaitingTime += 100; CompactionScheduler.scheduleCompaction(tsFileManager, 0); @@ -1043,7 +1043,7 @@ public void test9() throws IOException, MetadataException, InterruptedException fail(); } } - assertTrue(tsFileManager.getTsFileList(true).size() <= 25); + assertTrue(tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() <= 25); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -1121,7 +1121,7 @@ public void test10() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 50) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 50) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1135,11 +1135,11 @@ public void test10() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 25) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 25) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1229,7 +1229,7 @@ public void test11() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 50) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 50) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1246,7 +1246,7 @@ public void test11() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 25) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 25) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1333,7 +1333,7 @@ public void test12() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 98) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 98) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1347,11 +1347,11 @@ public void test12() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 96) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 96) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1365,7 +1365,7 @@ public void test12() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -1436,16 +1436,16 @@ public void test14() throws IOException, MetadataException, InterruptedException tsFileManager.add(tsFileResource, false); } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); CompactionScheduler.scheduleCompaction(tsFileManager, 0); - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 99) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 99) { try { Thread.sleep(100); totalWaitingTime += 100; - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); CompactionScheduler.scheduleCompaction(tsFileManager, 0); if (totalWaitingTime > MAX_WAITING_TIME) { @@ -1456,14 +1456,14 @@ public void test14() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 98) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 98) { try { Thread.sleep(100); - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); totalWaitingTime += 100; CompactionScheduler.scheduleCompaction(tsFileManager, 0); @@ -1475,7 +1475,7 @@ public void test14() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -1549,7 +1549,7 @@ public void test15() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 99) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 99) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1566,7 +1566,7 @@ public void test15() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(true).size() > 98) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() > 98) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1580,7 +1580,7 @@ public void test15() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1666,7 +1666,7 @@ public void test16() throws IOException, MetadataException, InterruptedException CompactionScheduler.scheduleCompaction(tsFileManager, 0); long totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 98) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 98) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1680,11 +1680,11 @@ public void test16() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); CompactionScheduler.scheduleCompaction(tsFileManager, 0); totalWaitingTime = 0; - while (tsFileManager.getTsFileList(false).size() > 96) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 96) { try { Thread.sleep(100); totalWaitingTime += 100; @@ -1698,7 +1698,7 @@ public void test16() throws IOException, MetadataException, InterruptedException e.printStackTrace(); } } - assertEquals(100, tsFileManager.getTsFileList(true).size()); + assertEquals(100, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); tsFileManager.setAllowCompaction(false); stopCompactionTaskManager(); } finally { @@ -1798,10 +1798,10 @@ public void testLargeFileInLowerLevel() throws Exception { CompactionScheduler.scheduleCompaction(tsFileManager, 0); Thread.sleep(100); long sleepTime = 0; - while (tsFileManager.getTsFileList(true).size() >= 2) { + while (tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size() >= 2) { CompactionScheduler.scheduleCompaction(tsFileManager, 0); tsFileManager.readLock(); - List resources = tsFileManager.getTsFileList(true); + List resources = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); int previousFileLevel = TsFileNameGenerator.getTsFileName(resources.get(0).getTsFile().getName()) .getInnerCompactionCnt(); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionWithMinTimestampTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionWithMinTimestampTest.java index 567c67128905a..91174a22dfd56 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionWithMinTimestampTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/CompactionWithMinTimestampTest.java @@ -119,7 +119,7 @@ public void testCompactionPerformer() throws IOException { new InnerSpaceCompactionTask( 0, tsFileManager, Collections.singletonList(resource), false, performer, 0); Assert.assertTrue(task.start()); - TsFileResource target = tsFileManager.getTsFileList(false).get(0); + TsFileResource target = tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0); Assert.assertEquals( Long.MIN_VALUE, target diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastAlignedCrossCompactionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastAlignedCrossCompactionTest.java index f4c59e8743d51..b9c57d5d2dc99 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastAlignedCrossCompactionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastAlignedCrossCompactionTest.java @@ -301,7 +301,7 @@ public void test1() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -606,7 +606,7 @@ public void test2() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -910,7 +910,7 @@ public void test3() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -1231,7 +1231,7 @@ public void test4() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -1667,7 +1667,7 @@ public void test5() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -2111,7 +2111,7 @@ public void test6() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -2568,7 +2568,7 @@ public void test7() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -3006,7 +3006,7 @@ public void test8() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -3445,7 +3445,7 @@ public void test9() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -3898,7 +3898,7 @@ public void test10() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -4410,7 +4410,7 @@ public void test11() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -4925,7 +4925,7 @@ public void test12() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -5440,7 +5440,7 @@ public void test13() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -6015,7 +6015,7 @@ public void test14() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -6527,7 +6527,7 @@ public void test15() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -6788,7 +6788,7 @@ public void test16() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -7119,7 +7119,7 @@ public void test17() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -7601,7 +7601,7 @@ public void test18() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -8040,7 +8040,7 @@ public void test19() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -8345,7 +8345,7 @@ public void test20() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -8512,7 +8512,7 @@ public void test21() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -8738,7 +8738,7 @@ public void test22() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -9009,7 +9009,7 @@ public void test23() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCompactionPerformerWithEmptyPageTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCompactionPerformerWithEmptyPageTest.java index 7e9c25316a548..f4d117458a095 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCompactionPerformerWithEmptyPageTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCompactionPerformerWithEmptyPageTest.java @@ -122,7 +122,7 @@ public void test1() throws IOException, IllegalPathException { } catch (Exception e) { Assert.fail(); } - TsFileResource result = tsFileManager.getTsFileList(true).get(0); + TsFileResource result = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); result.buildDeviceTimeIndex(); Assert.assertEquals(20, ((long) result.getStartTime(device).get())); Assert.assertEquals(50, ((long) result.getEndTime(device).get())); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCompactionPerformerWithInconsistentCompressionTypeAndEncodingTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCompactionPerformerWithInconsistentCompressionTypeAndEncodingTest.java index f62480c55291f..2fccf21f20a13 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCompactionPerformerWithInconsistentCompressionTypeAndEncodingTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCompactionPerformerWithInconsistentCompressionTypeAndEncodingTest.java @@ -124,7 +124,7 @@ public void test1() throws MetadataException, IOException, WriteProcessException 0); Assert.assertTrue(task.start()); - TsFileResource targetFile = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetFile = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); TsFileSequenceReader reader = new TsFileSequenceReader(targetFile.getTsFilePath()); validateSingleTsFileWithNonAlignedSeries(reader); } @@ -202,7 +202,7 @@ public void test2() throws MetadataException, IOException, WriteProcessException 0); Assert.assertTrue(task.start()); - TsFileResource targetFile = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetFile = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); TsFileSequenceReader reader = new TsFileSequenceReader(targetFile.getTsFilePath()); validateSingleTsFileWithNonAlignedSeries(reader); } @@ -276,7 +276,7 @@ public void test3() throws MetadataException, IOException, WriteProcessException 0); Assert.assertTrue(task.start()); - TsFileResource targetFile = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetFile = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); TsFileSequenceReader reader = new TsFileSequenceReader(targetFile.getTsFilePath()); // validateSingleTsFileWithNonAlignedSeries(reader); } @@ -327,7 +327,7 @@ public void test4() throws MetadataException, IOException, WriteProcessException 0); Assert.assertTrue(task.start()); - TsFileResource targetFile = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetFile = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); TsFileSequenceReader reader = new TsFileSequenceReader(targetFile.getTsFilePath()); // validateSingleTsFileWithNonAlignedSeries(reader); } @@ -405,7 +405,7 @@ public void test5() throws MetadataException, IOException, WriteProcessException 0); Assert.assertTrue(task.start()); - TsFileResource targetFile = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetFile = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); TsFileSequenceReader reader = new TsFileSequenceReader(targetFile.getTsFilePath()); // validateSingleTsFileWithNonAlignedSeries(reader); } @@ -479,7 +479,7 @@ public void test6() throws MetadataException, IOException, WriteProcessException 0); Assert.assertTrue(task.start()); - TsFileResource targetFile = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetFile = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); TsFileSequenceReader reader = new TsFileSequenceReader(targetFile.getTsFilePath()); validateSingleTsFileWithAlignedSeries(reader); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCrossCompactionPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCrossCompactionPerformerTest.java index 25f3acd89ee62..4928d000b2a59 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCrossCompactionPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastCrossCompactionPerformerTest.java @@ -4764,8 +4764,8 @@ public void testCompactionWithMinTimestamp() throws IOException { 1000, 0); Assert.assertTrue(task.start()); - TsFileResource target1 = tsFileManager.getTsFileList(true).get(0); - TsFileResource target2 = tsFileManager.getTsFileList(true).get(1); + TsFileResource target1 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); + TsFileResource target2 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1); Assert.assertEquals(1, target1.getDevices().size()); Assert.assertEquals(1, target2.getDevices().size()); Assert.assertEquals( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastInnerCompactionPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastInnerCompactionPerformerTest.java index d75d7c4ede795..3c056e5eb303b 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastInnerCompactionPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastInnerCompactionPerformerTest.java @@ -156,8 +156,8 @@ public void testSeqInnerSpaceCompactionWithSameTimeseries() throws Exception { path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); count = 0; @@ -240,7 +240,7 @@ public void testSeqInnerSpaceCompactionWithDifferentTimeseries() throws Exceptio Assert.assertTrue(task.start()); Assert.assertEquals(0, FileReaderManager.getInstance().getClosedFileReaderMap().size()); Assert.assertEquals(0, FileReaderManager.getInstance().getUnclosedFileReaderMap().size()); - List targetResources = tsFileManager.getTsFileList(true); + List targetResources = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); validateSeqFiles(true); assertEquals( @@ -308,8 +308,8 @@ public void testSeqInnerSpaceCompactionWithDifferentTimeseries() throws Exceptio path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); int count = 0; while (tsBlockReader.hasNextBatch()) { @@ -411,7 +411,7 @@ public void testSeqInnerSpaceCompactionWithFileTimeIndex() throws Exception { Assert.assertTrue(task.start()); Assert.assertEquals(0, FileReaderManager.getInstance().getClosedFileReaderMap().size()); Assert.assertEquals(0, FileReaderManager.getInstance().getUnclosedFileReaderMap().size()); - List targetResources = tsFileManager.getTsFileList(true); + List targetResources = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); validateSeqFiles(true); assertEquals( @@ -479,8 +479,8 @@ public void testSeqInnerSpaceCompactionWithFileTimeIndex() throws Exception { path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); int count = 0; while (tsBlockReader.hasNextBatch()) { @@ -555,7 +555,7 @@ public void testUnSeqInnerSpaceCompactionWithSameTimeseries() throws Exception { Assert.assertTrue(task.start()); Assert.assertEquals(0, FileReaderManager.getInstance().getClosedFileReaderMap().size()); Assert.assertEquals(0, FileReaderManager.getInstance().getUnclosedFileReaderMap().size()); - List targetResources = tsFileManager.getTsFileList(false); + List targetResources = tsFileManager.getTsFileList(false, COMPACTION_TEST_SG); validateSeqFiles(false); for (int i = 0; i < 2; i++) { @@ -570,8 +570,8 @@ public void testUnSeqInnerSpaceCompactionWithSameTimeseries() throws Exception { path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); int count = 0; while (tsBlockReader.hasNextBatch()) { @@ -679,8 +679,8 @@ public void testUnSeqInnerSpaceCompactionWithDifferentTimeseries() throws Except path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); int count = 0; while (tsBlockReader.hasNextBatch()) { @@ -818,8 +818,8 @@ public void testUnSeqInnerSpaceCompactionWithAllDataDeletedInTimeseries() throws path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); int count = 0; while (tsBlockReader.hasNextBatch()) { @@ -947,8 +947,8 @@ public void testUnSeqInnerSpaceCompactionWithAllDataDeletedInDevice() throws Exc path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); int count = 0; while (tsBlockReader.hasNextBatch()) { @@ -1062,8 +1062,8 @@ public void testUnSeqInnerSpaceCompactionWithAllDataDeletedInTargetFile() throws path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); int count = 0; while (tsBlockReader.hasNextBatch()) { @@ -1134,7 +1134,7 @@ public void testAlignedSeqInnerSpaceCompactionWithSameTimeseries() throws Except Assert.assertTrue(task.start()); Assert.assertEquals(0, FileReaderManager.getInstance().getClosedFileReaderMap().size()); Assert.assertEquals(0, FileReaderManager.getInstance().getUnclosedFileReaderMap().size()); - List targetResources = tsFileManager.getTsFileList(true); + List targetResources = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); validateSeqFiles(true); for (int i = TsFileGeneratorUtils.getAlignDeviceOffset(); @@ -1154,8 +1154,8 @@ public void testAlignedSeqInnerSpaceCompactionWithSameTimeseries() throws Except path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); int count = 0; while (tsBlockReader.hasNextBatch()) { @@ -1269,8 +1269,8 @@ public void testAlignedSeqInnerSpaceCompactionWithDifferentTimeseriesAndEmptyPag path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); int count = 0; while (tsBlockReader.hasNextBatch()) { @@ -1394,8 +1394,8 @@ public void testAlignedSeqInnerSpaceCompactionWithDifferentTimeseriesAndEmptyChu path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), false); int count = 0; while (tsBlockReader.hasNextBatch()) { @@ -1524,8 +1524,8 @@ public void testAlignedUnSeqInnerSpaceCompactionWithEmptyChunkAndEmptyPage() thr path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); int count = 0; while (tsBlockReader.hasNextBatch()) { @@ -1704,8 +1704,8 @@ public void testAlignedUnSeqInnerSpaceCompactionWithAllDataDeletedInTimeseries() path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); int count = 0; while (tsBlockReader.hasNextBatch()) { @@ -1864,8 +1864,8 @@ public void testAlignedUnSeqInnerSpaceCompactionWithAllDataDeletedInDevice() thr path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); int count = 0; while (tsBlockReader.hasNextBatch()) { @@ -1976,8 +1976,8 @@ public void testAlignedUnSeqInnerSpaceCompactionWithSameTimeseries() throws Exce path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), - tsFileManager.getTsFileList(false), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG), true); int count = 0; while (tsBlockReader.hasNextBatch()) { @@ -2059,7 +2059,7 @@ public void testMergeAlignedSeriesTimeValuePairFromDifferentFiles() 0, tsFileManager, unseqResources, false, new FastCompactionPerformer(false), 0); Assert.assertTrue(task.start()); - TsFileResource targetResource = tsFileManager.getTsFileList(false).get(0); + TsFileResource targetResource = tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0); try (TsFileSequenceReader reader = new TsFileSequenceReader(targetResource.getTsFilePath())) { List chunkMetadataList = reader.getAlignedChunkMetadata( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastNonAlignedCrossCompactionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastNonAlignedCrossCompactionTest.java index 64d70d30cdb10..fe6de7b30f9b2 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastNonAlignedCrossCompactionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/FastNonAlignedCrossCompactionTest.java @@ -335,7 +335,7 @@ public void test1() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -642,7 +642,7 @@ public void test2() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -954,7 +954,7 @@ public void test3() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -1278,7 +1278,7 @@ public void test4() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -1717,7 +1717,7 @@ public void test5() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -2163,7 +2163,7 @@ public void test6() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -2622,7 +2622,7 @@ public void test7() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -3063,7 +3063,7 @@ public void test8() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -3505,7 +3505,7 @@ public void test9() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -3960,7 +3960,7 @@ public void test10() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -4475,7 +4475,7 @@ public void test11() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -4993,7 +4993,7 @@ public void test12() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -5511,7 +5511,7 @@ public void test13() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -6089,7 +6089,7 @@ public void test14() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -6604,7 +6604,7 @@ public void test15() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -6871,7 +6871,7 @@ public void test16() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -7208,7 +7208,7 @@ public void test17() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -7696,7 +7696,7 @@ public void test18() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -8138,7 +8138,7 @@ public void test19() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -8449,7 +8449,7 @@ public void test20() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -8622,7 +8622,7 @@ public void test21() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -8854,7 +8854,7 @@ public void test22() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); @@ -9124,7 +9124,7 @@ public void test23() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadChunkInnerCompactionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadChunkInnerCompactionTest.java index 972db81b20a23..7b738c1486ad6 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadChunkInnerCompactionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadChunkInnerCompactionTest.java @@ -447,7 +447,7 @@ public void testReadChunkPerformerWithEmptyTargetFile1() throws IOException { new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, new ReadChunkCompactionPerformer(), 0); Assert.assertTrue(task.start()); - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } @Test @@ -473,7 +473,7 @@ public void testReadChunkPerformerWithEmptyTargetFile2() throws IOException { new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, new ReadChunkCompactionPerformer(), 0); Assert.assertTrue(task.start()); - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } @Test @@ -531,7 +531,7 @@ public void testReadChunkPerformerWithEmptyTargetFile3() new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, new ReadChunkCompactionPerformer(), 0); Assert.assertTrue(task.start()); - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } @Test @@ -580,7 +580,7 @@ public void testReadChunkPerformerWithEmptyTargetFile4() new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, new ReadChunkCompactionPerformer(), 0); Assert.assertTrue(task.start()); - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } @Test @@ -629,7 +629,7 @@ public void testReadChunkPerformerWithEmptyTargetFile5() new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, new ReadChunkCompactionPerformer(), 0); Assert.assertTrue(task.start()); - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } @Test @@ -687,7 +687,7 @@ public void testReadChunkPerformerWithEmptyTargetFile6() new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, new ReadChunkCompactionPerformer(), 0); Assert.assertTrue(task.start()); - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } @Test @@ -715,7 +715,7 @@ public void testReadChunkPerformerWithNonEmptyTargetFile() throws IOException { new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, new ReadChunkCompactionPerformer(), 0); Assert.assertTrue(task.start()); - Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } @Test @@ -885,8 +885,8 @@ public void testCascadedDeletionDuringCompaction() throws IOException, Interrupt }) .start(); Assert.assertTrue(task.start()); - Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); - tsFileManager.getTsFileList(true).get(0).getExclusiveModFile(); + Assert.assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0).getExclusiveModFile(); Assert.assertEquals(1, FileMetrics.getInstance().getModFileNum()); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadPointAlignedCrossCompactionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadPointAlignedCrossCompactionTest.java index f96cdf85cda10..9451fd6971373 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadPointAlignedCrossCompactionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadPointAlignedCrossCompactionTest.java @@ -304,7 +304,7 @@ public void test1() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -609,7 +609,7 @@ public void test2() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -913,7 +913,7 @@ public void test3() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -1234,7 +1234,7 @@ public void test4() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -1670,7 +1670,7 @@ public void test5() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -2114,7 +2114,7 @@ public void test6() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -2571,7 +2571,7 @@ public void test7() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -3009,7 +3009,7 @@ public void test8() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -3448,7 +3448,7 @@ public void test9() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -3901,7 +3901,7 @@ public void test10() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -4413,7 +4413,7 @@ public void test11() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -4928,7 +4928,7 @@ public void test12() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -5443,7 +5443,7 @@ public void test13() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -6018,7 +6018,7 @@ public void test14() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -6530,7 +6530,7 @@ public void test15() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -6791,7 +6791,7 @@ public void test16() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -7122,7 +7122,7 @@ public void test17() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -7604,7 +7604,7 @@ public void test18() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -8043,7 +8043,7 @@ public void test19() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -8348,7 +8348,7 @@ public void test20() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -8515,7 +8515,7 @@ public void test21() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -8741,7 +8741,7 @@ public void test22() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -9012,7 +9012,7 @@ public void test23() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadPointNonAlignedCrossCompactionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadPointNonAlignedCrossCompactionTest.java index 289ff7756b8e9..295acfafd4d25 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadPointNonAlignedCrossCompactionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/ReadPointNonAlignedCrossCompactionTest.java @@ -330,7 +330,7 @@ public void test1() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -631,7 +631,7 @@ public void test2() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -931,7 +931,7 @@ public void test3() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -1249,7 +1249,7 @@ public void test4() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -1682,7 +1682,7 @@ public void test5() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -2122,7 +2122,7 @@ public void test6() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -2575,7 +2575,7 @@ public void test7() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -3010,7 +3010,7 @@ public void test8() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -3446,7 +3446,7 @@ public void test9() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -3895,7 +3895,7 @@ public void test10() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -4404,7 +4404,7 @@ public void test11() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -4916,7 +4916,7 @@ public void test12() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -5428,7 +5428,7 @@ public void test13() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -6000,7 +6000,7 @@ public void test14() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -6509,7 +6509,7 @@ public void test15() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -6770,7 +6770,7 @@ public void test16() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -7101,7 +7101,7 @@ public void test17() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -7583,7 +7583,7 @@ public void test18() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -8019,7 +8019,7 @@ public void test19() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -8324,7 +8324,7 @@ public void test20() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -8491,7 +8491,7 @@ public void test21() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -8717,7 +8717,7 @@ public void test22() throws IOException, IllegalPathException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); @@ -8981,7 +8981,7 @@ public void test23() throws MetadataException, IOException { new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeAlterTableTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeAlterTableTest.java index 24551efa8d2cf..201141e9f7d47 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeAlterTableTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeAlterTableTest.java @@ -102,7 +102,7 @@ public void testAlter() throws IOException, WriteProcessException { new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, getPerformer(performerType), 0); Assert.assertTrue(task.start()); - TsFileResource target = tsFileManager.getTsFileList(true).get(0); + TsFileResource target = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); TimeseriesMetadata timeseriesMetadata = getTimeseriesMetadata(target, tableDevice, "s1"); Assert.assertEquals(1L, timeseriesMetadata.getStatistics().getStartTime()); @@ -116,7 +116,7 @@ public void testCannotAlter() throws IOException, WriteProcessException { new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, getPerformer(performerType), 0); Assert.assertTrue(task.start()); - TsFileResource target = tsFileManager.getTsFileList(true).get(0); + TsFileResource target = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); TimeseriesMetadata timeseriesMetadata = getTimeseriesMetadata(target, tableDevice, "s1"); if (!reverse) { Assert.assertEquals(1L, timeseriesMetadata.getStatistics().getStartTime()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeAlterTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeAlterTest.java index 96ddb03396ceb..81d3b6af14eb0 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeAlterTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeAlterTest.java @@ -94,12 +94,21 @@ public void testCompactNonAlignedSeries() new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, getPerformer(performerType), 0); Assert.assertTrue(task.start()); - TsFileResourceUtils.validateTsFileDataCorrectness(tsFileManager.getTsFileList(true).get(0)); + TsFileResourceUtils.validateTsFileDataCorrectness( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0)); Assert.assertEquals( - 1, ((long) tsFileManager.getTsFileList(true).get(0).getStartTime(device).get())); + 1, + ((long) + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .getStartTime(device) + .get())); Assert.assertEquals( - 2, ((long) tsFileManager.getTsFileList(true).get(0).getEndTime(device).get())); - TsFileResource tsFileResource = tsFileManager.getTsFileList(true).get(0); + 2, + ((long) + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0).getEndTime(device).get())); + TsFileResource tsFileResource = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFileResource.getTsFile().getAbsolutePath()); TsFileReader readTsFile = new TsFileReader(reader)) { @@ -123,12 +132,21 @@ public void testCompactAlignedSeries() new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, getPerformer(performerType), 0); Assert.assertTrue(task.start()); - TsFileResourceUtils.validateTsFileDataCorrectness(tsFileManager.getTsFileList(true).get(0)); + TsFileResourceUtils.validateTsFileDataCorrectness( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0)); Assert.assertEquals( - 1, ((long) tsFileManager.getTsFileList(true).get(0).getStartTime(device).get())); + 1, + ((long) + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .getStartTime(device) + .get())); Assert.assertEquals( - 2, ((long) tsFileManager.getTsFileList(true).get(0).getEndTime(device).get())); - TsFileResource tsFileResource = tsFileManager.getTsFileList(true).get(0); + 2, + ((long) + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0).getEndTime(device).get())); + TsFileResource tsFileResource = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFileResource.getTsFile().getAbsolutePath()); TsFileReader readTsFile = new TsFileReader(reader)) { @@ -223,7 +241,7 @@ public void testAlterDataTypeWithAlignedSeriesWithTimeDeletion() new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, getPerformer(performerType), 0); Assert.assertTrue(task.start()); - TsFileResource target = tsFileManager.getTsFileList(true).get(0); + TsFileResource target = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); Assert.assertEquals(41L, (long) target.getStartTime(device).get()); Assert.assertEquals(200L, (long) target.getEndTime(device).get()); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeNotMatchAlterableDataTypeTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeNotMatchAlterableDataTypeTest.java index cac37c3f385f9..b658e1b8506d8 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeNotMatchAlterableDataTypeTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeNotMatchAlterableDataTypeTest.java @@ -89,11 +89,20 @@ public void testCompactNonAlignedSeries() new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, getPerformer(performerType), 0); Assert.assertTrue(task.start()); - TsFileResourceUtils.validateTsFileDataCorrectness(tsFileManager.getTsFileList(true).get(0)); + TsFileResourceUtils.validateTsFileDataCorrectness( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0)); Assert.assertEquals( - 1, ((long) tsFileManager.getTsFileList(true).get(0).getStartTime(device).get())); + 1, + ((long) + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .getStartTime(device) + .get())); Assert.assertEquals( - 2, ((long) tsFileManager.getTsFileList(true).get(0).getEndTime(device).get())); + 2, + ((long) + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0).getEndTime(device).get())); } @Test @@ -104,11 +113,20 @@ public void testCompactAlignedSeries() new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, getPerformer(performerType), 0); Assert.assertTrue(task.start()); - TsFileResourceUtils.validateTsFileDataCorrectness(tsFileManager.getTsFileList(true).get(0)); + TsFileResourceUtils.validateTsFileDataCorrectness( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0)); Assert.assertEquals( - 1, ((long) tsFileManager.getTsFileList(true).get(0).getStartTime(device).get())); + 1, + ((long) + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .getStartTime(device) + .get())); Assert.assertEquals( - 2, ((long) tsFileManager.getTsFileList(true).get(0).getEndTime(device).get())); + 2, + ((long) + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0).getEndTime(device).get())); } private void generateDataTypeNotMatchFilesWithNonAlignedSeries() diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeNotMatchTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeNotMatchTest.java index 3002ef8981aaf..678cc96f69ccc 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeNotMatchTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/alterDataType/CompactionDataTypeNotMatchTest.java @@ -89,11 +89,20 @@ public void testCompactNonAlignedSeries() new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, getPerformer(performerType), 0); Assert.assertTrue(task.start()); - TsFileResourceUtils.validateTsFileDataCorrectness(tsFileManager.getTsFileList(true).get(0)); + TsFileResourceUtils.validateTsFileDataCorrectness( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0)); Assert.assertEquals( - 2, ((long) tsFileManager.getTsFileList(true).get(0).getStartTime(device).get())); + 2, + ((long) + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .getStartTime(device) + .get())); Assert.assertEquals( - 2, ((long) tsFileManager.getTsFileList(true).get(0).getEndTime(device).get())); + 2, + ((long) + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0).getEndTime(device).get())); } @Test @@ -112,11 +121,20 @@ public void testCompactNonAlignedSeries2() new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, getPerformer(performerType), 0); Assert.assertTrue(task.start()); - TsFileResourceUtils.validateTsFileDataCorrectness(tsFileManager.getTsFileList(true).get(0)); + TsFileResourceUtils.validateTsFileDataCorrectness( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0)); Assert.assertEquals( - 1, ((long) tsFileManager.getTsFileList(true).get(0).getStartTime(device).get())); + 1, + ((long) + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .getStartTime(device) + .get())); Assert.assertEquals( - 1, ((long) tsFileManager.getTsFileList(true).get(0).getEndTime(device).get())); + 1, + ((long) + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0).getEndTime(device).get())); } @Test @@ -127,11 +145,20 @@ public void testCompactAlignedSeries() new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, getPerformer(performerType), 0); Assert.assertTrue(task.start()); - TsFileResourceUtils.validateTsFileDataCorrectness(tsFileManager.getTsFileList(true).get(0)); + TsFileResourceUtils.validateTsFileDataCorrectness( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0)); Assert.assertEquals( - 2, ((long) tsFileManager.getTsFileList(true).get(0).getStartTime(device).get())); + 2, + ((long) + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .getStartTime(device) + .get())); Assert.assertEquals( - 2, ((long) tsFileManager.getTsFileList(true).get(0).getEndTime(device).get())); + 2, + ((long) + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0).getEndTime(device).get())); } @Test @@ -154,11 +181,20 @@ public void testCompactAlignedSeries2() new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, getPerformer(performerType), 0); Assert.assertTrue(task.start()); - TsFileResourceUtils.validateTsFileDataCorrectness(tsFileManager.getTsFileList(true).get(0)); + TsFileResourceUtils.validateTsFileDataCorrectness( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0)); Assert.assertEquals( - 1, ((long) tsFileManager.getTsFileList(true).get(0).getStartTime(device).get())); + 1, + ((long) + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .getStartTime(device) + .get())); Assert.assertEquals( - 1, ((long) tsFileManager.getTsFileList(true).get(0).getEndTime(device).get())); + 1, + ((long) + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0).getEndTime(device).get())); } private void generateDataTypeNotMatchFilesWithNonAlignedSeries() diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithFastPerformerValidationTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithFastPerformerValidationTest.java index 644ddf77e1f24..1c0b9c09df496 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithFastPerformerValidationTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/CrossSpaceCompactionWithFastPerformerValidationTest.java @@ -2152,7 +2152,7 @@ public void testTsFileValidationWithFileTimeIndex() // set the end time of d1 in the first seq file to 1100 tsFileManager - .getTsFileList(true) + .getTsFileList(true, COMPACTION_TEST_SG) .get(0) .updateEndTime( IDeviceID.Factory.DEFAULT_FACTORY.create(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1"), @@ -2160,14 +2160,14 @@ public void testTsFileValidationWithFileTimeIndex() // set the end time of d1 in the second seq file to 1200 tsFileManager - .getTsFileList(true) + .getTsFileList(true, COMPACTION_TEST_SG) .get(1) .updateStartTime( IDeviceID.Factory.DEFAULT_FACTORY.create(COMPACTION_TEST_SG + PATH_SEPARATOR + "d1"), 1200L); for (int i = 1; i < seqResources.size(); i++) { - tsFileManager.getTsFileList(true).get(i).degradeTimeIndex(); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(i).degradeTimeIndex(); } // meet overlap files @@ -2175,10 +2175,10 @@ public void testTsFileValidationWithFileTimeIndex() TsFileResourceUtils.validateTsFileResourcesHasNoOverlap( tsFileManager.getOrCreateSequenceListByTimePartition(0).getArrayList())); - tsFileManager.getTsFileList(true).get(0).deserialize(); - tsFileManager.getTsFileList(true).get(1).deserialize(); - tsFileManager.getTsFileList(true).get(0).degradeTimeIndex(); - tsFileManager.getTsFileList(true).get(1).degradeTimeIndex(); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0).deserialize(); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1).deserialize(); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0).degradeTimeIndex(); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1).degradeTimeIndex(); Assert.assertTrue( TsFileResourceUtils.validateTsFileResourcesHasNoOverlap( tsFileManager.getOrCreateSequenceListByTimePartition(0).getArrayList())); @@ -2282,11 +2282,14 @@ public void testCompactionSchedule() throws Exception { Assert.assertEquals(2, pairs.get(0).getSeqFiles().size()); Assert.assertEquals(1, pairs.get(0).getUnseqFiles().size()); Assert.assertEquals( - tsFileManager.getTsFileList(true).get(4), pairs.get(0).getSeqFiles().get(0)); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(4), + pairs.get(0).getSeqFiles().get(0)); Assert.assertEquals( - tsFileManager.getTsFileList(true).get(5), pairs.get(0).getSeqFiles().get(1)); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(5), + pairs.get(0).getSeqFiles().get(1)); Assert.assertEquals( - tsFileManager.getTsFileList(false).get(0), pairs.get(0).getUnseqFiles().get(0)); + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0), + pairs.get(0).getUnseqFiles().get(0)); // target file of first compaction task can be selected to participate in another inner // compaction task diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionSelectorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionSelectorTest.java index 5478d05004dff..da231ce6786aa 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionSelectorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionSelectorTest.java @@ -173,7 +173,8 @@ public void testInsertionCompactionWithCachedDeviceInfoAndUnclosedResource() CompactionScheduler.tryToSubmitInsertionCompactionTask(tsFileManager, 0, phaser, context); Assert.assertEquals(0, submitTaskNum); Assert.assertTrue( - TsFileResourceUtils.validateTsFileResourcesHasNoOverlap(tsFileManager.getTsFileList(true))); + TsFileResourceUtils.validateTsFileResourcesHasNoOverlap( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG))); } @Test @@ -1090,23 +1091,28 @@ public void testInsertionSelectorWithNoSeqFiles() throws MergeException, IOExcep InsertionCrossCompactionTaskResource taskResource; int i = 1; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { taskResource = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate( - tsFileManager.getTsFileList(true), tsFileManager.getTsFileList(false))); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG))); Assert.assertTrue(taskResource.isValid()); Assert.assertEquals( - tsFileManager.getTsFileList(false).get(0), taskResource.firstUnSeqFileInParitition); + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0), + taskResource.firstUnSeqFileInParitition); Assert.assertEquals( - tsFileManager.getTsFileList(false).get(0), taskResource.toInsertUnSeqFile); + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0), + taskResource.toInsertUnSeqFile); Assert.assertEquals(null, taskResource.nextSeqFile); if (i == 1) { Assert.assertEquals(null, taskResource.prevSeqFile); } else if (i == 2) { - Assert.assertEquals(tsFileManager.getTsFileList(true).get(0), taskResource.prevSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0), taskResource.prevSeqFile); } else { - Assert.assertEquals(tsFileManager.getTsFileList(true).get(1), taskResource.prevSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1), taskResource.prevSeqFile); } InsertionCrossSpaceCompactionTask task = new InsertionCrossSpaceCompactionTask( @@ -1118,8 +1124,8 @@ public void testInsertionSelectorWithNoSeqFiles() throws MergeException, IOExcep task.start(); i++; } - Assert.assertEquals(3, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(0, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(3, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); } @Test @@ -1304,10 +1310,15 @@ public void testInsertionIntoCompactingSeqFiles() throws IOException, MergeExcep InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); - Assert.assertEquals(tsFileManager.getTsFileList(false).get(0), task.toInsertUnSeqFile); - Assert.assertEquals(tsFileManager.getTsFileList(true).get(1), task.prevSeqFile); - Assert.assertEquals(tsFileManager.getTsFileList(true).get(2), task.nextSeqFile); - Assert.assertEquals(tsFileManager.getTsFileList(false).get(0), task.firstUnSeqFileInParitition); + Assert.assertEquals( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0), task.toInsertUnSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1), task.prevSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(2), task.nextSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0), + task.firstUnSeqFileInParitition); // seq file 1 ~ 3 is compaction candidate for (int i = 0; i < 3; i++) { @@ -1316,10 +1327,15 @@ public void testInsertionIntoCompactingSeqFiles() throws IOException, MergeExcep task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); - Assert.assertEquals(tsFileManager.getTsFileList(false).get(0), task.toInsertUnSeqFile); - Assert.assertEquals(tsFileManager.getTsFileList(true).get(3), task.prevSeqFile); - Assert.assertEquals(tsFileManager.getTsFileList(true).get(4), task.nextSeqFile); - Assert.assertEquals(tsFileManager.getTsFileList(false).get(0), task.firstUnSeqFileInParitition); + Assert.assertEquals( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0), task.toInsertUnSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(3), task.prevSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(4), task.nextSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0), + task.firstUnSeqFileInParitition); // all seq file are compacting for (TsFileResource resource : seqResources) { @@ -1579,23 +1595,28 @@ public void testInsertionSelectorWithNoSeqFilesAndFileTimeIndex() InsertionCrossCompactionTaskResource taskResource; int i = 1; - while (tsFileManager.getTsFileList(false).size() > 0) { + while (tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size() > 0) { taskResource = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate( - tsFileManager.getTsFileList(true), tsFileManager.getTsFileList(false))); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG))); Assert.assertTrue(taskResource.isValid()); Assert.assertEquals( - tsFileManager.getTsFileList(false).get(0), taskResource.firstUnSeqFileInParitition); + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0), + taskResource.firstUnSeqFileInParitition); Assert.assertEquals( - tsFileManager.getTsFileList(false).get(0), taskResource.toInsertUnSeqFile); + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0), + taskResource.toInsertUnSeqFile); Assert.assertEquals(null, taskResource.nextSeqFile); if (i == 1) { Assert.assertEquals(null, taskResource.prevSeqFile); } else if (i == 2) { - Assert.assertEquals(tsFileManager.getTsFileList(true).get(0), taskResource.prevSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0), taskResource.prevSeqFile); } else { - Assert.assertEquals(tsFileManager.getTsFileList(true).get(1), taskResource.prevSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1), taskResource.prevSeqFile); } InsertionCrossSpaceCompactionTask task = new InsertionCrossSpaceCompactionTask( @@ -1607,8 +1628,8 @@ public void testInsertionSelectorWithNoSeqFilesAndFileTimeIndex() task.start(); i++; } - Assert.assertEquals(3, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(0, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(3, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); } @Test @@ -1812,10 +1833,15 @@ public void testInsertionIntoCompactingSeqFilesAndFileTimeIndex() InsertionCrossCompactionTaskResource task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); - Assert.assertEquals(tsFileManager.getTsFileList(false).get(0), task.toInsertUnSeqFile); - Assert.assertEquals(tsFileManager.getTsFileList(true).get(1), task.prevSeqFile); - Assert.assertEquals(tsFileManager.getTsFileList(true).get(2), task.nextSeqFile); - Assert.assertEquals(tsFileManager.getTsFileList(false).get(0), task.firstUnSeqFileInParitition); + Assert.assertEquals( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0), task.toInsertUnSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1), task.prevSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(2), task.nextSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0), + task.firstUnSeqFileInParitition); // seq file 1 ~ 3 is compaction candidate for (int i = 0; i < 3; i++) { @@ -1824,10 +1850,15 @@ public void testInsertionIntoCompactingSeqFilesAndFileTimeIndex() task = selector.selectOneInsertionTask( new CrossSpaceCompactionCandidate(seqResources, unseqResources)); - Assert.assertEquals(tsFileManager.getTsFileList(false).get(0), task.toInsertUnSeqFile); - Assert.assertEquals(tsFileManager.getTsFileList(true).get(3), task.prevSeqFile); - Assert.assertEquals(tsFileManager.getTsFileList(true).get(4), task.nextSeqFile); - Assert.assertEquals(tsFileManager.getTsFileList(false).get(0), task.firstUnSeqFileInParitition); + Assert.assertEquals( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0), task.toInsertUnSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(3), task.prevSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(4), task.nextSeqFile); + Assert.assertEquals( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0), + task.firstUnSeqFileInParitition); // all seq file are compacting for (TsFileResource resource : seqResources) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionTest.java index 0e0516a46c067..2d1e93f87f228 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/InsertionCrossSpaceCompactionTest.java @@ -123,13 +123,13 @@ public void test1() throws IOException, InterruptedException { task.setSourceFilesToCompactionCandidate(); candidateCompactionTaskQueue.put(task); Assert.assertTrue(worker.processOneCompactionTask(candidateCompactionTaskQueue.take())); - Assert.assertEquals(3, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(seqResource1, tsFileManager.getTsFileList(true).get(0)); - Assert.assertEquals(seqResource2, tsFileManager.getTsFileList(true).get(2)); - TsFileResource targetFile = tsFileManager.getTsFileList(true).get(1); + Assert.assertEquals(3, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(seqResource1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0)); + Assert.assertEquals(seqResource2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(2)); + TsFileResource targetFile = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1); long timestamp = TsFileNameGenerator.getTsFileName(targetFile.getTsFile().getName()).getTime(); Assert.assertEquals(2, timestamp); - Assert.assertTrue(tsFileManager.getTsFileList(false).isEmpty()); + Assert.assertTrue(tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).isEmpty()); Assert.assertEquals( tsFileManager.size(true) + tsFileManager.size(false), TsFileResourceManager.getInstance().getPriorityQueueSize()); @@ -168,10 +168,10 @@ public void test2() throws IOException, InterruptedException { task.setSourceFilesToCompactionCandidate(); candidateCompactionTaskQueue.put(task); Assert.assertTrue(worker.processOneCompactionTask(candidateCompactionTaskQueue.take())); - Assert.assertEquals(2, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(seqResource1, tsFileManager.getTsFileList(true).get(0)); - Assert.assertTrue(tsFileManager.getTsFileList(false).isEmpty()); - TsFileResource targetFile = tsFileManager.getTsFileList(true).get(1); + Assert.assertEquals(2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(seqResource1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0)); + Assert.assertTrue(tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).isEmpty()); + TsFileResource targetFile = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1); long timestamp = TsFileNameGenerator.getTsFileName(targetFile.getTsFile().getName()).getTime(); Assert.assertEquals(2, timestamp); Assert.assertEquals( @@ -212,10 +212,10 @@ public void test3() throws IOException, InterruptedException { task.setSourceFilesToCompactionCandidate(); candidateCompactionTaskQueue.put(task); Assert.assertTrue(worker.processOneCompactionTask(candidateCompactionTaskQueue.take())); - Assert.assertEquals(2, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(seqResource1, tsFileManager.getTsFileList(true).get(1)); - Assert.assertTrue(tsFileManager.getTsFileList(false).isEmpty()); - TsFileResource targetFile = tsFileManager.getTsFileList(true).get(0); + Assert.assertEquals(2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(seqResource1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1)); + Assert.assertTrue(tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).isEmpty()); + TsFileResource targetFile = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); long timestamp = TsFileNameGenerator.getTsFileName(targetFile.getTsFile().getName()).getTime(); Assert.assertEquals(0, timestamp); Assert.assertEquals( @@ -261,13 +261,13 @@ public void test4() throws IOException, InterruptedException { task.setSourceFilesToCompactionCandidate(); candidateCompactionTaskQueue.put(task); Assert.assertTrue(worker.processOneCompactionTask(candidateCompactionTaskQueue.take())); - Assert.assertEquals(3, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(seqResource1, tsFileManager.getTsFileList(true).get(0)); - Assert.assertEquals(seqResource2, tsFileManager.getTsFileList(true).get(2)); - TsFileResource targetFile = tsFileManager.getTsFileList(true).get(1); + Assert.assertEquals(3, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(seqResource1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0)); + Assert.assertEquals(seqResource2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(2)); + TsFileResource targetFile = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1); long timestamp = TsFileNameGenerator.getTsFileName(targetFile.getTsFile().getName()).getTime(); Assert.assertEquals(4, timestamp); - Assert.assertTrue(tsFileManager.getTsFileList(false).isEmpty()); + Assert.assertTrue(tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).isEmpty()); Assert.assertEquals( tsFileManager.size(true) + tsFileManager.size(false), TsFileResourceManager.getInstance().getPriorityQueueSize()); @@ -301,9 +301,9 @@ public void test5() throws IOException, InterruptedException { task.setSourceFilesToCompactionCandidate(); candidateCompactionTaskQueue.put(task); Assert.assertTrue(worker.processOneCompactionTask(candidateCompactionTaskQueue.take())); - Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); - Assert.assertTrue(tsFileManager.getTsFileList(false).isEmpty()); - TsFileResource targetFile = tsFileManager.getTsFileList(true).get(0); + Assert.assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertTrue(tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).isEmpty()); + TsFileResource targetFile = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); long timestamp = TsFileNameGenerator.getTsFileName(targetFile.getTsFile().getName()).getTime(); Assert.assertEquals(2, timestamp); Assert.assertEquals( @@ -349,13 +349,13 @@ public void testInsertionCompactionSchedule() throws IOException, InterruptedExc tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); Assert.assertEquals(2, dataRegion.executeInsertionCompaction()); - Assert.assertEquals(4, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(4, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); - TsFileResource targetFile1 = tsFileManager.getTsFileList(true).get(1); + TsFileResource targetFile1 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1); long timestamp = TsFileNameGenerator.getTsFileName(targetFile1.getTsFile().getName()).getTime(); Assert.assertEquals(4, timestamp); - TsFileResource targetFile2 = tsFileManager.getTsFileList(true).get(2); + TsFileResource targetFile2 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(2); timestamp = TsFileNameGenerator.getTsFileName(targetFile2.getTsFile().getName()).getTime(); Assert.assertEquals(6, timestamp); Assert.assertEquals( @@ -388,15 +388,15 @@ public void testInsertionCompactionScheduleWithEmptySeqSpace() tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); Assert.assertEquals(3, dataRegion.executeInsertionCompaction()); - Assert.assertEquals(3, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(3, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); - TsFileResource targetFile1 = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetFile1 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); long timestamp = TsFileNameGenerator.getTsFileName(targetFile1.getTsFile().getName()).getTime(); Assert.assertEquals(2, timestamp); - TsFileResource targetFile2 = tsFileManager.getTsFileList(true).get(1); + TsFileResource targetFile2 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1); timestamp = TsFileNameGenerator.getTsFileName(targetFile2.getTsFile().getName()).getTime(); Assert.assertEquals(3, timestamp); - TsFileResource targetFile3 = tsFileManager.getTsFileList(true).get(2); + TsFileResource targetFile3 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(2); timestamp = TsFileNameGenerator.getTsFileName(targetFile3.getTsFile().getName()).getTime(); Assert.assertEquals(4, timestamp); Assert.assertEquals( @@ -439,12 +439,12 @@ public void testInsertionCompactionScheduleWithEmptySeqSpace2() tsFileManager.addAll(seqResources, true); tsFileManager.addAll(unseqResources, false); Assert.assertEquals(2, dataRegion.executeInsertionCompaction()); - Assert.assertEquals(2, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); - TsFileResource targetFile1 = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetFile1 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); long timestamp = TsFileNameGenerator.getTsFileName(targetFile1.getTsFile().getName()).getTime(); Assert.assertEquals(1, timestamp); - TsFileResource targetFile2 = tsFileManager.getTsFileList(true).get(1); + TsFileResource targetFile2 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1); timestamp = TsFileNameGenerator.getTsFileName(targetFile2.getTsFile().getName()).getTime(); Assert.assertEquals(2, timestamp); Assert.assertEquals( @@ -487,7 +487,7 @@ public void testInsertionCompactionScheduleWithMultiTimePartitions1() tsFileManager.getOrCreateUnsequenceListByTimePartition(2808).keepOrderInsert(unseqResource3); Assert.assertEquals(3, dataRegion.executeInsertionCompaction()); - Assert.assertEquals(3, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(3, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); Assert.assertEquals(2, tsFileManager.getOrCreateSequenceListByTimePartition(0).size()); Assert.assertEquals(1, tsFileManager.getOrCreateSequenceListByTimePartition(2808).size()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java index ead500524e550..8b14a26cc088f 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithFastPerformerTest.java @@ -276,7 +276,7 @@ public void testAlignedCrossSpaceCompactionWithAllDataDeletedInTimeseries() thro path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); int count = 0; @@ -506,7 +506,7 @@ public void testAlignedCrossSpaceCompactionWithAllDataDeletedInOneTargetFile() t path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); int count = 0; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java index d65400144137a..84bbe5f668871 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/cross/RewriteCrossSpaceCompactionWithReadPointPerformerTest.java @@ -271,7 +271,7 @@ public void testAlignedCrossSpaceCompactionWithAllDataDeletedInTimeseries() thro path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); int count = 0; @@ -501,7 +501,7 @@ public void testAlignedCrossSpaceCompactionWithAllDataDeletedInOneTargetFile() t path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); int count = 0; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerCompactionMoreDataTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerCompactionMoreDataTest.java index 09e593766c86d..ef7b54182af1d 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerCompactionMoreDataTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerCompactionMoreDataTest.java @@ -171,7 +171,7 @@ public void testSensorWithTwoOrThreeNode() new SeriesDataBlockReader( path, TEST_QUERY_FI_CONTEXT, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerCompactionSchedulerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerCompactionSchedulerTest.java index 340270514713f..993ebfb747957 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerCompactionSchedulerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/InnerCompactionSchedulerTest.java @@ -103,7 +103,7 @@ public void testFileSelector1() } catch (Exception e) { } - Assert.assertEquals(3, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(3, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } @Test @@ -134,7 +134,7 @@ public void testFileSelector2() e.printStackTrace(); } } - Assert.assertEquals(4, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(4, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } @Test @@ -163,6 +163,6 @@ public void testFileSelectorWithUnclosedFile() e.printStackTrace(); } } - Assert.assertEquals(4, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(4, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/sizetiered/NewSizeTieredCompactionSelectorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/sizetiered/NewSizeTieredCompactionSelectorTest.java index 2ede5aa26f6cc..83a5394bb35bc 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/sizetiered/NewSizeTieredCompactionSelectorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/sizetiered/NewSizeTieredCompactionSelectorTest.java @@ -126,7 +126,7 @@ public void testSelectAllFiles() throws IOException { Assert.assertTrue(task.start()); Assert.assertEquals(10, task.getSelectedTsFileResourceList().size()); Assert.assertEquals(10, task.getAllSourceTsFiles().size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } @Test @@ -152,12 +152,12 @@ public void testSelectWithFileNumLimit() throws IOException { Assert.assertTrue(task1.start()); Assert.assertEquals(8, task1.getSelectedTsFileResourceList().size()); Assert.assertEquals(8, task1.getAllSourceTsFiles().size()); - Assert.assertEquals(3, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(3, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); InnerSpaceCompactionTask task2 = innerSpaceCompactionTasks.get(1); Assert.assertEquals(2, task2.getSelectedTsFileResourceList().size()); Assert.assertEquals(2, task2.getAllSourceTsFiles().size()); Assert.assertTrue(task2.start()); - Assert.assertEquals(2, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } @Test @@ -224,7 +224,8 @@ public void testSkipSomeFilesAndRenamePreviousFiles() throws IOException { Assert.assertTrue(task.start()); Assert.assertEquals(2, task.getSelectedTsFileResourceList().size()); Assert.assertEquals(10, task.getAllSourceTsFiles().size()); - List filesAfterCompaction = tsFileManager.getTsFileList(true); + List filesAfterCompaction = + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); Assert.assertEquals(9, filesAfterCompaction.size()); Assert.assertEquals(0, filesAfterCompaction.get(0).getTsFileID().fileVersion); Assert.assertEquals(101L, filesAfterCompaction.get(0).getFileStartTime()); @@ -266,7 +267,8 @@ public void testSkipSomeFilesAndRenamePreviousFiles2() throws IOException { // select resource1, resource3, resource5 Assert.assertEquals(3, task.getSelectedTsFileResourceList().size()); Assert.assertEquals(5, task.getAllSourceTsFiles().size()); - List filesAfterCompaction = tsFileManager.getTsFileList(true); + List filesAfterCompaction = + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); Assert.assertEquals(5, filesAfterCompaction.size()); Assert.assertEquals( 5, filesAfterCompaction.get(filesAfterCompaction.size() - 1).getTsFileID().fileVersion); @@ -319,7 +321,8 @@ public void testSkipSomeFiles() throws IOException { Assert.assertTrue(task.start()); Assert.assertEquals(3, task.getSelectedTsFileResourceList().size()); Assert.assertEquals(5, task.getAllSourceTsFiles().size()); - List filesAfterCompaction = tsFileManager.getTsFileList(true); + List filesAfterCompaction = + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); Assert.assertEquals(3, filesAfterCompaction.size()); Assert.assertEquals( 5, filesAfterCompaction.get(filesAfterCompaction.size() - 1).getTsFileID().fileVersion); @@ -370,7 +373,8 @@ public void testSkipSomeFilesAndRenamePreviousFilesWithCompactionMods() Assert.assertTrue(task.start()); Assert.assertEquals(2, task.getSelectedTsFileResourceList().size()); Assert.assertEquals(10, task.getAllSourceTsFiles().size()); - List filesAfterCompaction = tsFileManager.getTsFileList(true); + List filesAfterCompaction = + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); Assert.assertEquals(9, filesAfterCompaction.size()); Assert.assertEquals(0, filesAfterCompaction.get(0).getTsFileID().fileVersion); Assert.assertEquals(101L, filesAfterCompaction.get(0).getFileStartTime()); @@ -417,7 +421,8 @@ public void testAllTargetFilesEmpty() throws IOException, IllegalPathException { Assert.assertTrue(task.start()); Assert.assertEquals(2, task.getSelectedTsFileResourceList().size()); Assert.assertEquals(2, task.getAllSourceTsFiles().size()); - List filesAfterCompaction = tsFileManager.getTsFileList(true); + List filesAfterCompaction = + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); Assert.assertEquals(0, filesAfterCompaction.size()); } @@ -471,7 +476,8 @@ public void testAllTargetFilesEmptyWithSkippedSourceFiles() Assert.assertTrue(task.start()); Assert.assertEquals(3, task.getSelectedTsFileResourceList().size()); Assert.assertEquals(5, task.getAllSourceTsFiles().size()); - List filesAfterCompaction = tsFileManager.getTsFileList(true); + List filesAfterCompaction = + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); Assert.assertEquals(2, filesAfterCompaction.size()); Assert.assertEquals( 4, filesAfterCompaction.get(filesAfterCompaction.size() - 1).getTsFileID().fileVersion); @@ -503,7 +509,8 @@ public void testSelectFilesInOtherLevel() throws IOException { Assert.assertTrue(task.start()); Assert.assertEquals(6, task.getSelectedTsFileResourceList().size()); Assert.assertEquals(6, task.getAllSourceTsFiles().size()); - List filesAfterCompaction = tsFileManager.getTsFileList(true); + List filesAfterCompaction = + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); Assert.assertEquals(5, filesAfterCompaction.size()); Assert.assertEquals(0, filesAfterCompaction.get(0).getTsFileID().fileVersion); Assert.assertEquals(1L, filesAfterCompaction.get(0).getFileStartTime()); @@ -540,7 +547,8 @@ public void testSkipToPreviousIndexAndSelectSkippedFiles() throws IOException { Assert.assertTrue(task.start()); Assert.assertEquals(5, task.getSelectedTsFileResourceList().size()); Assert.assertEquals(5, task.getAllSourceTsFiles().size()); - List filesAfterCompaction = tsFileManager.getTsFileList(true); + List filesAfterCompaction = + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); Assert.assertEquals(6, filesAfterCompaction.size()); Assert.assertEquals(5, filesAfterCompaction.get(5).getTsFileID().fileVersion); Assert.assertEquals(501L, filesAfterCompaction.get(5).getFileStartTime()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/sizetiered/SizeTieredCompactionRecoverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/sizetiered/SizeTieredCompactionRecoverTest.java index cc9e713c94c6d..1652d48bd0625 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/sizetiered/SizeTieredCompactionRecoverTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/inner/sizetiered/SizeTieredCompactionRecoverTest.java @@ -109,7 +109,7 @@ public void testCompactionRecoverWithUncompletedTargetFileAndLog() throws Except path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); int count = 0; @@ -197,7 +197,7 @@ public void testCompactionRecoverWithUncompletedTargetFileAndLog() throws Except path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); count = 0; @@ -229,7 +229,7 @@ public void testRecoverWithAllSourceFilesExisted() throws Exception { path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); int count = 0; @@ -305,7 +305,7 @@ public void testRecoverWithAllSourceFilesExisted() throws Exception { path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); count = 0; @@ -337,7 +337,7 @@ public void testRecoverWithAllSourceFilesExistedAndTargetFileNotExist() throws E path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); int count = 0; @@ -415,7 +415,7 @@ public void testRecoverWithAllSourceFilesExistedAndTargetFileNotExist() throws E path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); count = 0; @@ -445,7 +445,7 @@ public void testRecoverWithoutAllSourceFilesExisted() throws Exception { path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); int count = 0; @@ -518,7 +518,7 @@ public void testRecoverWithoutAllSourceFilesExisted() throws Exception { path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true).subList(3, 6), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).subList(3, 6), new ArrayList<>(), true); count = 0; @@ -772,7 +772,7 @@ public void testRecoverCompleteTargetFileAndCompactionLog() throws Exception { path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); int count = 0; @@ -838,7 +838,7 @@ public void testRecoverCompleteTargetFileAndCompactionLog() throws Exception { path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true).subList(0, 5), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).subList(0, 5), new ArrayList<>(), true); count = 0; @@ -870,7 +870,7 @@ public void testCompactionRecoverWithCompletedTargetFileAndLog() throws Exceptio path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); int count = 0; @@ -929,7 +929,7 @@ public void testCompactionRecoverWithCompletedTargetFileAndLog() throws Exceptio path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); count = 0; @@ -962,7 +962,7 @@ public void testCompactionRecoverWithCompletedTargetFile() throws Exception { path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); int count = 0; @@ -1017,13 +1017,13 @@ public void testCompactionRecoverWithCompletedTargetFile() throws Exception { deviceIds[0] + TsFileConstant.PATH_SEPARATOR + measurementSchemas[0].getMeasurementName()); - logger.warn("TsFiles in list is {}", tsFileManager.getTsFileList(true)); + logger.warn("TsFiles in list is {}", tsFileManager.getTsFileList(true, COMPACTION_TEST_SG)); tsFilesReader = new SeriesDataBlockReader( path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); count = 0; @@ -1066,7 +1066,7 @@ public void testCompactionMergeRecoverMergeStartSourceLog() path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); int count = 0; @@ -1107,7 +1107,7 @@ public void testCompactionMergeRecoverMergeStartSequenceLog() path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); int count = 0; @@ -1162,7 +1162,7 @@ public void testCompactionMergeRecoverMergeStart() throws IOException, MetadataE path, FragmentInstanceContext.createFragmentInstanceContextForCompaction( EnvironmentUtils.TEST_QUERY_CONTEXT.getQueryId()), - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), new ArrayList<>(), true); int count = 0; diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairUnsortedFileCompactionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairUnsortedFileCompactionTest.java index 7774bbb609b3c..f222d2261723d 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairUnsortedFileCompactionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairUnsortedFileCompactionTest.java @@ -124,7 +124,8 @@ public void testMoveOldVersionResourceFile() throws IOException { Assert.assertTrue(task.start()); TsFileResource resource2 = - new TsFileResource(tsFileManager.getTsFileList(false).get(0).getTsFile()); + new TsFileResource( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0).getTsFile()); resource2.deserialize(); Assert.assertEquals( ArrayDeviceTimeIndex.ARRAY_DEVICE_TIME_INDEX_TYPE, resource2.getTimeIndexType()); @@ -148,11 +149,11 @@ public void testRepairUnsortedDataBetweenPageWithNonAlignedSeries() throws IOExc RepairUnsortedFileCompactionTask task = new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0); task.start(); - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); Assert.assertTrue( TsFileResourceUtils.validateTsFileDataCorrectness( - tsFileManager.getTsFileList(false).get(0))); + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0))); } @Test @@ -173,11 +174,11 @@ public void testRepairUnsortedDataBetweenPageWithAlignedSeries() throws IOExcept RepairUnsortedFileCompactionTask task = new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0); task.start(); - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); Assert.assertTrue( TsFileResourceUtils.validateTsFileDataCorrectness( - tsFileManager.getTsFileList(false).get(0))); + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0))); } @Test @@ -202,11 +203,11 @@ public void testRepairUnsortedDataInOnePageWithNonAlignedSeries() throws IOExcep RepairUnsortedFileCompactionTask task = new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0); task.start(); - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); Assert.assertTrue( TsFileResourceUtils.validateTsFileDataCorrectness( - tsFileManager.getTsFileList(false).get(0))); + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0))); } @Test @@ -235,11 +236,11 @@ public void testRepairUnsortedDataInOnePageWithMultiNonAlignedSeries() throws IO RepairUnsortedFileCompactionTask task = new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0); task.start(); - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); Assert.assertTrue( TsFileResourceUtils.validateTsFileDataCorrectness( - tsFileManager.getTsFileList(false).get(0))); + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0))); } @Test @@ -264,11 +265,11 @@ public void testRepairUnsortedDataInOnePageWithUnseqFile() throws IOException { RepairUnsortedFileCompactionTask task = new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0); task.start(); - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); Assert.assertTrue( TsFileResourceUtils.validateTsFileDataCorrectness( - tsFileManager.getTsFileList(false).get(0))); + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0))); } @Test @@ -293,11 +294,11 @@ public void testRepairUnsortedDataInOnePageWithAlignedSeries() throws IOExceptio RepairUnsortedFileCompactionTask task = new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, resource.isSeq(), 0); task.start(); - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); Assert.assertTrue( TsFileResourceUtils.validateTsFileDataCorrectness( - tsFileManager.getTsFileList(false).get(0))); + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0))); Assert.assertTrue( TsFileResourceUtils.validateTsFileResourcesHasNoOverlap( tsFileManager.getOrCreateSequenceListByTimePartition(0))); @@ -344,7 +345,7 @@ public void testMarkFileAndRepairWithInnerSeqSpaceCompactionTask() 0); Assert.assertFalse(task.start()); - for (TsFileResource resource : tsFileManager.getTsFileList(true)) { + for (TsFileResource resource : tsFileManager.getTsFileList(true, COMPACTION_TEST_SG)) { Assert.assertTrue(resource.getTsFileRepairStatus().isRepairCompactionCandidate()); } @@ -368,9 +369,9 @@ public void testMarkFileAndRepairWithInnerSeqSpaceCompactionTask() throw new RuntimeException(e); } } - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(2, tsFileManager.getTsFileList(false).size()); - for (TsFileResource resource : tsFileManager.getTsFileList(false)) { + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); + for (TsFileResource resource : tsFileManager.getTsFileList(false, COMPACTION_TEST_SG)) { TsFileResourceUtils.validateTsFileDataCorrectness(resource); } Assert.assertTrue( @@ -419,7 +420,7 @@ public void testMarkFileAndRepairWithInnerUnSeqSpaceCompactionTask() 0); Assert.assertFalse(task.start()); - for (TsFileResource resource : tsFileManager.getTsFileList(true)) { + for (TsFileResource resource : tsFileManager.getTsFileList(true, COMPACTION_TEST_SG)) { Assert.assertTrue(resource.getTsFileRepairStatus().isRepairCompactionCandidate()); } @@ -443,9 +444,9 @@ public void testMarkFileAndRepairWithInnerUnSeqSpaceCompactionTask() throw new RuntimeException(e); } } - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(2, tsFileManager.getTsFileList(false).size()); - for (TsFileResource resource : tsFileManager.getTsFileList(false)) { + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); + for (TsFileResource resource : tsFileManager.getTsFileList(false, COMPACTION_TEST_SG)) { TsFileResourceUtils.validateTsFileDataCorrectness(resource); } Assert.assertTrue( @@ -495,7 +496,7 @@ public void testMarkFileAndRepairWithCrossSpaceCompactionTask() 0); Assert.assertFalse(task.start()); - for (TsFileResource resource : tsFileManager.getTsFileList(true)) { + for (TsFileResource resource : tsFileManager.getTsFileList(true, COMPACTION_TEST_SG)) { Assert.assertTrue(resource.getTsFileRepairStatus().isRepairCompactionCandidate()); } @@ -520,9 +521,9 @@ public void testMarkFileAndRepairWithCrossSpaceCompactionTask() throw new RuntimeException(e); } } - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(2, tsFileManager.getTsFileList(false).size()); - for (TsFileResource resource : tsFileManager.getTsFileList(false)) { + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); + for (TsFileResource resource : tsFileManager.getTsFileList(false, COMPACTION_TEST_SG)) { TsFileResourceUtils.validateTsFileDataCorrectness(resource); } } @@ -563,14 +564,14 @@ public void testRepairOverlapBetweenFile() throws IOException { RepairUnsortedFileCompactionTask task = new RepairUnsortedFileCompactionTask(0, tsFileManager, seqResource2, true, 0); Assert.assertTrue(task.start()); - Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); Assert.assertTrue( TsFileResourceUtils.validateTsFileDataCorrectness( - tsFileManager.getTsFileList(false).get(0))); + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0))); Assert.assertTrue( TsFileResourceUtils.validateTsFileResourceCorrectness( - tsFileManager.getTsFileList(false).get(0))); + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0))); } @Test @@ -614,9 +615,9 @@ public void testRepairOverlapBetweenFileWithModFile() throws IOException, Illega RepairUnsortedFileCompactionTask task = new RepairUnsortedFileCompactionTask(0, tsFileManager, seqResource2, true, 0); Assert.assertTrue(task.start()); - Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); - TsFileResource targetResource = tsFileManager.getTsFileList(false).get(0); + Assert.assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); + TsFileResource targetResource = tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0); Assert.assertTrue(TsFileResourceUtils.validateTsFileDataCorrectness(targetResource)); Assert.assertTrue(TsFileResourceUtils.validateTsFileResourceCorrectness(targetResource)); Assert.assertTrue(targetResource.anyModFileExists()); @@ -706,7 +707,7 @@ public void testMergeAlignedSeriesPointWithSameTimestamp() throws IOException { RepairUnsortedFileCompactionTask task = new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, true, 0); Assert.assertTrue(task.start()); - TsFileResource target = tsFileManager.getTsFileList(false).get(0); + TsFileResource target = tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0); try (TsFileSequenceReader reader = new TsFileSequenceReader(target.getTsFilePath())) { List chunkMetadataList = reader.getAlignedChunkMetadata( @@ -820,7 +821,7 @@ public void testSplitChunk() throws IOException { RepairUnsortedFileCompactionTask task = new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, true, 0); Assert.assertTrue(task.start()); - TsFileResource target = tsFileManager.getTsFileList(false).get(0); + TsFileResource target = tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0); try (TsFileSequenceReader reader = new TsFileSequenceReader(target.getTsFilePath())) { List chunkMetadataList = reader.getAlignedChunkMetadata( @@ -864,7 +865,7 @@ public void testResourceFileLostDevices() throws IOException { RepairUnsortedFileCompactionTask task = new RepairUnsortedFileCompactionTask(0, tsFileManager, resource, true, 0); Assert.assertTrue(task.start()); - TsFileResource target = tsFileManager.getTsFileList(false).get(0); + TsFileResource target = tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0); try (TsFileSequenceReader reader = new TsFileSequenceReader(target.getTsFilePath())) { List devicesInTargetFile = reader.getAllDevices(); Assert.assertEquals(Arrays.asList(d1, d2), devicesInTargetFile); @@ -909,7 +910,8 @@ public void testQueryRepairResult() throws IOException, IllegalPathException { task = new RepairUnsortedFileCompactionTask(0, tsFileManager, resource2, true, 0); Assert.assertTrue(task.start()); - List sourceTsFileResources = tsFileManager.getTsFileList(false); + List sourceTsFileResources = + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG); List fullPaths = getPaths(sourceTsFileResources); Map> dataByQuery1 = CompactionCheckerUtils.getDataByQuery( diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairUnsortedFileSchedulerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairUnsortedFileSchedulerTest.java index 3108a20468694..404228e9f2f19 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairUnsortedFileSchedulerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/repair/RepairUnsortedFileSchedulerTest.java @@ -127,8 +127,8 @@ public void testScheduleRepairInternalUnsortedFile() throws IOException { new UnsortedFileRepairTaskScheduler( Collections.singletonList(mockDataRegion), false, tempDir); scheduler.run(); - Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); } @Test @@ -196,7 +196,7 @@ public void testRecoverRepairScheduleSkipRepairedTimePartitionAndMarkFile() thro new UnsortedFileRepairTaskScheduler( Collections.singletonList(mockDataRegion), true, tempDir); scheduler.run(); - Assert.assertEquals(3, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(3, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); // check whether the repair status is marked correctly Assert.assertTrue(seqResource3.getTsFileRepairStatus().isRepairCompactionCandidate()); } @@ -256,8 +256,8 @@ public void testScheduleRepairOverlapFileAndInternalUnsortedFile() throws IOExce new UnsortedFileRepairTaskScheduler( Collections.singletonList(mockDataRegion), false, tempDir); scheduler.run(); - Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(2, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); } @Test @@ -303,8 +303,8 @@ public void testScheduleRepairOverlapFile() throws IOException { new UnsortedFileRepairTaskScheduler( Collections.singletonList(mockDataRegion), false, tempDir); scheduler.run(); - Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); } @Test @@ -379,6 +379,6 @@ public void testRecoverRepairScheduleSkipRepairedTimePartitionWithDeletedFile() new UnsortedFileRepairTaskScheduler( Collections.singletonList(mockDataRegion), true, tempDir); scheduler.run(); - Assert.assertEquals(2, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionRecoverTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionRecoverTest.java index d587aa4ca344f..f40c2ed9f6eb6 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionRecoverTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionRecoverTest.java @@ -123,8 +123,8 @@ public void handExceptionWhenSettlingAllDeletedFilesWithOnlyAllDeletedFiles() Assert.assertFalse(resource.getCompactionModFile().exists()); } - Assert.assertEquals(3, tsFileManager.getTsFileList(false).size()); - Assert.assertEquals(6, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(3, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); + Assert.assertEquals(6, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } @Test @@ -178,8 +178,8 @@ public void handExceptionWhenSettlingAllDeletedFiles() Assert.assertFalse(resource.getCompactionModFile().exists()); } - Assert.assertEquals(3, tsFileManager.getTsFileList(false).size()); - Assert.assertEquals(6, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(3, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); + Assert.assertEquals(6, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } @Test @@ -254,12 +254,12 @@ public void handExceptionWhenSettlingPartialDeletedFilesWithAllSourceFileExisted Assert.assertFalse(resource.getCompactionModFile().exists()); } - Assert.assertEquals(3, tsFileManager.getTsFileList(false).size()); - Assert.assertEquals(6, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(3, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); + Assert.assertEquals(6, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); Assert.assertTrue(!tsFileManager.contains(targetResource, false)); Assert.assertTrue(!tsFileManager.contains(targetResource, true)); // resource file exist - for (TsFileResource resource : tsFileManager.getTsFileList(false)) { + for (TsFileResource resource : tsFileManager.getTsFileList(false, COMPACTION_TEST_SG)) { Assert.assertTrue(resource.tsFileExists()); Assert.assertTrue(resource.anyModFileExists()); Assert.assertTrue(resource.resourceFileExists()); @@ -357,8 +357,8 @@ public void handExceptionWhenSettlingPartialDeletedFilesWithSomeSourceFileLosted Assert.assertTrue(targetResource.tsFileExists()); Assert.assertTrue(targetResource.anyModFileExists()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); - for (TsFileResource resource : tsFileManager.getTsFileList(false)) { + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); + for (TsFileResource resource : tsFileManager.getTsFileList(false, COMPACTION_TEST_SG)) { Assert.assertTrue(resource.tsFileExists()); Assert.assertTrue(resource.anyModFileExists()); Assert.assertTrue(resource.resourceFileExists()); @@ -455,7 +455,7 @@ public void handExceptionWhenSettlingPartialDeletedFilesWithSomeSourceFileLosted Assert.assertFalse(targetResource.anyModFileExists()); Assert.assertFalse(targetResource.getCompactionModFile().exists()); - Assert.assertEquals(0, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); } @Test diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionSelectorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionSelectorTest.java index b37997706de27..e06c91fd73a88 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionSelectorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionSelectorTest.java @@ -94,11 +94,13 @@ public void testSelectContinuousFileWithLightSelect() Assert.assertTrue(seqTasks.get(0).start()); Assert.assertTrue(unseqTasks.get(0).start()); - Assert.assertEquals(3, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(3, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(3, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(3, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); - Assert.assertFalse(tsFileManager.getTsFileList(true).get(0).anyModFileExists()); - Assert.assertFalse(tsFileManager.getTsFileList(false).get(0).anyModFileExists()); + Assert.assertFalse( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0).anyModFileExists()); + Assert.assertFalse( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0).anyModFileExists()); // select second time seqTasks = settleSelector.selectSettleTask(seqResources); @@ -110,11 +112,13 @@ public void testSelectContinuousFileWithLightSelect() Assert.assertTrue(seqTasks.get(0).start()); Assert.assertTrue(unseqTasks.get(0).start()); - Assert.assertEquals(2, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(2, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); for (int i = 0; i < 2; i++) { - Assert.assertFalse(tsFileManager.getTsFileList(true).get(i).anyModFileExists()); - Assert.assertFalse(tsFileManager.getTsFileList(false).get(i).anyModFileExists()); + Assert.assertFalse( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(i).anyModFileExists()); + Assert.assertFalse( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(i).anyModFileExists()); } seqTasks = settleSelector.selectSettleTask(seqResources); @@ -164,8 +168,8 @@ public void testSelectUnContinuousFileWithLightSelect() Assert.assertTrue(seqTasks.get(0).start()); Assert.assertTrue(unseqTasks.get(0).start()); - Assert.assertEquals(--num, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(num, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(--num, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(num, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); } List seqTasks = settleSelector.selectSettleTask(seqResources); @@ -232,7 +236,7 @@ public void testSelectContinuousFilesBaseOnDirtyRateByModsWithHeavySelect() Assert.assertEquals(0, seqTasks.get(0).getPartiallyDirtyFiles().size()); Assert.assertTrue(seqTasks.get(0).start()); - Assert.assertEquals(4, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(4, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); // select second time settleSelector = @@ -243,7 +247,7 @@ public void testSelectContinuousFilesBaseOnDirtyRateByModsWithHeavySelect() Assert.assertEquals(0, seqTasks.get(0).getFullyDirtyFiles().size()); Assert.assertEquals(3, seqTasks.get(0).getPartiallyDirtyFiles().size()); Assert.assertTrue(seqTasks.get(0).start()); - Assert.assertEquals(2, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); // select third time seqTasks = settleSelector.selectSettleTask(seqResources); @@ -251,10 +255,11 @@ public void testSelectContinuousFilesBaseOnDirtyRateByModsWithHeavySelect() Assert.assertEquals(0, seqTasks.get(0).getFullyDirtyFiles().size()); Assert.assertEquals(1, seqTasks.get(0).getPartiallyDirtyFiles().size()); Assert.assertTrue(seqTasks.get(0).start()); - Assert.assertEquals(2, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); for (int i = 0; i < 2; i++) { - Assert.assertFalse(tsFileManager.getTsFileList(true).get(i).anyModFileExists()); + Assert.assertFalse( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(i).anyModFileExists()); } } @@ -301,8 +306,9 @@ public void testSelectContinuousFileBaseOnDirtyDataRateWithHeavySelect() Assert.assertEquals(0, unseqTasks.get(0).getFullyDirtyFiles().size()); Assert.assertTrue(unseqTasks.get(0).start()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); - Assert.assertFalse(tsFileManager.getTsFileList(false).get(0).anyModFileExists()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); + Assert.assertFalse( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0).anyModFileExists()); // select third time // all seq files is partial_deleted @@ -321,7 +327,7 @@ public void testSelectContinuousFileBaseOnDirtyDataRateWithHeavySelect() Assert.assertEquals(0, seqTasks.get(0).getFullyDirtyFiles().size()); Assert.assertTrue(seqTasks.get(0).start()); - Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } // base on outdated too long @@ -369,12 +375,14 @@ public void testSelectContinuousFileBaseOnDirtyDataOutdatedTooLongWithHeavySelec Assert.assertTrue(seqTasks.get(1).start()); Assert.assertTrue(unseqTasks.get(0).start()); Assert.assertTrue(unseqTasks.get(1).start()); - Assert.assertEquals(2, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(2, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); for (int i = 0; i < 2; i++) { - Assert.assertFalse(tsFileManager.getTsFileList(true).get(i).anyModFileExists()); - Assert.assertFalse(tsFileManager.getTsFileList(false).get(i).anyModFileExists()); + Assert.assertFalse( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(i).anyModFileExists()); + Assert.assertFalse( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(i).anyModFileExists()); } // select third time @@ -461,7 +469,7 @@ public void testSelectUncontinuousFileBaseOnDirtyDataRateWithHeavySelect() Assert.assertEquals(2, seqTasks.get(1).getPartiallyDirtyFiles().size()); Assert.assertTrue(seqTasks.get(0).start()); Assert.assertTrue(seqTasks.get(1).start()); - Assert.assertEquals(2, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } /** @@ -528,7 +536,7 @@ public void testSelectFileBaseOnDirtyDataRateWithHeavySelect() Assert.assertTrue(seqTasks.get(1).start()); Assert.assertTrue(seqTasks.get(2).start()); Assert.assertTrue(seqTasks.get(3).start()); - Assert.assertEquals(8, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(8, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } /** @@ -594,7 +602,7 @@ public void testSelectFileBaseOnDirtyDataRateWithHeavySelect2() Assert.assertTrue(seqTasks.get(0).start()); Assert.assertTrue(seqTasks.get(1).start()); Assert.assertTrue(seqTasks.get(2).start()); - Assert.assertEquals(6, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(6, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } // endregion @@ -630,11 +638,13 @@ public void testSelectContinuousFileWithLightSelectAligned() Assert.assertTrue(seqTasks.get(0).start()); Assert.assertTrue(unseqTasks.get(0).start()); - Assert.assertEquals(3, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(3, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(3, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(3, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); - Assert.assertFalse(tsFileManager.getTsFileList(true).get(0).anyModFileExists()); - Assert.assertFalse(tsFileManager.getTsFileList(false).get(0).anyModFileExists()); + Assert.assertFalse( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0).anyModFileExists()); + Assert.assertFalse( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0).anyModFileExists()); // select second time seqTasks = settleSelector.selectSettleTask(seqResources); @@ -646,11 +656,13 @@ public void testSelectContinuousFileWithLightSelectAligned() Assert.assertTrue(seqTasks.get(0).start()); Assert.assertTrue(unseqTasks.get(0).start()); - Assert.assertEquals(2, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(2, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); for (int i = 0; i < 2; i++) { - Assert.assertFalse(tsFileManager.getTsFileList(true).get(i).anyModFileExists()); - Assert.assertFalse(tsFileManager.getTsFileList(false).get(i).anyModFileExists()); + Assert.assertFalse( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(i).anyModFileExists()); + Assert.assertFalse( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(i).anyModFileExists()); } seqTasks = settleSelector.selectSettleTask(seqResources); @@ -701,8 +713,8 @@ public void testSelectUnContinuousFileWithLightSelectAligned() Assert.assertTrue(seqTasks.get(0).start()); Assert.assertTrue(unseqTasks.get(0).start()); - Assert.assertEquals(--num, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(num, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(--num, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(num, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); } List seqTasks = settleSelector.selectSettleTask(seqResources); @@ -769,7 +781,7 @@ public void testSelectContinuousFilesBaseOnDirtyRateByModsWithHeavySelectAligned Assert.assertEquals(0, seqTasks.get(0).getPartiallyDirtyFiles().size()); Assert.assertTrue(seqTasks.get(0).start()); - Assert.assertEquals(4, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(4, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); // select second time settleSelector = @@ -780,7 +792,7 @@ public void testSelectContinuousFilesBaseOnDirtyRateByModsWithHeavySelectAligned Assert.assertEquals(0, seqTasks.get(0).getFullyDirtyFiles().size()); Assert.assertEquals(3, seqTasks.get(0).getPartiallyDirtyFiles().size()); Assert.assertTrue(seqTasks.get(0).start()); - Assert.assertEquals(2, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); // select third time seqTasks = settleSelector.selectSettleTask(seqResources); @@ -788,10 +800,11 @@ public void testSelectContinuousFilesBaseOnDirtyRateByModsWithHeavySelectAligned Assert.assertEquals(0, seqTasks.get(0).getFullyDirtyFiles().size()); Assert.assertEquals(1, seqTasks.get(0).getPartiallyDirtyFiles().size()); Assert.assertTrue(seqTasks.get(0).start()); - Assert.assertEquals(2, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); for (int i = 0; i < 2; i++) { - Assert.assertFalse(tsFileManager.getTsFileList(true).get(i).anyModFileExists()); + Assert.assertFalse( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(i).anyModFileExists()); } } @@ -838,8 +851,9 @@ public void testSelectContinuousFileBaseOnDirtyDataRateWithHeavySelectAligned() Assert.assertEquals(0, unseqTasks.get(0).getFullyDirtyFiles().size()); Assert.assertTrue(unseqTasks.get(0).start()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); - Assert.assertFalse(tsFileManager.getTsFileList(false).get(0).anyModFileExists()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); + Assert.assertFalse( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(0).anyModFileExists()); // select third time // all seq files is partial_deleted @@ -858,7 +872,7 @@ public void testSelectContinuousFileBaseOnDirtyDataRateWithHeavySelectAligned() Assert.assertEquals(0, seqTasks.get(0).getFullyDirtyFiles().size()); Assert.assertTrue(seqTasks.get(0).start()); - Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } // base on outdated too long @@ -906,12 +920,14 @@ public void testSelectContinuousFileBaseOnDirtyDataOutdatedTooLongWithHeavySelec Assert.assertTrue(seqTasks.get(1).start()); Assert.assertTrue(unseqTasks.get(0).start()); Assert.assertTrue(unseqTasks.get(1).start()); - Assert.assertEquals(2, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(2, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); for (int i = 0; i < 2; i++) { - Assert.assertFalse(tsFileManager.getTsFileList(true).get(i).anyModFileExists()); - Assert.assertFalse(tsFileManager.getTsFileList(false).get(i).anyModFileExists()); + Assert.assertFalse( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(i).anyModFileExists()); + Assert.assertFalse( + tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).get(i).anyModFileExists()); } // select third time @@ -998,7 +1014,7 @@ public void testSelectUncontinuousFileBaseOnDirtyDataRateWithHeavySelectAligned( Assert.assertEquals(2, seqTasks.get(1).getPartiallyDirtyFiles().size()); Assert.assertTrue(seqTasks.get(0).start()); Assert.assertTrue(seqTasks.get(1).start()); - Assert.assertEquals(2, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(2, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } /** @@ -1066,7 +1082,7 @@ public void testSelectFileBaseOnDirtyDataRateWithHeavySelectAligned() Assert.assertTrue(seqTasks.get(1).start()); Assert.assertTrue(seqTasks.get(2).start()); Assert.assertTrue(seqTasks.get(3).start()); - Assert.assertEquals(8, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(8, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } /** @@ -1133,7 +1149,7 @@ public void testSelectFileBaseOnDirtyDataRateWithHeavySelect2Aligned() Assert.assertTrue(seqTasks.get(0).start()); Assert.assertTrue(seqTasks.get(1).start()); Assert.assertTrue(seqTasks.get(2).start()); - Assert.assertEquals(6, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(6, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } // endregion diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionTaskTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionTaskTest.java index e318d0e87d07a..bbcb6b22ec4d9 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionTaskTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/settle/SettleCompactionTaskTest.java @@ -187,8 +187,8 @@ public void settleWithOnlyAllDirtyFilesByMods() Assert.assertEquals(TsFileResourceStatus.DELETED, tsFileResource.getStatus()); } - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(0, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); DataNodeTTLCache.getInstance().clearAllTTLForTree(); validateTargetDatas(sourceDatas, Collections.emptyList()); @@ -226,8 +226,8 @@ public void settleWithOnlyPartialDirtyFilesByMods() Assert.assertEquals(TsFileResourceStatus.DELETED, tsFileResource.getStatus()); } - Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); DataNodeTTLCache.getInstance().clearAllTTLForTree(); validateTargetDatas(sourceDatas, Collections.emptyList()); @@ -259,8 +259,8 @@ public void settleWithMixedDirtyFilesByMods() 0, tsFileManager, allDeletedFiles, partialDeletedFiles, false, getPerformer(), 0); Assert.assertTrue(task.start()); - Assert.assertEquals(6, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(6, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); DataNodeTTLCache.getInstance().clearAllTTLForTree(); validateTargetDatas(sourceDatas, Collections.emptyList()); @@ -271,8 +271,8 @@ public void settleWithMixedDirtyFilesByMods() 0, tsFileManager, allDeletedFiles, partialDeletedFiles, true, getPerformer(), 0); Assert.assertTrue(task.start()); - Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); for (TsFileResource tsFileResource : seqResources) { Assert.assertEquals(TsFileResourceStatus.DELETED, tsFileResource.getStatus()); } @@ -313,8 +313,8 @@ public void settleWithOnlyAllDirtyFilesByTTL() Assert.assertEquals(TsFileResourceStatus.DELETED, tsFileResource.getStatus()); } - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(0, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); DataNodeTTLCache.getInstance().clearAllTTLForTree(); validateTargetDatas(sourceDatas, Collections.emptyList()); @@ -355,8 +355,8 @@ public void settleWithOnlyAllDirtyFilesByTTL2() Assert.assertEquals(TsFileResourceStatus.DELETED, tsFileResource.getStatus()); } - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(0, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); DataNodeTTLCache.getInstance().clearAllTTLForTree(); validateTargetDatas(sourceDatas, Collections.emptyList()); @@ -402,8 +402,8 @@ public void settleWithOnlyPartialDirtyFilesByTTL() Assert.assertEquals(TsFileResourceStatus.DELETED, tsFileResource.getStatus()); } - Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); DataNodeTTLCache.getInstance().clearAllTTLForTree(); validateTargetDatas(sourceDatas, Collections.emptyList()); @@ -428,7 +428,7 @@ public void getModsFileAfterSettleCompaction() getPerformer(), 0); Assert.assertTrue(task.start()); - List tsFileList = tsFileManager.getTsFileList(true); + List tsFileList = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); for (TsFileResource resource : tsFileList) { Assert.assertTrue( resource @@ -476,8 +476,8 @@ public void settleWithMixedDirtyFilesByTTL() Assert.assertEquals(TsFileResourceStatus.DELETED, tsFileResource.getStatus()); } - Assert.assertEquals(1, tsFileManager.getTsFileList(true).size()); - Assert.assertEquals(1, tsFileManager.getTsFileList(false).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); + Assert.assertEquals(1, tsFileManager.getTsFileList(false, COMPACTION_TEST_SG).size()); DataNodeTTLCache.getInstance().clearAllTTLForTree(); validateTargetDatas(sourceDatas, Collections.emptyList()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/CompactionWithAllNullRowsTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/CompactionWithAllNullRowsTest.java index 9ff8a401150b8..59aad2eefffd6 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/CompactionWithAllNullRowsTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/CompactionWithAllNullRowsTest.java @@ -133,14 +133,19 @@ public void testCompactionWithAllNullRows1() throws IOException { InnerSpaceCompactionTask task = new InnerSpaceCompactionTask(0, tsFileManager, seqResources, true, getPerformer(), 0); Assert.assertTrue(task.start()); - TsFileResource target = tsFileManager.getTsFileList(true).get(0); + TsFileResource target = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); Assert.assertEquals(10, target.getFileStartTime()); Assert.assertEquals(12, target.getFileEndTime()); InnerSpaceCompactionTask task2 = new InnerSpaceCompactionTask( - 0, tsFileManager, tsFileManager.getTsFileList(true), true, getPerformer(), 0); + 0, + tsFileManager, + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + true, + getPerformer(), + 0); Assert.assertTrue(task2.start()); - TsFileResource target2 = tsFileManager.getTsFileList(true).get(0); + TsFileResource target2 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); Assert.assertEquals(10, target2.getFileStartTime()); Assert.assertEquals(12, target2.getFileEndTime()); } @@ -168,14 +173,19 @@ public void testCompactionWithAllNullRows2() throws IOException { InnerSpaceCompactionTask task = new InnerSpaceCompactionTask(0, tsFileManager, seqResources, true, getPerformer(), 0); Assert.assertTrue(task.start()); - TsFileResource target = tsFileManager.getTsFileList(true).get(0); + TsFileResource target = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); Assert.assertEquals(1, target.getFileStartTime()); Assert.assertEquals(12, target.getFileEndTime()); InnerSpaceCompactionTask task2 = new InnerSpaceCompactionTask( - 0, tsFileManager, tsFileManager.getTsFileList(true), true, getPerformer(), 0); + 0, + tsFileManager, + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + true, + getPerformer(), + 0); Assert.assertTrue(task2.start()); - TsFileResource target2 = tsFileManager.getTsFileList(true).get(0); + TsFileResource target2 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); Assert.assertEquals(1, target2.getFileStartTime()); Assert.assertEquals(12, target2.getFileEndTime()); } @@ -201,14 +211,19 @@ public void testCompactionWithAllNullRows3() throws IOException { InnerSpaceCompactionTask task = new InnerSpaceCompactionTask(0, tsFileManager, seqResources, true, getPerformer(), 0); Assert.assertTrue(task.start()); - TsFileResource target = tsFileManager.getTsFileList(true).get(0); + TsFileResource target = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); Assert.assertEquals(10000, target.getFileStartTime()); Assert.assertEquals(19999, target.getFileEndTime()); InnerSpaceCompactionTask task2 = new InnerSpaceCompactionTask( - 0, tsFileManager, tsFileManager.getTsFileList(true), true, getPerformer(), 0); + 0, + tsFileManager, + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + true, + getPerformer(), + 0); Assert.assertTrue(task2.start()); - TsFileResource target2 = tsFileManager.getTsFileList(true).get(0); + TsFileResource target2 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); Assert.assertEquals(10000, target2.getFileStartTime()); Assert.assertEquals(19999, target2.getFileEndTime()); } @@ -234,14 +249,19 @@ public void testCompactionWithAllNullRows4() throws IOException { InnerSpaceCompactionTask task = new InnerSpaceCompactionTask(0, tsFileManager, seqResources, true, getPerformer(), 0); Assert.assertTrue(task.start()); - TsFileResource target = tsFileManager.getTsFileList(true).get(0); + TsFileResource target = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); Assert.assertEquals(100000, target.getFileStartTime()); Assert.assertEquals(199999, target.getFileEndTime()); InnerSpaceCompactionTask task2 = new InnerSpaceCompactionTask( - 0, tsFileManager, tsFileManager.getTsFileList(true), true, getPerformer(), 0); + 0, + tsFileManager, + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + true, + getPerformer(), + 0); Assert.assertTrue(task2.start()); - TsFileResource target2 = tsFileManager.getTsFileList(true).get(0); + TsFileResource target2 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); Assert.assertEquals(100000, target2.getFileStartTime()); Assert.assertEquals(199999, target2.getFileEndTime()); } @@ -274,7 +294,7 @@ public void testCompactionWithAllDeletion1() throws IOException { InnerSpaceCompactionTask task = new InnerSpaceCompactionTask(0, tsFileManager, seqResources, true, getPerformer(), 0); Assert.assertTrue(task.start()); - Assert.assertTrue(tsFileManager.getTsFileList(true).isEmpty()); + Assert.assertTrue(tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).isEmpty()); } @Test @@ -305,7 +325,7 @@ public void testCompactionWithAllDeletion2() throws IOException { InnerSpaceCompactionTask task = new InnerSpaceCompactionTask(0, tsFileManager, seqResources, true, getPerformer(), 0); Assert.assertTrue(task.start()); - TsFileResource target = tsFileManager.getTsFileList(true).get(0); + TsFileResource target = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); Assert.assertEquals(12, target.getFileStartTime()); Assert.assertEquals(12, target.getFileEndTime()); } @@ -368,7 +388,7 @@ public void testCompactionWithAllValueColumnDeletion() throws IOException, Illeg InnerSpaceCompactionTask task = new InnerSpaceCompactionTask(0, tsFileManager, seqResources, true, getPerformer(), 0); Assert.assertTrue(task.start()); - TsFileResource target = tsFileManager.getTsFileList(true).get(0); + TsFileResource target = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); Assert.assertEquals(10, target.getFileStartTime()); Assert.assertEquals(12, target.getFileEndTime()); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelCompactionWithTTLTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelCompactionWithTTLTest.java index 6746f0a274697..9ffb3d65a9098 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelCompactionWithTTLTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelCompactionWithTTLTest.java @@ -102,7 +102,7 @@ public void testAllDataExpired() throws IOException { new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, getPerformer(performerType), 0); Assert.assertTrue(task.start()); - Assert.assertEquals(0, tsFileManager.getTsFileList(true).size()); + Assert.assertEquals(0, tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).size()); } @Test @@ -131,7 +131,7 @@ public void testPartialDataExpired() throws IOException { new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, getPerformer(performerType), 0); Assert.assertTrue(task.start()); - TsFileResource target = tsFileManager.getTsFileList(true).get(0); + TsFileResource target = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); Assert.assertTrue(target.getFileStartTime() > startTime && target.getFileEndTime() == endTime); } @@ -160,7 +160,7 @@ public void testTableNotExist() throws IOException { new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, getPerformer(performerType), 0); Assert.assertTrue(task.start()); - TsFileResource target = tsFileManager.getTsFileList(true).get(0); + TsFileResource target = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); Assert.assertTrue(target.getFileStartTime() == startTime && target.getFileEndTime() == endTime); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelFastCompactionPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelFastCompactionPerformerTest.java index 778220d826389..a60c65d63599d 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelFastCompactionPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelFastCompactionPerformerTest.java @@ -131,12 +131,12 @@ public void testCrossSpaceCompactionOfTwoTableModelWithFastCompactionPerformer() 0, 0); Assert.assertTrue(task.start()); - TsFileResource targetResource0 = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetResource0 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); try (TsFileSequenceReader reader = new TsFileSequenceReader(targetResource0.getTsFile().getAbsolutePath())) { Assert.assertEquals(1, reader.getTableSchemaMap().size()); } - TsFileResource targetResource1 = tsFileManager.getTsFileList(true).get(1); + TsFileResource targetResource1 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1); try (TsFileSequenceReader reader = new TsFileSequenceReader(targetResource1.getTsFile().getAbsolutePath())) { Assert.assertEquals(1, reader.getTableSchemaMap().size()); @@ -194,7 +194,11 @@ public void testCompactionWithV3Tsfile() throws IOException { Assert.assertTrue(task.start()); try (TsFileSequenceReader reader = new TsFileSequenceReader( - tsFileManager.getTsFileList(true).get(0).getTsFile().getAbsolutePath())) { + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .getTsFile() + .getAbsolutePath())) { Assert.assertEquals(1, reader.getTableSchemaMap().size()); } } @@ -232,7 +236,11 @@ public void testCompactionWithMultiTableSchema() throws IOException { Assert.assertTrue(task.start()); try (TsFileSequenceReader reader = new TsFileSequenceReader( - tsFileManager.getTsFileList(true).get(0).getTsFile().getAbsolutePath())) { + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .getTsFile() + .getAbsolutePath())) { Assert.assertEquals(1, reader.getTableSchemaMap().size()); } } @@ -293,7 +301,7 @@ public void testCrossSpaceCompactionOfTableModelCanNotMatchTableSchema() throws 0, 0); Assert.assertTrue(task.start()); - TsFileResource targetResource = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetResource = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); Assert.assertEquals(1, targetResource.getDevices().size()); try (TsFileSequenceReader reader = new TsFileSequenceReader(targetResource.getTsFile().getAbsolutePath())) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadChunkCompactionPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadChunkCompactionPerformerTest.java index f2721652f99c5..b0bab6d80bfbb 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadChunkCompactionPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadChunkCompactionPerformerTest.java @@ -109,7 +109,7 @@ public void testSequenceInnerSpaceCompactionOfTwoTableModel() throws IOException new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, new ReadChunkCompactionPerformer(), 0); Assert.assertTrue(task.start()); - TsFileResource targetResource = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetResource = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); try (TsFileSequenceReader reader = new TsFileSequenceReader(targetResource.getTsFile().getAbsolutePath())) { TableSchema tableSchema = reader.getTableSchemaMap().get("t1"); @@ -152,7 +152,7 @@ public void testSequenceInnerSpaceCompactionOfTwoV4TreeModel() throws IOExceptio new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, new ReadChunkCompactionPerformer(), 0); Assert.assertTrue(task.start()); - TsFileResource targetResource = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetResource = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); try (TsFileSequenceReader reader = new TsFileSequenceReader(targetResource.getTsFile().getAbsolutePath())) { Assert.assertTrue(reader.getTableSchemaMap().isEmpty()); @@ -199,7 +199,7 @@ public void testSequenceInnerSpaceCompactionOfTableModelAndTreeModel() throws IO new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, new ReadChunkCompactionPerformer(), 0); Assert.assertTrue(task.start()); - TsFileResource targetResource = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetResource = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); try (TsFileSequenceReader reader = new TsFileSequenceReader(targetResource.getTsFile().getAbsolutePath())) { Assert.assertEquals(1, reader.getTableSchemaMap().size()); @@ -257,7 +257,7 @@ public void testSequenceInnerSpaceCompactionOfTableModelCanNotMatchTableSchema() new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, new ReadChunkCompactionPerformer(), 0); Assert.assertTrue(task.start()); - TsFileResource targetResource = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetResource = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); Assert.assertEquals(1, targetResource.getDevices().size()); try (TsFileSequenceReader reader = new TsFileSequenceReader(targetResource.getTsFile().getAbsolutePath())) { @@ -330,7 +330,11 @@ public void testCompactionWithV3Tsfile() throws IOException { Assert.assertTrue(task.start()); try (TsFileSequenceReader reader = new TsFileSequenceReader( - tsFileManager.getTsFileList(true).get(0).getTsFile().getAbsolutePath())) { + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .getTsFile() + .getAbsolutePath())) { Assert.assertEquals(1, reader.getTableSchemaMap().size()); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadPointCompactionPerformerTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadPointCompactionPerformerTest.java index 740c2434fdb31..39b954968c8cd 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadPointCompactionPerformerTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/tablemodel/TableModelReadPointCompactionPerformerTest.java @@ -155,12 +155,12 @@ public void testCrossSpaceCompactionOfTwoTableModelWithReadPointCompactionPerfor 0); Assert.assertTrue(task.start()); // Assert can not pass for now - TsFileResource targetResource0 = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetResource0 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); try (TsFileSequenceReader reader = new TsFileSequenceReader(targetResource0.getTsFile().getAbsolutePath())) { Assert.assertEquals(1, reader.getTableSchemaMap().size()); } - TsFileResource targetResource1 = tsFileManager.getTsFileList(true).get(1); + TsFileResource targetResource1 = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(1); try (TsFileSequenceReader reader = new TsFileSequenceReader(targetResource1.getTsFile().getAbsolutePath())) { Assert.assertEquals(1, reader.getTableSchemaMap().size()); @@ -218,7 +218,11 @@ public void testCompactionWithV3Tsfile() throws IOException { Assert.assertTrue(task.start()); try (TsFileSequenceReader reader = new TsFileSequenceReader( - tsFileManager.getTsFileList(true).get(0).getTsFile().getAbsolutePath())) { + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .getTsFile() + .getAbsolutePath())) { Assert.assertEquals(1, reader.getTableSchemaMap().size()); } } @@ -256,7 +260,11 @@ public void testCompactionWithMultiTableSchema() throws IOException { Assert.assertTrue(task.start()); try (TsFileSequenceReader reader = new TsFileSequenceReader( - tsFileManager.getTsFileList(true).get(0).getTsFile().getAbsolutePath())) { + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .getTsFile() + .getAbsolutePath())) { Assert.assertEquals(1, reader.getTableSchemaMap().size()); } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/CompactionTaskMemCostEstimatorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/CompactionTaskMemCostEstimatorTest.java index e96ea2653d890..8f9fa2f8c5592 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/CompactionTaskMemCostEstimatorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/CompactionTaskMemCostEstimatorTest.java @@ -70,7 +70,7 @@ public void testEstimateReadChunkInnerSpaceCompactionTaskMemCost() throws IOException, MetadataException, WriteProcessException { createFiles(3, 10, 5, 100000, 0, 0, 50, 50, true, true); tsFileManager.addAll(seqResources, true); - List tsFileList = tsFileManager.getTsFileList(true); + List tsFileList = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); System.out.println(tsFileList.get(0).getTsFile().getAbsolutePath()); long cost = new ReadChunkInnerCompactionEstimator().estimateInnerCompactionMemory(tsFileList); Assert.assertTrue(cost > 0); @@ -81,7 +81,7 @@ public void testEstimateReadChunkInnerSpaceCompactionTaskMemCost2() throws IOException, MetadataException, WriteProcessException { createFiles(3, 10, 5, 100, 0, 0, 50, 50, false, true); tsFileManager.addAll(seqResources, true); - List tsFileList = tsFileManager.getTsFileList(true); + List tsFileList = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); long cost = new ReadChunkInnerCompactionEstimator().estimateInnerCompactionMemory(tsFileList); Assert.assertTrue(cost > 0); } @@ -103,7 +103,7 @@ public void testEstimateFastCompactionInnerSpaceCompactionTaskMemCost2() throws IOException, MetadataException, WriteProcessException { createFiles(3, 10, 5, 100, 0, 0, 50, 50, false, true); tsFileManager.addAll(seqResources, true); - List tsFileList = tsFileManager.getTsFileList(true); + List tsFileList = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG); long cost = new FastCompactionInnerCompactionEstimator().estimateInnerCompactionMemory(tsFileList); Assert.assertTrue(cost > 0); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/MultiTsFileDeviceIteratorTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/MultiTsFileDeviceIteratorTest.java index 1180247a2d3fc..4f625fdd69d93 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/MultiTsFileDeviceIteratorTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/compaction/utils/MultiTsFileDeviceIteratorTest.java @@ -146,7 +146,7 @@ public void testMeasurementIterator() throws IOException, MetadataException { new InnerSpaceCompactionTask( 0, tsFileManager, seqResources, true, new ReadChunkCompactionPerformer(), 0) .start(); - TsFileResource targetFile = tsFileManager.getTsFileList(true).get(0); + TsFileResource targetFile = tsFileManager.getTsFileList(true, COMPACTION_TEST_SG).get(0); try (TsFileSequenceReader reader = new TsFileSequenceReader(targetFile.getTsFile().getAbsolutePath())) { Assert.assertEquals(4000, reader.getAllMeasurements().size()); @@ -448,7 +448,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadChunkPerformer() int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true, COMPACTION_TEST_SG))) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -488,7 +488,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadChunkPerformer() new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadChunkCompactionPerformer(), 0L); @@ -505,7 +505,10 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadChunkPerformer() } } generateModsFile( - seriesPaths, tsFileManager.getTsFileList(true), Long.MIN_VALUE, Long.MAX_VALUE); + seriesPaths, + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Long.MIN_VALUE, + Long.MAX_VALUE); deleteTimeseriesInMManager(seriesPaths); @@ -527,7 +530,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadChunkPerformer() deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true, COMPACTION_TEST_SG))) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -539,17 +542,24 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadChunkPerformer() TsFileGeneratorUtils.alignDeviceOffset = oldAlignedDeviceOffset; List targetResources = CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources( - tsFileManager.getTsFileList(true), true); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true); ReadChunkCompactionPerformer performer = - new ReadChunkCompactionPerformer(tsFileManager.getTsFileList(true), targetResources.get(0)); + new ReadChunkCompactionPerformer( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), targetResources.get(0)); performer.setSummary(new CompactionTaskSummary()); performer.perform(); CompactionUtils.moveTargetFile( targetResources, CompactionTaskType.INNER_SEQ, COMPACTION_TEST_SG); tsFileManager.replace( - tsFileManager.getTsFileList(true), Collections.emptyList(), targetResources, 0); - tsFileManager.getTsFileList(true).get(0).setStatusForTest(TsFileResourceStatus.NORMAL); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Collections.emptyList(), + targetResources, + 0); + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .setStatusForTest(TsFileResourceStatus.NORMAL); validateSeqFiles(true); validateTargetDatas(sourceData, Collections.emptyList()); @@ -602,7 +612,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadChunkPerformer2() int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true, COMPACTION_TEST_SG))) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -642,7 +652,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadChunkPerformer2() new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadChunkCompactionPerformer(), 0L); @@ -659,7 +669,10 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadChunkPerformer2() } } generateModsFile( - seriesPaths, tsFileManager.getTsFileList(true), Long.MIN_VALUE, Long.MAX_VALUE); + seriesPaths, + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Long.MIN_VALUE, + Long.MAX_VALUE); deleteTimeseriesInMManager(seriesPaths); @@ -682,7 +695,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadChunkPerformer2() deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true, COMPACTION_TEST_SG))) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -694,17 +707,24 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadChunkPerformer2() TsFileGeneratorUtils.alignDeviceOffset = oldAlignedDeviceOffset; List targetResources = CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources( - tsFileManager.getTsFileList(true), true); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true); ReadChunkCompactionPerformer performer = - new ReadChunkCompactionPerformer(tsFileManager.getTsFileList(true), targetResources.get(0)); + new ReadChunkCompactionPerformer( + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), targetResources.get(0)); performer.setSummary(new CompactionTaskSummary()); performer.perform(); CompactionUtils.moveTargetFile( targetResources, CompactionTaskType.INNER_SEQ, COMPACTION_TEST_SG); tsFileManager.replace( - tsFileManager.getTsFileList(true), Collections.emptyList(), targetResources, 0); - tsFileManager.getTsFileList(true).get(0).setStatusForTest(TsFileResourceStatus.NORMAL); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Collections.emptyList(), + targetResources, + 0); + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .setStatusForTest(TsFileResourceStatus.NORMAL); validateSeqFiles(true); validateTargetDatas(sourceData, Collections.emptyList()); @@ -752,7 +772,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadPointPerformer() thro int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true, COMPACTION_TEST_SG))) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -792,7 +812,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadPointPerformer() thro new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0L); @@ -809,7 +829,10 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadPointPerformer() thro } } generateModsFile( - seriesPaths, tsFileManager.getTsFileList(true), Long.MIN_VALUE, Long.MAX_VALUE); + seriesPaths, + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Long.MIN_VALUE, + Long.MAX_VALUE); deleteTimeseriesInMManager(seriesPaths); @@ -831,7 +854,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadPointPerformer() thro deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true, COMPACTION_TEST_SG))) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -843,18 +866,26 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadPointPerformer() thro TsFileGeneratorUtils.alignDeviceOffset = oldAlignedDeviceOffset; List targetResources = CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources( - tsFileManager.getTsFileList(true), true); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true); ReadPointCompactionPerformer performer = new ReadPointCompactionPerformer( - tsFileManager.getTsFileList(true), Collections.emptyList(), targetResources); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Collections.emptyList(), + targetResources); performer.setSummary(new CompactionTaskSummary()); performer.perform(); CompactionUtils.moveTargetFile( targetResources, CompactionTaskType.INNER_SEQ, COMPACTION_TEST_SG); tsFileManager.replace( - tsFileManager.getTsFileList(true), Collections.emptyList(), targetResources, 0); - tsFileManager.getTsFileList(true).get(0).setStatusForTest(TsFileResourceStatus.NORMAL); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Collections.emptyList(), + targetResources, + 0); + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .setStatusForTest(TsFileResourceStatus.NORMAL); validateSeqFiles(true); validateTargetDatas(sourceData, Collections.emptyList()); @@ -902,7 +933,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadPointPerformer2() thr int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true, COMPACTION_TEST_SG))) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -942,7 +973,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadPointPerformer2() thr new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new ReadPointCompactionPerformer(), 0L); @@ -959,7 +990,10 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadPointPerformer2() thr } } generateModsFile( - seriesPaths, tsFileManager.getTsFileList(true), Long.MIN_VALUE, Long.MAX_VALUE); + seriesPaths, + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Long.MIN_VALUE, + Long.MAX_VALUE); deleteTimeseriesInMManager(seriesPaths); @@ -991,7 +1025,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadPointPerformer2() thr deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true, COMPACTION_TEST_SG))) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -1003,18 +1037,26 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByReadPointPerformer2() thr TsFileGeneratorUtils.alignDeviceOffset = oldAlignedDeviceOffset; List targetResources = CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources( - tsFileManager.getTsFileList(true), true); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true); ReadPointCompactionPerformer performer = new ReadPointCompactionPerformer( - tsFileManager.getTsFileList(true), Collections.emptyList(), targetResources); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Collections.emptyList(), + targetResources); performer.setSummary(new CompactionTaskSummary()); performer.perform(); CompactionUtils.moveTargetFile( targetResources, CompactionTaskType.INNER_SEQ, COMPACTION_TEST_SG); tsFileManager.replace( - tsFileManager.getTsFileList(true), Collections.emptyList(), targetResources, 0); - tsFileManager.getTsFileList(true).get(0).setStatusForTest(TsFileResourceStatus.NORMAL); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Collections.emptyList(), + targetResources, + 0); + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .setStatusForTest(TsFileResourceStatus.NORMAL); validateSeqFiles(true); validateTargetDatas(sourceData, Collections.emptyList()); @@ -1062,7 +1104,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByFastPerformer() throws Ex int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true, COMPACTION_TEST_SG))) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -1102,7 +1144,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByFastPerformer() throws Ex new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0L); @@ -1119,7 +1161,10 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByFastPerformer() throws Ex } } generateModsFile( - seriesPaths, tsFileManager.getTsFileList(true), Long.MIN_VALUE, Long.MAX_VALUE); + seriesPaths, + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Long.MIN_VALUE, + Long.MAX_VALUE); deleteTimeseriesInMManager(seriesPaths); @@ -1141,7 +1186,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByFastPerformer() throws Ex deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true, COMPACTION_TEST_SG))) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -1153,18 +1198,26 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByFastPerformer() throws Ex TsFileGeneratorUtils.alignDeviceOffset = oldAlignedDeviceOffset; List targetResources = CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources( - tsFileManager.getTsFileList(true), true); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true); FastCompactionPerformer performer = new FastCompactionPerformer( - tsFileManager.getTsFileList(true), Collections.emptyList(), targetResources); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Collections.emptyList(), + targetResources); performer.setSummary(new FastCompactionTaskSummary()); performer.perform(); CompactionUtils.moveTargetFile( targetResources, CompactionTaskType.INNER_SEQ, COMPACTION_TEST_SG); tsFileManager.replace( - tsFileManager.getTsFileList(true), Collections.emptyList(), targetResources, 0); - tsFileManager.getTsFileList(true).get(0).setStatusForTest(TsFileResourceStatus.NORMAL); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Collections.emptyList(), + targetResources, + 0); + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .setStatusForTest(TsFileResourceStatus.NORMAL); validateSeqFiles(true); validateTargetDatas(sourceData, Collections.emptyList()); @@ -1212,7 +1265,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByFastPerformer2() throws E int deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true, COMPACTION_TEST_SG))) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -1252,7 +1305,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByFastPerformer2() throws E new InnerSpaceCompactionTask( 0, tsFileManager, - tsFileManager.getTsFileList(true), + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true, new FastCompactionPerformer(false), 0L); @@ -1269,7 +1322,10 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByFastPerformer2() throws E } } generateModsFile( - seriesPaths, tsFileManager.getTsFileList(true), Long.MIN_VALUE, Long.MAX_VALUE); + seriesPaths, + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Long.MIN_VALUE, + Long.MAX_VALUE); deleteTimeseriesInMManager(seriesPaths); @@ -1301,7 +1357,7 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByFastPerformer2() throws E deviceNum = 0; try (MultiTsFileDeviceIterator multiTsFileDeviceIterator = - new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true))) { + new MultiTsFileDeviceIterator(tsFileManager.getTsFileList(true, COMPACTION_TEST_SG))) { while (multiTsFileDeviceIterator.hasNextDevice()) { Pair deviceInfo = multiTsFileDeviceIterator.nextDevice(); Assert.assertEquals(deviceIds.get(deviceNum), deviceInfo.left); @@ -1313,18 +1369,26 @@ public void getDeletedDevicesWithSameNameFromSeqFilesByFastPerformer2() throws E TsFileGeneratorUtils.alignDeviceOffset = oldAlignedDeviceOffset; List targetResources = CompactionFileGeneratorUtils.getInnerCompactionTargetTsFileResources( - tsFileManager.getTsFileList(true), true); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), true); FastCompactionPerformer performer = new FastCompactionPerformer( - tsFileManager.getTsFileList(true), Collections.emptyList(), targetResources); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Collections.emptyList(), + targetResources); performer.setSummary(new FastCompactionTaskSummary()); performer.perform(); CompactionUtils.moveTargetFile( targetResources, CompactionTaskType.INNER_SEQ, COMPACTION_TEST_SG); tsFileManager.replace( - tsFileManager.getTsFileList(true), Collections.emptyList(), targetResources, 0); - tsFileManager.getTsFileList(true).get(0).setStatusForTest(TsFileResourceStatus.NORMAL); + tsFileManager.getTsFileList(true, COMPACTION_TEST_SG), + Collections.emptyList(), + targetResources, + 0); + tsFileManager + .getTsFileList(true, COMPACTION_TEST_SG) + .get(0) + .setStatusForTest(TsFileResourceStatus.NORMAL); validateSeqFiles(true); validateTargetDatas(sourceData, Collections.emptyList()); diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/snapshot/IoTDBSnapshotTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/snapshot/IoTDBSnapshotTest.java index 7976c14e87a8d..4946d4a44c0fc 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/snapshot/IoTDBSnapshotTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/snapshot/IoTDBSnapshotTest.java @@ -198,7 +198,8 @@ public void testLoadSnapshot() new SnapshotLoader(snapshotDir.getAbsolutePath(), testSgName, "0") .loadSnapshotForStateMachine(); Assert.assertNotNull(dataRegion); - List resource = dataRegion.getTsFileManager().getTsFileList(true); + List resource = + dataRegion.getTsFileManager().getTsFileList(true, testSgName); Assert.assertEquals(100, resource.size()); Assert.assertEquals( new Pair<>(100L, 100L), 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 059c438bea8b0..9a8cd3c5b429b 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 @@ -41,6 +41,7 @@ import org.apache.thrift.transport.TTransportException; import java.io.DataOutputStream; +import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; @@ -227,7 +228,17 @@ public static TTimeSlotList deserializeTTimePartitionSlotList(ByteBuffer buffer) } public static void serializeTTimePartitionSlot( - Map timePartitionSlot, DataOutputStream stream) { + Map timePartitionSlot, DataOutputStream stream) + throws IOException { + stream.writeInt(timePartitionSlot.size()); + for (Map.Entry entry : timePartitionSlot.entrySet()) { + BasicStructureSerDeUtil.write(entry.getKey(), stream); + serializeTTimePartitionSlot(entry.getValue(), stream); + } + } + + public static void serializeTTimePartitionSlot( + TTimePartitionSlot timePartitionSlot, DataOutputStream stream) throws IOException { try { timePartitionSlot.write(generateWriteProtocol(stream)); } catch (TException e) { @@ -245,6 +256,17 @@ public static TTimePartitionSlot deserializeTTimePartitionSlot(ByteBuffer buffer return timePartitionSlot; } + public static Map deserializeTTimePartitionSlotMap( + ByteBuffer buffer, Map map) { + int size = buffer.getInt(); + for (int i = 0; i < size; i++) { + String key = BasicStructureSerDeUtil.readString(buffer); + TTimePartitionSlot value = deserializeTTimePartitionSlot(buffer); + map.put(key, value); + } + return map; + } + public static void serializeTConsensusGroupId( TConsensusGroupId consensusGroupId, DataOutputStream stream) { try { 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 798345511ab21..9286fc4ad7321 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 @@ -329,66 +329,4 @@ private static long getTimePartitionIdWithoutOverflow(long time, long origin, lo : bigTime.divide(bigInterval).subtract(BigInteger.ONE); return partitionId.longValue(); } - - // 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); - } } From d9c6715cf8cb12a76848c08c8108e48b6f2314d1 Mon Sep 17 00:00:00 2001 From: libo Date: Sun, 5 Apr 2026 07:50:43 +0800 Subject: [PATCH 04/15] Fix the problem that don't override abstract method getTimePartition(java.lang.String,java.lang.String) --- .../dataregion/tsfile/timeindex/FileTimeIndex.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndex.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndex.java index 3854c8c6622af..22f7c22220522 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndex.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/tsfile/timeindex/FileTimeIndex.java @@ -181,7 +181,7 @@ public long calculateRamSize() { } @Override - public long getTimePartition(String tsFilePath) { + public long getTimePartition(String tsFilePath, String database) { try { String[] filePathSplits = FilePathUtils.splitTsFilePath(tsFilePath); return Long.parseLong(filePathSplits[filePathSplits.length - 2]); From ac83b66876d2b64e7bfc8d05011c93341b2f95c0 Mon Sep 17 00:00:00 2001 From: libo Date: Sun, 5 Apr 2026 08:48:43 +0800 Subject: [PATCH 05/15] Fix the problem caused by compile failure. --- .../procedure/PartitionTableAutoCleaner.java | 10 +- .../dataregion/DataRegionTest.java | 264 ++++++++++++++++++ 2 files changed, 271 insertions(+), 3 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/PartitionTableAutoCleaner.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/PartitionTableAutoCleaner.java index 84f12a78ec46b..c442307013dad 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/PartitionTableAutoCleaner.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/PartitionTableAutoCleaner.java @@ -60,6 +60,7 @@ public PartitionTableAutoCleaner(ConfigManager configManager) { protected void periodicExecute(Env env) { List databases = configManager.getClusterSchemaManager().getDatabaseNames(null); Map databaseTTLMap = new TreeMap<>(); + Map timePartitionSlotMap = new TreeMap<>(); for (String database : databases) { long databaseTTL; if (PathUtils.isTableModelDatabase(database)) { @@ -93,12 +94,15 @@ protected void periodicExecute(Env env) { LOGGER.info( "[PartitionTableCleaner] Periodically activate PartitionTableAutoCleaner for: {}", databaseTTLMap); - // Only clean the partition table when necessary - TTimePartitionSlot currentTimePartitionSlot = getCurrentTimePartitionSlot(); + for (String database : databaseTTLMap.keySet()) { + // Only clean the partition table when necessary + TTimePartitionSlot currentTimePartitionSlot = getCurrentTimePartitionSlot(); + timePartitionSlotMap.put(database, currentTimePartitionSlot); + } try { configManager .getConsensusManager() - .write(new AutoCleanPartitionTablePlan(databaseTTLMap, currentTimePartitionSlot)); + .write(new AutoCleanPartitionTablePlan(databaseTTLMap, timePartitionSlotMap)); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, e); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java index 993c87a4f3b1e..7282ce65e2cd9 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/storageengine/dataregion/DataRegionTest.java @@ -58,6 +58,7 @@ import org.apache.iotdb.db.storageengine.dataregion.compaction.selector.constant.InnerUnsequenceCompactionSelector; import org.apache.iotdb.db.storageengine.dataregion.compaction.utils.CompactionConfigRestorer; import org.apache.iotdb.db.storageengine.dataregion.flush.FlushManager; +import org.apache.iotdb.db.storageengine.dataregion.flush.TsFileFlushPolicy; import org.apache.iotdb.db.storageengine.dataregion.memtable.ReadOnlyMemChunk; import org.apache.iotdb.db.storageengine.dataregion.memtable.TsFileProcessor; import org.apache.iotdb.db.storageengine.dataregion.read.QueryDataSource; @@ -1463,6 +1464,269 @@ public void testDeleteDataInSeqFlushingMemtable() record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); dataRegion.insert(buildInsertRowNodeByTSRecord(record)); } + TsFileResource tsFileResource = + dataRegion.getTsFileManager().getTsFileList(true, databaseName).get(0); + TsFileProcessor tsFileProcessor = tsFileResource.getProcessor(); + tsFileProcessor.getFlushingMemTable().addLast(tsFileProcessor.getWorkMemTable()); + + MeasurementPath path = new MeasurementPath("root.vehicle.d0.s0"); + DeleteDataNode deleteDataNode1 = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 99); + deleteDataNode1.setSearchIndex(0); + MeasurementPath path2 = new MeasurementPath("root.vehicle.d200.s0"); + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path2), 50, 70); + deleteDataNode2.setSearchIndex(0); + // delete data which is not in flushing memtable + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode1); + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode2); + + DeleteDataNode deleteDataNode3 = + new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path), 50, 100); + deleteDataNode3.setSearchIndex(0); + DeleteDataNode deleteDataNode4 = + new DeleteDataNode(new PlanNodeId("4"), Collections.singletonList(path), 50, 150); + deleteDataNode4.setSearchIndex(0); + DeleteDataNode deleteDataNode5 = + new DeleteDataNode(new PlanNodeId("5"), Collections.singletonList(path), 100, 190); + deleteDataNode5.setSearchIndex(0); + // delete data which is in flushing memtable + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode3); + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode4); + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode4); + + dataRegion.syncCloseAllWorkingTsFileProcessors(); + assertTrue(tsFileResource.anyModFileExists()); + Assert.assertEquals(3, tsFileResource.getAllModEntries().size()); + } + + @Test + public void testDeleteDataInUnSeqFlushingMemtable() + throws IllegalPathException, WriteProcessException, IOException { + for (int j = 100; j < 200; j++) { + TSRecord record = new TSRecord(deviceId, j); + record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); + dataRegion.insert(buildInsertRowNodeByTSRecord(record)); + } + TsFileResource tsFileResource = + dataRegion.getTsFileManager().getTsFileList(true, databaseName).get(0); + + MeasurementPath path = new MeasurementPath("root.vehicle.d0.s0"); + DeleteDataNode deleteDataNode1 = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 99); + deleteDataNode1.setSearchIndex(0); + MeasurementPath path2 = new MeasurementPath("root.vehicle.d200.s0"); + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path2), 50, 70); + deleteDataNode2.setSearchIndex(0); + // delete data which is not in work memtable + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode1); + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode2); + + DeleteDataNode deleteDataNode3 = + new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path), 50, 100); + deleteDataNode3.setSearchIndex(0); + DeleteDataNode deleteDataNode4 = + new DeleteDataNode(new PlanNodeId("4"), Collections.singletonList(path), 50, 150); + deleteDataNode4.setSearchIndex(0); + DeleteDataNode deleteDataNode5 = + new DeleteDataNode(new PlanNodeId("5"), Collections.singletonList(path), 100, 190); + deleteDataNode5.setSearchIndex(0); + // delete data which is in work memtable + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode3); + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode4); + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode5); + + dataRegion.syncCloseAllWorkingTsFileProcessors(); + Assert.assertFalse(tsFileResource.anyModFileExists()); + + // insert unseq data points + for (int j = 50; j < 100; j++) { + TSRecord record = new TSRecord(deviceId, j); + record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); + dataRegion.insert(buildInsertRowNodeByTSRecord(record)); + } + + DeleteDataNode deleteDataNode6 = + new DeleteDataNode(new PlanNodeId("6"), Collections.singletonList(path), 200, 299); + deleteDataNode6.setSearchIndex(0); + // delete data which is not in work memtable + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode6); + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d200.s0"), deleteDataNode2); + + DeleteDataNode deleteDataNode7 = + new DeleteDataNode(new PlanNodeId("7"), Collections.singletonList(path), 80, 85); + deleteDataNode7.setSearchIndex(0); + // delete data which is in work memtable + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode7); + + Assert.assertFalse(tsFileResource.anyModFileExists()); + + tsFileResource = dataRegion.getTsFileManager().getTsFileList(false, databaseName).get(0); + TsFileProcessor tsFileProcessor = tsFileResource.getProcessor(); + tsFileProcessor.getFlushingMemTable().addLast(tsFileProcessor.getWorkMemTable()); + + DeleteDataNode deleteDataNode8 = + new DeleteDataNode(new PlanNodeId("8"), Collections.singletonList(path), 0, 49); + deleteDataNode8.setSearchIndex(0); + DeleteDataNode deleteDataNode9 = + new DeleteDataNode(new PlanNodeId("9"), Collections.singletonList(path), 100, 200); + deleteDataNode9.setSearchIndex(0); + // delete data which is not in flushing memtable + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode8); + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode9); + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d200.s0"), deleteDataNode2); + + DeleteDataNode deleteDataNode10 = + new DeleteDataNode(new PlanNodeId("10"), Collections.singletonList(path2), 25, 50); + deleteDataNode10.setSearchIndex(0); + DeleteDataNode deleteDataNode11 = + new DeleteDataNode(new PlanNodeId("11"), Collections.singletonList(path2), 50, 80); + deleteDataNode11.setSearchIndex(0); + DeleteDataNode deleteDataNode12 = + new DeleteDataNode(new PlanNodeId("12"), Collections.singletonList(path2), 99, 150); + deleteDataNode12.setSearchIndex(0); + // delete data which is in flushing memtable + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode10); + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode11); + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode12); + + dataRegion.syncCloseAllWorkingTsFileProcessors(); + assertTrue(tsFileResource.anyModFileExists()); + Assert.assertEquals(3, tsFileResource.getAllModEntries().size()); + } + + @Test + public void testDeleteDataInSeqWorkingMemtable() + throws IllegalPathException, WriteProcessException, IOException { + for (int j = 100; j < 200; j++) { + TSRecord record = new TSRecord("root.vehicle.d0", j); + record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); + dataRegion.insert(buildInsertRowNodeByTSRecord(record)); + } + for (int j = 100; j < 200; j++) { + TSRecord record = new TSRecord("root.vehicle.d199", j); + record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); + dataRegion.insert(buildInsertRowNodeByTSRecord(record)); + } + TsFileResource tsFileResource = + dataRegion.getTsFileManager().getTsFileList(true, databaseName).get(0); + + MeasurementPath path = new MeasurementPath("root.vehicle.d0.s0"); + DeleteDataNode deleteDataNode1 = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 99); + deleteDataNode1.setSearchIndex(0); + MeasurementPath path2 = new MeasurementPath("root.vehicle.d200.s0"); + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path2), 50, 70); + deleteDataNode2.setSearchIndex(0); + // delete data which is not in working memtable + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode1); + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d200.s0"), deleteDataNode2); + + MeasurementPath path3 = new MeasurementPath("root.vehicle.d199.*"); + DeleteDataNode deleteDataNode3 = + new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path3), 50, 500); + deleteDataNode3.setSearchIndex(0); + // delete data which is in working memtable + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d199.*"), deleteDataNode3); + + dataRegion.syncCloseAllWorkingTsFileProcessors(); + Assert.assertFalse(tsFileResource.anyModFileExists()); + Assert.assertFalse( + tsFileResource + .getDevices() + .contains(IDeviceID.Factory.DEFAULT_FACTORY.create("root.vehicle.d199"))); + } + + @Test + public void testFlushingEmptyMemtable() + throws IllegalPathException, WriteProcessException, IOException { + for (int j = 100; j < 200; j++) { + TSRecord record = new TSRecord(deviceId, j); + record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); + dataRegion.insert(buildInsertRowNodeByTSRecord(record)); + } + TsFileResource tsFileResource = + dataRegion.getTsFileManager().getTsFileList(true, databaseName).get(0); + + MeasurementPath path = new MeasurementPath("root.vehicle.d0.s0"); + DeleteDataNode deleteDataNode = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 100, 200); + deleteDataNode.setSearchIndex(0); + // delete all data which is in flushing memtable + dataRegion.deleteByDevice(new MeasurementPath("root.vehicle.d0.s0"), deleteDataNode); + + dataRegion.syncCloseAllWorkingTsFileProcessors(); + Assert.assertFalse(tsFileResource.getTsFile().exists()); + Assert.assertFalse(tsFileResource.anyModFileExists()); + Assert.assertFalse(dataRegion.getTsFileManager().contains(tsFileResource, true)); + Assert.assertFalse( + dataRegion.getWorkSequenceTsFileProcessors().contains(tsFileResource.getProcessor())); + } + + public static class DummyDataRegion extends DataRegion { + + public DummyDataRegion(String systemInfoDir, String storageGroupName) + throws DataRegionException { + super(systemInfoDir, "0", new TsFileFlushPolicy.DirectFlushPolicy(), storageGroupName); + } + } + + // -- test for deleting data directly + // -- delete data and file only when: + // 1. tsfile is closed + // 2. tsfile is not compating + // 3. tsfile's start time and end time must be a subinterval + // of the given time range. + + @Test + public void testDeleteDataDirectlySeqWriteModsOrDeleteFiles() + throws IllegalPathException, WriteProcessException, IOException { + for (int j = 100; j < 200; j++) { + TSRecord record = new TSRecord(deviceId, j); + record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); + dataRegion.insert(buildInsertRowNodeByTSRecord(record)); + } + + TsFileResource tsFileResource = + dataRegion.getTsFileManager().getTsFileList(true, databaseName).get(0); + // delete data in work mem, no mods. + MeasurementPath path = new MeasurementPath("root.vehicle.d0.**"); + DeleteDataNode deleteDataNode1 = + new DeleteDataNode(new PlanNodeId("1"), Collections.singletonList(path), 50, 100); + deleteDataNode1.setSearchIndex(0); + dataRegion.deleteDataDirectly(new MeasurementPath("root.vehicle.d0.**"), deleteDataNode1); + assertTrue(tsFileResource.getTsFile().exists()); + Assert.assertFalse(tsFileResource.anyModFileExists()); + + dataRegion.syncCloseAllWorkingTsFileProcessors(); + + // delete data in closed file, but time not match + DeleteDataNode deleteDataNode2 = + new DeleteDataNode(new PlanNodeId("2"), Collections.singletonList(path), 100, 120); + deleteDataNode2.setSearchIndex(0); + dataRegion.deleteDataDirectly(new MeasurementPath("root.vehicle.d0.**"), deleteDataNode2); + assertTrue(tsFileResource.getTsFile().exists()); + assertTrue(tsFileResource.anyModFileExists()); + + // delete data in closed file, and time all match + DeleteDataNode deleteDataNode3 = + new DeleteDataNode(new PlanNodeId("3"), Collections.singletonList(path), 100, 199); + deleteDataNode3.setSearchIndex(0); + dataRegion.deleteDataDirectly(new MeasurementPath("root.vehicle.d0.**"), deleteDataNode3); + Assert.assertFalse(tsFileResource.getTsFile().exists()); + Assert.assertFalse(tsFileResource.anyModFileExists()); + } + + @Test + public void testDeleteDataDirectlyUnseqWriteModsOrDeleteFiles() + throws IllegalPathException, WriteProcessException, IOException { + for (int j = 100; j < 200; j++) { + TSRecord record = new TSRecord(deviceId, j); + record.addTuple(DataPoint.getDataPoint(TSDataType.INT32, measurementId, String.valueOf(j))); + dataRegion.insert(buildInsertRowNodeByTSRecord(record)); + } TsFileResource tsFileResourceSeq = dataRegion.getTsFileManager().getTsFileList(true, databaseName).get(0); dataRegion.syncCloseAllWorkingTsFileProcessors(); From fc1b231790e1030b7aa0c6577b0562e958fef6b8 Mon Sep 17 00:00:00 2001 From: libo Date: Wed, 8 Apr 2026 15:14:03 +0800 Subject: [PATCH 06/15] Fix the problem that UT run failure, and revert previous logic, because only compare start time with TTL so that don't need the interval parameter. --- .../AutoCleanPartitionTablePlan.java | 19 +++++++------- .../persistence/partition/PartitionInfo.java | 3 +-- .../persistence/schema/ClusterSchemaInfo.java | 3 --- .../procedure/PartitionTableAutoCleaner.java | 10 +++----- .../thrift/ConfigNodeRPCServiceProcessor.java | 25 ++++++++----------- .../request/ConfigPhysicalPlanSerDeTest.java | 16 +++--------- .../queryengine/common/MPPQueryContext.java | 3 +++ .../utils/ThriftCommonsSerDeUtils.java | 22 ---------------- 8 files changed, 30 insertions(+), 71 deletions(-) diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/partition/AutoCleanPartitionTablePlan.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/partition/AutoCleanPartitionTablePlan.java index 9e5283f1ba6ed..1b1bfbe38f029 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/partition/AutoCleanPartitionTablePlan.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/consensus/request/write/partition/AutoCleanPartitionTablePlan.java @@ -35,25 +35,25 @@ public class AutoCleanPartitionTablePlan extends ConfigPhysicalPlan { Map databaseTTLMap; - Map currentTimeSlotMap; + TTimePartitionSlot currentTimeSlot; public AutoCleanPartitionTablePlan() { super(ConfigPhysicalPlanType.AutoCleanPartitionTable); } public AutoCleanPartitionTablePlan( - Map databaseTTLMap, Map currentTimeSlotMap) { + Map databaseTTLMap, TTimePartitionSlot currentTimeSlot) { this(); this.databaseTTLMap = databaseTTLMap; - this.currentTimeSlotMap = currentTimeSlotMap; + this.currentTimeSlot = currentTimeSlot; } public Map getDatabaseTTLMap() { return databaseTTLMap; } - public Map getCurrentTimeSlotMap() { - return currentTimeSlotMap; + public TTimePartitionSlot getCurrentTimeSlot() { + return currentTimeSlot; } @Override @@ -64,7 +64,7 @@ protected void serializeImpl(DataOutputStream stream) throws IOException { BasicStructureSerDeUtil.write(entry.getKey(), stream); stream.writeLong(entry.getValue()); } - ThriftCommonsSerDeUtils.serializeTTimePartitionSlot(currentTimeSlotMap, stream); + ThriftCommonsSerDeUtils.serializeTTimePartitionSlot(currentTimeSlot, stream); } @Override @@ -76,8 +76,7 @@ protected void deserializeImpl(ByteBuffer buffer) throws IOException { long value = buffer.getLong(); databaseTTLMap.put(key, value); } - currentTimeSlotMap = - ThriftCommonsSerDeUtils.deserializeTTimePartitionSlotMap(buffer, new TreeMap<>()); + currentTimeSlot = ThriftCommonsSerDeUtils.deserializeTTimePartitionSlot(buffer); } @Override @@ -90,11 +89,11 @@ public boolean equals(Object o) { } AutoCleanPartitionTablePlan that = (AutoCleanPartitionTablePlan) o; return Objects.equals(databaseTTLMap, that.databaseTTLMap) - && Objects.equals(currentTimeSlotMap, that.currentTimeSlotMap); + && Objects.equals(currentTimeSlot, that.currentTimeSlot); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), databaseTTLMap, currentTimeSlotMap); + return Objects.hash(super.hashCode(), databaseTTLMap, currentTimeSlot); } } 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 98ae75e22fe1f..b907527416bda 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,8 +521,7 @@ public TSStatus autoCleanPartitionTable(AutoCleanPartitionTablePlan plan) { if (isDatabaseExisted(database) && 0 < ttl && ttl < Long.MAX_VALUE) { databasePartitionTables .get(database) - .autoCleanPartitionTable( - ttl, plan.getCurrentTimeSlotMap().getOrDefault(database, null)); + .autoCleanPartitionTable(ttl, plan.getCurrentTimeSlot()); } }); 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 1417f5ae5e44a..6e55bfb02e924 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 @@ -286,9 +286,6 @@ 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); diff --git a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/PartitionTableAutoCleaner.java b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/PartitionTableAutoCleaner.java index c442307013dad..84f12a78ec46b 100644 --- a/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/PartitionTableAutoCleaner.java +++ b/iotdb-core/confignode/src/main/java/org/apache/iotdb/confignode/procedure/PartitionTableAutoCleaner.java @@ -60,7 +60,6 @@ public PartitionTableAutoCleaner(ConfigManager configManager) { protected void periodicExecute(Env env) { List databases = configManager.getClusterSchemaManager().getDatabaseNames(null); Map databaseTTLMap = new TreeMap<>(); - Map timePartitionSlotMap = new TreeMap<>(); for (String database : databases) { long databaseTTL; if (PathUtils.isTableModelDatabase(database)) { @@ -94,15 +93,12 @@ protected void periodicExecute(Env env) { LOGGER.info( "[PartitionTableCleaner] Periodically activate PartitionTableAutoCleaner for: {}", databaseTTLMap); - for (String database : databaseTTLMap.keySet()) { - // Only clean the partition table when necessary - TTimePartitionSlot currentTimePartitionSlot = getCurrentTimePartitionSlot(); - timePartitionSlotMap.put(database, currentTimePartitionSlot); - } + // Only clean the partition table when necessary + TTimePartitionSlot currentTimePartitionSlot = getCurrentTimePartitionSlot(); try { configManager .getConsensusManager() - .write(new AutoCleanPartitionTablePlan(databaseTTLMap, timePartitionSlotMap)); + .write(new AutoCleanPartitionTablePlan(databaseTTLMap, currentTimePartitionSlot)); } catch (ConsensusException e) { LOGGER.warn(CONSENSUS_WRITE_ERROR, 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 52f706fe99183..597844d717fa0 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 @@ -448,20 +448,17 @@ public TSStatus alterDatabase(final TDatabaseSchema databaseSchema) { "Failed to alter database. Doesn't support ALTER DataReplicationFactor 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 (databaseSchema.isSetTimePartitionOrigin()) { + errorResp = + new TSStatus(TSStatusCode.DATABASE_CONFIG_ERROR.getStatusCode()) + .setMessage("Failed to alter database. Doesn't support ALTER TimePartitionOrigin."); + } + + if (databaseSchema.isSetTimePartitionInterval()) { + errorResp = + new TSStatus(TSStatusCode.DATABASE_CONFIG_ERROR.getStatusCode()) + .setMessage("Failed to alter database. Doesn't support ALTER TimePartitionInterval."); + } if (errorResp != null) { LOGGER.warn("Execute AlterDatabase: {} with result: {}", databaseSchema, errorResp); 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 086a0bac931ab..6eb7ab0b5b91d 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,21 +486,11 @@ 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 db1CurrentTimeSlot = + TTimePartitionSlot currentTimeSlot = 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); + TimePartitionUtils.getTimePartitionSlot(System.currentTimeMillis(), "")); AutoCleanPartitionTablePlan req0 = - new AutoCleanPartitionTablePlan(databaseTTLMap, currentTimeSlotMap); + new AutoCleanPartitionTablePlan(databaseTTLMap, currentTimeSlot); AutoCleanPartitionTablePlan req1 = (AutoCleanPartitionTablePlan) ConfigPhysicalPlan.Factory.create(req0.serializeToByteBuffer()); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java index 88bd1998f683c..609405a5ea610 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java @@ -492,6 +492,9 @@ public void setNeedUpdateScanNumForLastQuery(boolean needUpdateScanNumForLastQue } public Optional getDatabaseName() { + if (session == null) { + return Optional.empty(); + } return session.getDatabaseName(); } 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 9a8cd3c5b429b..63dbb5158a58b 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 @@ -45,7 +45,6 @@ import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; -import java.util.Map; import static org.apache.iotdb.rpc.TConfigurationConst.defaultTConfiguration; @@ -227,16 +226,6 @@ public static TTimeSlotList deserializeTTimePartitionSlotList(ByteBuffer buffer) return timePartitionSlotList; } - public static void serializeTTimePartitionSlot( - Map timePartitionSlot, DataOutputStream stream) - throws IOException { - stream.writeInt(timePartitionSlot.size()); - for (Map.Entry entry : timePartitionSlot.entrySet()) { - BasicStructureSerDeUtil.write(entry.getKey(), stream); - serializeTTimePartitionSlot(entry.getValue(), stream); - } - } - public static void serializeTTimePartitionSlot( TTimePartitionSlot timePartitionSlot, DataOutputStream stream) throws IOException { try { @@ -256,17 +245,6 @@ public static TTimePartitionSlot deserializeTTimePartitionSlot(ByteBuffer buffer return timePartitionSlot; } - public static Map deserializeTTimePartitionSlotMap( - ByteBuffer buffer, Map map) { - int size = buffer.getInt(); - for (int i = 0; i < size; i++) { - String key = BasicStructureSerDeUtil.readString(buffer); - TTimePartitionSlot value = deserializeTTimePartitionSlot(buffer); - map.put(key, value); - } - return map; - } - public static void serializeTConsensusGroupId( TConsensusGroupId consensusGroupId, DataOutputStream stream) { try { From 9594dcc75da6d888ff903f78c620faafc9e900d5 Mon Sep 17 00:00:00 2001 From: libo Date: Fri, 24 Apr 2026 23:25:11 +0800 Subject: [PATCH 07/15] it response the "TimePartitionOrigin" field after "show database details" was executed. --- .../relational/it/schema/IoTDBDatabaseIT.java | 39 ++++++++--- .../persistence/schema/ClusterSchemaInfo.java | 47 ++++++++++---- .../schema/ClusterSchemaInfoTest.java | 33 +++++++++- ...formationSchemaContentSupplierFactory.java | 17 ++--- .../cache/partition/PartitionCache.java | 64 ++++++++++++++++--- .../config/TableConfigTaskVisitor.java | 5 ++ .../executor/ClusterConfigTaskExecutor.java | 2 + .../relational/AbstractDatabaseTask.java | 1 + .../metadata/relational/ShowDBTask.java | 7 +- .../table/InformationSchemaUtils.java | 1 + .../analyze/cache/PartitionCacheTest.java | 27 ++++++++ .../schema/column/ColumnHeaderConstant.java | 2 + .../schema/table/InformationSchema.java | 3 + .../commons/utils/TimePartitionUtils.java | 16 +++++ .../commons/utils/TimePartitionUtilsTest.java | 1 + 15 files changed, 221 insertions(+), 44 deletions(-) diff --git a/integration-test/src/test/java/org/apache/iotdb/relational/it/schema/IoTDBDatabaseIT.java b/integration-test/src/test/java/org/apache/iotdb/relational/it/schema/IoTDBDatabaseIT.java index 94c215a52ae21..e861ca7f06829 100644 --- a/integration-test/src/test/java/org/apache/iotdb/relational/it/schema/IoTDBDatabaseIT.java +++ b/integration-test/src/test/java/org/apache/iotdb/relational/it/schema/IoTDBDatabaseIT.java @@ -113,6 +113,7 @@ public void testManageDatabase() { String[] TTLs = new String[] {"INF"}; int[] schemaReplicaFactors = new int[] {1}; int[] dataReplicaFactors = new int[] {1}; + long[] timePartitionOrigin = new long[] {0L}; int[] timePartitionInterval = new int[] {604800000}; // show @@ -156,9 +157,10 @@ public void testManageDatabase() { assertEquals(TTLs[cnt], resultSet.getString(2)); assertEquals(schemaReplicaFactors[cnt], resultSet.getInt(3)); assertEquals(dataReplicaFactors[cnt], resultSet.getInt(4)); - assertEquals(timePartitionInterval[cnt], resultSet.getLong(5)); - assertEquals(schemaRegionGroupNum[cnt], resultSet.getInt(6)); - assertEquals(dataRegionGroupNum[cnt], resultSet.getInt(7)); + assertEquals(timePartitionOrigin[cnt], resultSet.getLong(5)); + assertEquals(timePartitionInterval[cnt], resultSet.getLong(6)); + assertEquals(schemaRegionGroupNum[cnt], resultSet.getInt(7)); + assertEquals(dataRegionGroupNum[cnt], resultSet.getInt(8)); cnt++; } assertEquals(databaseNames.length, cnt); @@ -196,9 +198,10 @@ public void testManageDatabase() { // Test create database with properties statement.execute( - "create database test_prop with (ttl=300, schema_region_group_num=DEFAULT, time_partition_interval=100000)"); + "create database test_prop with (ttl=300, schema_region_group_num=DEFAULT, time_partition_origin=2000, time_partition_interval=100000)"); databaseNames = new String[] {"test_prop"}; TTLs = new String[] {"300"}; + timePartitionOrigin = new long[] {2000L}; timePartitionInterval = new int[] {100000}; // show @@ -223,6 +226,23 @@ public void testManageDatabase() { assertEquals(databaseNames.length, cnt); } + try (final ResultSet resultSet = statement.executeQuery("SHOW DATABASES DETAILS")) { + int cnt = 0; + while (resultSet.next()) { + if (resultSet.getString(1).equals("information_schema")) { + continue; + } + assertEquals(databaseNames[cnt], resultSet.getString(1)); + assertEquals(TTLs[cnt], resultSet.getString(2)); + assertEquals(schemaReplicaFactors[cnt], resultSet.getInt(3)); + assertEquals(dataReplicaFactors[cnt], resultSet.getInt(4)); + assertEquals(timePartitionOrigin[cnt], resultSet.getLong(5)); + assertEquals(timePartitionInterval[cnt], resultSet.getLong(6)); + cnt++; + } + assertEquals(databaseNames.length, cnt); + } + try { statement.execute("create database test_prop_2 with (non_exist_prop=DEFAULT)"); fail( @@ -428,6 +448,7 @@ public void testInformationSchema() throws SQLException { "ttl(ms),STRING,ATTRIBUTE,", "schema_replication_factor,INT32,ATTRIBUTE,", "data_replication_factor,INT32,ATTRIBUTE,", + "time_partition_origin,INT64,ATTRIBUTE,", "time_partition_interval,INT64,ATTRIBUTE,", "schema_region_group_num,INT32,ATTRIBUTE,", "data_region_group_num,INT32,ATTRIBUTE,"))); @@ -637,11 +658,11 @@ public void testInformationSchema() throws SQLException { TestUtils.assertResultSetEqual( statement.executeQuery("select * from databases"), - "database,ttl(ms),schema_replication_factor,data_replication_factor,time_partition_interval,schema_region_group_num,data_region_group_num,", + "database,ttl(ms),schema_replication_factor,data_replication_factor,time_partition_origin,time_partition_interval,schema_region_group_num,data_region_group_num,", new HashSet<>( Arrays.asList( - "information_schema,INF,null,null,null,null,null,", - "test,INF,1,1,604800000,0,0,"))); + "information_schema,INF,null,null,null,null,null,null,", + "test,INF,1,1,0,604800000,0,0,"))); TestUtils.assertResultSetEqual( statement.executeQuery("show devices from tables where status = 'USING'"), "database,table_name,ttl(ms),status,comment,table_type,", @@ -844,8 +865,8 @@ public void testDBAuth() throws SQLException { Collections.singleton("information_schema,INF,null,null,null,")); TestUtils.assertResultSetEqual( userStmt.executeQuery("select * from information_schema.databases"), - "database,ttl(ms),schema_replication_factor,data_replication_factor,time_partition_interval,schema_region_group_num,data_region_group_num,", - Collections.singleton("information_schema,INF,null,null,null,null,null,")); + "database,ttl(ms),schema_replication_factor,data_replication_factor,time_partition_origin,time_partition_interval,schema_region_group_num,data_region_group_num,", + Collections.singleton("information_schema,INF,null,null,null,null,null,null,")); } try (final Connection adminCon = EnvFactory.getEnv().getConnection(BaseEnv.TABLE_SQL_DIALECT); 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 6e55bfb02e924..d490fda39a08f 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 @@ -198,10 +198,6 @@ 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); @@ -210,6 +206,7 @@ public TSStatus createDatabase(final DatabaseSchemaPlan plan) { .getDatabaseNodeByDatabasePath(partialPathName) .getAsMNode() .setDatabaseSchema(databaseSchema); + TimePartitionUtils.updateDatabaseTimePartitionConfig(databaseSchema.getName(), databaseSchema); result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode()); } catch (final MetadataException e) { @@ -473,11 +470,10 @@ public TSStatus setTimePartitionInterval(final SetTimePartitionIntervalPlan plan PathUtils.isTableModelDatabase(plan.getDatabase()) ? tableModelMTree : treeModelMTree; final PartialPath path = getQualifiedDatabasePartialPath(plan.getDatabase()); if (mTree.isDatabaseAlreadySet(path)) { - mTree - .getDatabaseNodeByDatabasePath(path) - .getAsMNode() - .getDatabaseSchema() - .setTimePartitionInterval(plan.getTimePartitionInterval()); + final TDatabaseSchema databaseSchema = + mTree.getDatabaseNodeByDatabasePath(path).getAsMNode().getDatabaseSchema(); + databaseSchema.setTimePartitionInterval(plan.getTimePartitionInterval()); + TimePartitionUtils.updateDatabaseTimePartitionConfig(plan.getDatabase(), databaseSchema); result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode()); } else { result.setCode(TSStatusCode.DATABASE_NOT_EXIST.getStatusCode()); @@ -499,11 +495,10 @@ public TSStatus setTimePartitionOrigin(final SetTimePartitionOriginPlan plan) { PathUtils.isTableModelDatabase(plan.getDatabase()) ? tableModelMTree : treeModelMTree; final PartialPath path = getQualifiedDatabasePartialPath(plan.getDatabase()); if (mTree.isDatabaseAlreadySet(path)) { - mTree - .getDatabaseNodeByDatabasePath(path) - .getAsMNode() - .getDatabaseSchema() - .setTimePartitionOrigin(plan.getTimePartitionOrigin()); + final TDatabaseSchema databaseSchema = + mTree.getDatabaseNodeByDatabasePath(path).getAsMNode().getDatabaseSchema(); + databaseSchema.setTimePartitionOrigin(plan.getTimePartitionOrigin()); + TimePartitionUtils.updateDatabaseTimePartitionConfig(plan.getDatabase(), databaseSchema); result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode()); } else { result.setCode(TSStatusCode.DATABASE_NOT_EXIST.getStatusCode()); @@ -835,6 +830,7 @@ public void processLoadSnapshot(final File snapshotDir) throws IOException { }); templateTable.processLoadSnapshot(snapshotDir); templatePreSetTable.processLoadSnapshot(snapshotDir); + rebuildTimePartitionUtilsCache(); } public void processMTreeLoadSnapshot( @@ -864,6 +860,28 @@ public interface SerDeFunction { void apply(final T stream) throws IOException; } + private void rebuildTimePartitionUtilsCache() { + databaseReadWriteLock.writeLock().lock(); + try { + TimePartitionUtils.clearDatabaseTimePartitionConfigCache(); + syncTimePartitionUtilsCache(treeModelMTree); + syncTimePartitionUtilsCache(tableModelMTree); + } catch (final MetadataException e) { + LOGGER.warn("Failed to rebuild time partition cache from database schema.", e); + } finally { + databaseReadWriteLock.writeLock().unlock(); + } + } + + private void syncTimePartitionUtilsCache(final ConfigMTree mTree) throws MetadataException { + for (final PartialPath databasePath : mTree.getAllDatabasePaths(null)) { + final TDatabaseSchema databaseSchema = + mTree.getDatabaseNodeByPath(databasePath).getAsMNode().getDatabaseSchema(); + TimePartitionUtils.updateDatabaseTimePartitionConfig( + databaseSchema.getName(), databaseSchema); + } + } + public Pair, Set> getNodesListInGivenLevel( PartialPath partialPath, int level, PathPatternTree scope) { Pair, Set> matchedPathsInNextLevel = @@ -1623,5 +1641,6 @@ public TSStatus commitAlterColumnDataType(AlterColumnDataTypePlan plan) { public void clear() { treeModelMTree.clear(); tableModelMTree.clear(); + TimePartitionUtils.clearDatabaseTimePartitionConfigCache(); } } diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfoTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfoTest.java index 1cf0b2913a832..3504bf06ab53b 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfoTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfoTest.java @@ -22,12 +22,15 @@ import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.schema.template.Template; +import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.commons.utils.PathUtils; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; import org.apache.iotdb.confignode.consensus.request.read.database.GetDatabasePlan; import org.apache.iotdb.confignode.consensus.request.read.template.GetPathsSetTemplatePlan; import org.apache.iotdb.confignode.consensus.request.read.template.GetTemplateSetInfoPlan; import org.apache.iotdb.confignode.consensus.request.write.database.DatabaseSchemaPlan; +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.template.CreateSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.template.PreSetSchemaTemplatePlan; import org.apache.iotdb.confignode.consensus.request.write.template.SetSchemaTemplatePlan; @@ -68,6 +71,7 @@ public class ClusterSchemaInfoTest { @Before public void setup() throws IOException { + TimePartitionUtils.clearDatabaseTimePartitionConfigCache(); clusterSchemaInfo = new ClusterSchemaInfo(); if (!snapshotDir.exists()) { snapshotDir.mkdirs(); @@ -77,6 +81,7 @@ public void setup() throws IOException { @After public void cleanup() throws IOException { clusterSchemaInfo.clear(); + TimePartitionUtils.clearDatabaseTimePartitionConfigCache(); if (snapshotDir.exists()) { FileUtils.deleteDirectory(snapshotDir); } @@ -91,12 +96,13 @@ public void testSnapshot() throws IOException, IllegalPathException { storageGroupPathList.add("root.a.a.a.b.sg"); Map testMap = new TreeMap<>(); - int i = 0; + int i = 1; for (String path : storageGroupPathList) { TDatabaseSchema tDatabaseSchema = new TDatabaseSchema(); tDatabaseSchema.setName(path); tDatabaseSchema.setDataReplicationFactor(i); tDatabaseSchema.setSchemaReplicationFactor(i); + tDatabaseSchema.setTimePartitionOrigin(i * 100L); tDatabaseSchema.setTimePartitionInterval(i); testMap.put(path, tDatabaseSchema); clusterSchemaInfo.createDatabase( @@ -120,6 +126,31 @@ public void testSnapshot() throws IOException, IllegalPathException { Map reloadResult = clusterSchemaInfo.getMatchedDatabaseSchemas(getStorageGroupReq).getSchemaMap(); Assert.assertEquals(testMap, reloadResult); + testMap.forEach( + (database, schema) -> { + Assert.assertEquals( + schema.getTimePartitionOrigin(), TimePartitionUtils.getTimePartitionOrigin(database)); + Assert.assertEquals( + schema.getTimePartitionInterval(), + TimePartitionUtils.getTimePartitionInterval(database)); + }); + } + + @Test + public void testTimePartitionConfigCacheConsistency() { + final TDatabaseSchema databaseSchema = + new TDatabaseSchema("root.sg").setTimePartitionOrigin(100L).setTimePartitionInterval(200L); + clusterSchemaInfo.createDatabase( + new DatabaseSchemaPlan(ConfigPhysicalPlanType.CreateDatabase, databaseSchema)); + + Assert.assertEquals(100L, TimePartitionUtils.getTimePartitionOrigin("root.sg")); + Assert.assertEquals(200L, TimePartitionUtils.getTimePartitionInterval("root.sg")); + + clusterSchemaInfo.setTimePartitionOrigin(new SetTimePartitionOriginPlan("root.sg", 300L)); + clusterSchemaInfo.setTimePartitionInterval(new SetTimePartitionIntervalPlan("root.sg", 400L)); + + Assert.assertEquals(300L, TimePartitionUtils.getTimePartitionOrigin("root.sg")); + Assert.assertEquals(400L, TimePartitionUtils.getTimePartitionInterval("root.sg")); } @Test diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java index d7162b03be316..2b98b9823c25b 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java @@ -337,15 +337,16 @@ protected void constructLine() { } else { columnBuilders[1].writeBinary( new Binary(String.valueOf(currentDatabase.getTTL()), TSFileConfig.STRING_CHARSET)); + } + columnBuilders[2].writeInt(currentDatabase.getSchemaReplicationFactor()); + columnBuilders[3].writeInt(currentDatabase.getDataReplicationFactor()); + columnBuilders[4].writeLong(currentDatabase.getTimePartitionOrigin()); + columnBuilders[5].writeLong(currentDatabase.getTimePartitionInterval()); + columnBuilders[6].writeInt(currentDatabase.getSchemaRegionNum()); + columnBuilders[7].writeInt(currentDatabase.getDataRegionNum()); + resultBuilder.declarePosition(); + currentDatabase = null; } - columnBuilders[2].writeInt(currentDatabase.getSchemaReplicationFactor()); - columnBuilders[3].writeInt(currentDatabase.getDataReplicationFactor()); - columnBuilders[4].writeLong(currentDatabase.getTimePartitionInterval()); - columnBuilders[5].writeInt(currentDatabase.getSchemaRegionNum()); - columnBuilders[6].writeInt(currentDatabase.getDataRegionNum()); - resultBuilder.declarePosition(); - currentDatabase = null; - } @Override public boolean hasNext() { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java index 330ff8b172f7a..3b82179cd2400 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java @@ -43,6 +43,7 @@ import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; import org.apache.iotdb.commons.schema.SchemaConstant; import org.apache.iotdb.commons.schema.table.Audit; +import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.commons.service.metric.PerformanceOverviewMetrics; import org.apache.iotdb.commons.utils.PathUtils; import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema; @@ -245,7 +246,7 @@ private void fetchDatabaseAndUpdateCache( if (databaseSchemaResp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { // update all database into cache - updateDatabaseCache(databaseSchemaResp.getDatabaseSchemaMap().keySet()); + updateDatabaseCache(databaseSchemaResp.getDatabaseSchemaMap()); getDatabaseMap(result, deviceIDs, true); } } @@ -266,7 +267,7 @@ private void fetchDatabaseAndUpdateCache() throws ClientManagerException, TExcep final TDatabaseSchemaResp databaseSchemaResp = client.getMatchedDatabaseSchemas(req); if (databaseSchemaResp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { // update all database into cache - updateDatabaseCache(databaseSchemaResp.getDatabaseSchemaMap().keySet()); + updateDatabaseCache(databaseSchemaResp.getDatabaseSchemaMap()); } } finally { databaseCacheLock.writeLock().unlock(); @@ -313,6 +314,8 @@ private void createDatabaseAndUpdateCache( // Try to create databases one by one until done or one database fail final Set successFullyCreatedDatabase = new HashSet<>(); + final Map successFullyCreatedDatabaseSchema = new HashMap<>(); + boolean needRefreshCacheFromConfigNode = false; for (final String databaseName : databaseNamesNeedCreated) { final long startTime = System.nanoTime(); try { @@ -332,9 +335,17 @@ private void createDatabaseAndUpdateCache( databaseSchema.setName(databaseName); databaseSchema.setIsTableModel(false); final TSStatus tsStatus = client.setDatabase(databaseSchema); - if (TSStatusCode.SUCCESS_STATUS.getStatusCode() == tsStatus.getCode() - || TSStatusCode.DATABASE_ALREADY_EXISTS.getStatusCode() == tsStatus.getCode()) { + if (TSStatusCode.SUCCESS_STATUS.getStatusCode() == tsStatus.getCode()) { successFullyCreatedDatabase.add(databaseName); + successFullyCreatedDatabaseSchema.put(databaseName, databaseSchema); + // In tree model, if the user creates a conflict database concurrently, for instance, + // the database created by user is root.db.ss.a, the auto-creation failed database is + // root.db, we wait till "getOrCreatePartition" to judge if the time series (like + // root.db.ss.a.e / root.db.ss.a) conflicts with the created database. just do not throw + // exception here. + } else if (TSStatusCode.DATABASE_ALREADY_EXISTS.getStatusCode() == tsStatus.getCode()) { + successFullyCreatedDatabase.add(databaseName); + needRefreshCacheFromConfigNode = true; // In tree model, if the user creates a conflict database concurrently, for instance, // the database created by user is root.db.ss.a, the auto-creation failed database is // root.db, we wait till "getOrCreatePartition" to judge if the time series (like @@ -342,7 +353,11 @@ private void createDatabaseAndUpdateCache( // exception here. } else if (TSStatusCode.DATABASE_CONFLICT.getStatusCode() != tsStatus.getCode()) { // Try to update cache by databases successfully created - updateDatabaseCache(successFullyCreatedDatabase); + if (needRefreshCacheFromConfigNode) { + fetchDatabaseAndUpdateCache(client, false); + } else { + updateDatabaseCache(successFullyCreatedDatabaseSchema); + } logger.warn( "[{} Cache] failed to create database {}", CacheMetrics.DATABASE_CACHE_NAME, @@ -351,7 +366,11 @@ private void createDatabaseAndUpdateCache( } } // Try to update database cache when all databases have already been created - updateDatabaseCache(successFullyCreatedDatabase); + if (needRefreshCacheFromConfigNode) { + fetchDatabaseAndUpdateCache(client, false); + } else { + updateDatabaseCache(successFullyCreatedDatabaseSchema); + } getDatabaseMap(result, deviceIDs, false); } } finally { @@ -383,10 +402,11 @@ private void createDatabaseAndUpdateCache(final String database, final String us databaseSchema.setName(database); databaseSchema.setIsTableModel(true); final TSStatus tsStatus = client.setDatabase(databaseSchema); - if (TSStatusCode.SUCCESS_STATUS.getStatusCode() == tsStatus.getCode() - || TSStatusCode.DATABASE_ALREADY_EXISTS.getStatusCode() == tsStatus.getCode()) { + if (TSStatusCode.SUCCESS_STATUS.getStatusCode() == tsStatus.getCode()) { // Try to update cache by databases successfully created - updateDatabaseCache(Collections.singleton(database)); + updateDatabaseCache(Collections.singletonMap(database, databaseSchema)); + } else if (TSStatusCode.DATABASE_ALREADY_EXISTS.getStatusCode() == tsStatus.getCode()) { + fetchDatabaseAndUpdateCache(client, true); } else { logger.warn( "[{} Cache] failed to create database {}", CacheMetrics.DATABASE_CACHE_NAME, database); @@ -546,16 +566,42 @@ public void updateDatabaseCache(final Set databaseNames) { } } + public void updateDatabaseCache(final Map databaseSchemaMap) { + if (databaseSchemaMap == null || databaseSchemaMap.isEmpty()) { + return; + } + databaseCacheLock.writeLock().lock(); + try { + databaseCache.addAll(databaseSchemaMap.keySet()); + TimePartitionUtils.updateDatabaseTimePartitionConfigs(databaseSchemaMap); + } finally { + databaseCacheLock.writeLock().unlock(); + } + } + /** invalidate all database cache */ public void removeFromDatabaseCache() { databaseCacheLock.writeLock().lock(); try { databaseCache.clear(); + TimePartitionUtils.clearDatabaseTimePartitionConfigCache(); } finally { databaseCacheLock.writeLock().unlock(); } } + private void fetchDatabaseAndUpdateCache(final ConfigNodeClient client, final boolean isTableModel) + throws TException { + final TGetDatabaseReq req = + new TGetDatabaseReq(ROOT_PATH, SchemaConstant.ALL_MATCH_SCOPE_BINARY) + .setIsTableModel(isTableModel) + .setCanSeeAuditDB(true); + final TDatabaseSchemaResp databaseSchemaResp = client.getMatchedDatabaseSchemas(req); + if (databaseSchemaResp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + updateDatabaseCache(databaseSchemaResp.getDatabaseSchemaMap()); + } + } + // endregion // region replicaSet cache diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java index c2f903398e2ab..c8db04d5e7193 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java @@ -281,6 +281,7 @@ import static org.apache.iotdb.commons.schema.table.TsTable.TTL_PROPERTY; import static org.apache.iotdb.db.queryengine.plan.execution.config.metadata.relational.CreateDBTask.DATA_REGION_GROUP_NUM_KEY; import static org.apache.iotdb.db.queryengine.plan.execution.config.metadata.relational.CreateDBTask.SCHEMA_REGION_GROUP_NUM_KEY; +import static org.apache.iotdb.db.queryengine.plan.execution.config.metadata.relational.CreateDBTask.TIME_PARTITION_ORIGIN_KEY; import static org.apache.iotdb.db.queryengine.plan.execution.config.metadata.relational.CreateDBTask.TIME_PARTITION_INTERVAL_KEY; import static org.apache.iotdb.db.queryengine.plan.execution.config.metadata.relational.CreateDBTask.TTL_KEY; import static org.apache.iotdb.db.queryengine.plan.relational.type.InternalTypeManager.getTSDataType; @@ -346,6 +347,7 @@ private IConfigTask visitDatabaseStatement( final String key = property.getName().getValue().toLowerCase(Locale.ENGLISH); if (property.isSetToDefault()) { switch (key) { + case TIME_PARTITION_ORIGIN_KEY: case TIME_PARTITION_INTERVAL_KEY: case SCHEMA_REGION_GROUP_NUM_KEY: case DATA_REGION_GROUP_NUM_KEY: @@ -378,6 +380,9 @@ private IConfigTask visitDatabaseStatement( } schema.setTTL(parseLongFromLiteral(value, TTL_KEY)); break; + case TIME_PARTITION_ORIGIN_KEY: + schema.setTimePartitionOrigin(parseLongFromLiteral(value, TIME_PARTITION_ORIGIN_KEY)); + break; case TIME_PARTITION_INTERVAL_KEY: schema.setTimePartitionInterval(parseLongFromLiteral(value, TIME_PARTITION_INTERVAL_KEY)); break; 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 376e434c62e01..b10a5e54640ae 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 @@ -476,6 +476,7 @@ public SettableFuture setDatabase( } } else { + TimePartitionUtils.updateDatabaseTimePartitionConfig(databaseSchema.getName(), databaseSchema); future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); } } catch (final ClientManagerException | TException e) { @@ -4217,6 +4218,7 @@ public SettableFuture createDatabase( final TSStatus tsStatus = configNodeClient.setDatabase(databaseSchema); if (TSStatusCode.SUCCESS_STATUS.getStatusCode() == tsStatus.getCode()) { + TimePartitionUtils.updateDatabaseTimePartitionConfig(databaseSchema.getName(), databaseSchema); future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); } else if (TSStatusCode.DATABASE_ALREADY_EXISTS.getStatusCode() == tsStatus.getCode()) { if (ifExists) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/relational/AbstractDatabaseTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/relational/AbstractDatabaseTask.java index df280c8759de2..9582f7f4809a9 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/relational/AbstractDatabaseTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/relational/AbstractDatabaseTask.java @@ -26,6 +26,7 @@ public abstract class AbstractDatabaseTask implements IConfigTask { /////////////////////////////// Allowed properties /////////////////////////////// public static final String TTL_KEY = "ttl"; + public static final String TIME_PARTITION_ORIGIN_KEY = "time_partition_origin"; public static final String TIME_PARTITION_INTERVAL_KEY = "time_partition_interval"; public static final String SCHEMA_REGION_GROUP_NUM_KEY = "schema_region_group_num"; public static final String DATA_REGION_GROUP_NUM_KEY = "data_region_group_num"; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/relational/ShowDBTask.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/relational/ShowDBTask.java index f5ab58d3ead4f..573b5560c0c68 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/relational/ShowDBTask.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/metadata/relational/ShowDBTask.java @@ -164,9 +164,10 @@ private static void buildTSBlockForDetails( } builder.getColumnBuilder(2).writeInt(storageGroupInfo.getSchemaReplicationFactor()); builder.getColumnBuilder(3).writeInt(storageGroupInfo.getDataReplicationFactor()); - builder.getColumnBuilder(4).writeLong(storageGroupInfo.getTimePartitionInterval()); - builder.getColumnBuilder(5).writeInt(storageGroupInfo.getSchemaRegionNum()); - builder.getColumnBuilder(6).writeInt(storageGroupInfo.getDataRegionNum()); + builder.getColumnBuilder(4).writeLong(storageGroupInfo.getTimePartitionOrigin()); + builder.getColumnBuilder(5).writeLong(storageGroupInfo.getTimePartitionInterval()); + builder.getColumnBuilder(6).writeInt(storageGroupInfo.getSchemaRegionNum()); + builder.getColumnBuilder(7).writeInt(storageGroupInfo.getDataRegionNum()); builder.declarePosition(); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/table/InformationSchemaUtils.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/table/InformationSchemaUtils.java index 337b0ffd78fe3..2822c91b6675c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/table/InformationSchemaUtils.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/schemaengine/table/InformationSchemaUtils.java @@ -75,6 +75,7 @@ public static void buildDatabaseTsBlock( if (details) { builder.getColumnBuilder(5).appendNull(); builder.getColumnBuilder(6).appendNull(); + builder.getColumnBuilder(7).appendNull(); } builder.declarePosition(); } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/PartitionCacheTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/PartitionCacheTest.java index e08a2b610c996..86b7c31a951a6 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/PartitionCacheTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/PartitionCacheTest.java @@ -25,9 +25,12 @@ import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot; import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; +import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.partition.DataPartition; import org.apache.iotdb.commons.partition.DataPartitionQueryParam; import org.apache.iotdb.commons.partition.SchemaPartition; +import org.apache.iotdb.commons.utils.TimePartitionUtils; +import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema; import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor; import org.apache.iotdb.db.auth.AuthorityChecker; import org.apache.iotdb.db.conf.IoTDBConfig; @@ -147,6 +150,7 @@ private static String getDeviceName(String storageGroupName, int deviceNumber) { @Before public void setUp() throws Exception { + TimePartitionUtils.clearDatabaseTimePartitionConfigCache(); partitionCache = new PartitionCache(); partitionCache.updateDatabaseCache(storageGroups); partitionCache.updateSchemaPartitionCache(schemaPartitionTable); @@ -157,6 +161,7 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { partitionCache.invalidAllCache(); + TimePartitionUtils.clearDatabaseTimePartitionConfigCache(); } @Test @@ -243,6 +248,28 @@ public void testStorageGroupCache() { assertEquals(0, deviceToStorageGroupMap.size()); } + @Test + public void testTimePartitionConfigCache() { + final TDatabaseSchema databaseSchema = new TDatabaseSchema(getDatabaseName(0)); + databaseSchema.setTimePartitionOrigin(123L); + databaseSchema.setTimePartitionInterval(456L); + + partitionCache.updateDatabaseCache( + Collections.singletonMap(databaseSchema.getName(), databaseSchema)); + + assertEquals(123L, TimePartitionUtils.getTimePartitionOrigin(databaseSchema.getName())); + assertEquals(456L, TimePartitionUtils.getTimePartitionInterval(databaseSchema.getName())); + + partitionCache.removeFromDatabaseCache(); + + assertEquals( + CommonDescriptor.getInstance().getConfig().getTimePartitionOrigin(), + TimePartitionUtils.getTimePartitionOrigin(databaseSchema.getName())); + assertEquals( + CommonDescriptor.getInstance().getConfig().getTimePartitionInterval(), + TimePartitionUtils.getTimePartitionInterval(databaseSchema.getName())); + } + @Test public void testRegionReplicaSetCache() { // test update regionReplicaSetCache with small timestamp diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java index f436165f9e8c3..7e39d734bc2b4 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/column/ColumnHeaderConstant.java @@ -246,6 +246,7 @@ private ColumnHeaderConstant() { public static final String SCHEMA_REPLICATION_FACTOR_TABLE_MODEL = "schema_replication_factor"; public static final String DATA_REPLICATION_FACTOR_TABLE_MODEL = "data_replication_factor"; + public static final String TIME_PARTITION_ORIGIN_TABLE_MODEL = "time_partition_origin"; public static final String TIME_PARTITION_INTERVAL_TABLE_MODEL = "time_partition_interval"; public static final String SCHEMA_REGION_GROUP_NUM_TABLE_MODEL = "schema_region_group_num"; public static final String DATA_REGION_GROUP_NUM_TABLE_MODEL = "data_region_group_num"; @@ -727,6 +728,7 @@ private ColumnHeaderConstant() { new ColumnHeader(COLUMN_TTL, TSDataType.TEXT), new ColumnHeader(SCHEMA_REPLICATION_FACTOR, TSDataType.INT32), new ColumnHeader(DATA_REPLICATION_FACTOR, TSDataType.INT32), + new ColumnHeader(TIME_PARTITION_ORIGIN, TSDataType.INT64), new ColumnHeader(TIME_PARTITION_INTERVAL, TSDataType.INT64), new ColumnHeader(SCHEMA_REGION_GROUP_NUM, TSDataType.INT32), new ColumnHeader(DATA_REGION_GROUP_NUM, TSDataType.INT32)); diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java index 98bf4a9a83023..a4721a2e808b9 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/schema/table/InformationSchema.java @@ -100,6 +100,9 @@ public class InformationSchema { databaseTable.addColumnSchema( new AttributeColumnSchema( ColumnHeaderConstant.DATA_REPLICATION_FACTOR_TABLE_MODEL, TSDataType.INT32)); + databaseTable.addColumnSchema( + new AttributeColumnSchema( + ColumnHeaderConstant.TIME_PARTITION_ORIGIN_TABLE_MODEL, TSDataType.INT64)); databaseTable.addColumnSchema( new AttributeColumnSchema( ColumnHeaderConstant.TIME_PARTITION_INTERVAL_TABLE_MODEL, TSDataType.INT64)); 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 9286fc4ad7321..a2dd8f9e9973b 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 @@ -170,11 +170,27 @@ public static void updateDatabaseTimePartitionConfig(String database, TDatabaseS databaseConfigCache.put(database, new DatabaseTimePartitionConfig(origin, interval)); } + public static void updateDatabaseTimePartitionConfigs(Map schemaMap) { + if (schemaMap == null || schemaMap.isEmpty()) { + return; + } + schemaMap.forEach( + (database, schema) -> { + if (database != null && schema != null) { + updateDatabaseTimePartitionConfig(database, schema); + } + }); + } + // Remove database-specific time partition configuration public static void removeDatabaseTimePartitionConfig(String database) { databaseConfigCache.remove(database); } + public static void clearDatabaseTimePartitionConfigCache() { + databaseConfigCache.clear(); + } + // Get database-specific configuration, fallback to global if not found private static DatabaseTimePartitionConfig getDatabaseConfig(String database) { if (database == 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 83b52b3bde8cd..fddc4faec9d06 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 @@ -40,6 +40,7 @@ public void setUp() { CommonDescriptor.getInstance() .getConfig() .setTimePartitionInterval(TEST_TIME_PARTITION_INTERVAL); + TimePartitionUtils.clearDatabaseTimePartitionConfigCache(); } @Test From b33725adc585edd45622d279d81aff80d86d6a8a Mon Sep 17 00:00:00 2001 From: libo Date: Wed, 20 May 2026 18:38:24 +0800 Subject: [PATCH 08/15] Support database-specific time partitions Defer tree insert time slot calculation until database resolution. Refresh database time partition config before data partition fetch and compute query slots per database. Expose database time partition metadata consistently in information schema. --- .../persistence/schema/ClusterSchemaInfo.java | 3 +- .../schema/ClusterSchemaInfoTest.java | 2 +- ...formationSchemaContentSupplierFactory.java | 18 +-- .../plan/analyze/AnalyzeUtils.java | 109 +++++++++++++++--- .../plan/analyze/AnalyzeVisitor.java | 61 +++++----- .../plan/analyze/ClusterPartitionFetcher.java | 50 +++++++- .../plan/analyze/IPartitionFetcher.java | 5 + .../plan/analyze/TemplatedAnalyze.java | 43 ++++--- .../cache/partition/PartitionCache.java | 6 +- .../config/TableConfigTaskVisitor.java | 2 +- .../executor/ClusterConfigTaskExecutor.java | 6 +- .../analyze/cache/PartitionCacheTest.java | 2 +- .../partition/DataPartitionQueryParam.java | 13 +++ .../commons/utils/TimePartitionUtils.java | 4 + .../commons/utils/TimePartitionUtilsTest.java | 17 +++ 15 files changed, 260 insertions(+), 81 deletions(-) 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 eb6fa0d73cb9e..20b9550595d3f 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 @@ -206,7 +206,8 @@ public TSStatus createDatabase(final DatabaseSchemaPlan plan) { .getDatabaseNodeByDatabasePath(partialPathName) .getAsMNode() .setDatabaseSchema(databaseSchema); - TimePartitionUtils.updateDatabaseTimePartitionConfig(databaseSchema.getName(), databaseSchema); + TimePartitionUtils.updateDatabaseTimePartitionConfig( + databaseSchema.getName(), databaseSchema); result.setCode(TSStatusCode.SUCCESS_STATUS.getStatusCode()); } catch (final MetadataException e) { diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfoTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfoTest.java index 3504bf06ab53b..466760ea66f71 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfoTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/schema/ClusterSchemaInfoTest.java @@ -22,8 +22,8 @@ import org.apache.iotdb.commons.exception.IllegalPathException; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.schema.template.Template; -import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.commons.utils.PathUtils; +import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.confignode.consensus.request.ConfigPhysicalPlanType; import org.apache.iotdb.confignode.consensus.request.read.database.GetDatabasePlan; import org.apache.iotdb.confignode.consensus.request.read.template.GetPathsSetTemplatePlan; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java index 0d597f7e96eef..79581f05800dd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/source/relational/InformationSchemaContentSupplierFactory.java @@ -338,16 +338,16 @@ protected void constructLine() { } else { columnBuilders[1].writeBinary( new Binary(String.valueOf(currentDatabase.getTTL()), TSFileConfig.STRING_CHARSET)); - } - columnBuilders[2].writeInt(currentDatabase.getSchemaReplicationFactor()); - columnBuilders[3].writeInt(currentDatabase.getDataReplicationFactor()); - columnBuilders[4].writeLong(currentDatabase.getTimePartitionOrigin()); - columnBuilders[5].writeLong(currentDatabase.getTimePartitionInterval()); - columnBuilders[6].writeInt(currentDatabase.getSchemaRegionNum()); - columnBuilders[7].writeInt(currentDatabase.getDataRegionNum()); - resultBuilder.declarePosition(); - currentDatabase = null; } + columnBuilders[2].writeInt(currentDatabase.getSchemaReplicationFactor()); + columnBuilders[3].writeInt(currentDatabase.getDataReplicationFactor()); + columnBuilders[4].writeLong(currentDatabase.getTimePartitionOrigin()); + columnBuilders[5].writeLong(currentDatabase.getTimePartitionInterval()); + columnBuilders[6].writeInt(currentDatabase.getSchemaRegionNum()); + columnBuilders[7].writeInt(currentDatabase.getDataRegionNum()); + resultBuilder.declarePosition(); + currentDatabase = null; + } @Override public boolean hasNext() { 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 c4b9319c90695..a1598c2c6cf92 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 @@ -48,6 +48,7 @@ import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertBaseStatement; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertMultiTabletsStatement; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertRowStatement; +import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertRowsOfOneDeviceStatement; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertRowsStatement; import org.apache.iotdb.db.queryengine.plan.statement.crud.InsertTabletStatement; import org.apache.iotdb.db.schemaengine.table.DataNodeTableCache; @@ -174,13 +175,43 @@ public static List computeTableDataPartitionParams( public static List computeTreeDataPartitionParams( InsertBaseStatement statement, MPPQueryContext context) { + final String database = getDatabaseName(statement, context); if (statement instanceof InsertTabletStatement) { DataPartitionQueryParam dataPartitionQueryParam = - getTreeDataPartitionQueryParam((InsertTabletStatement) statement, context); + getTreeDataPartitionQueryParam((InsertTabletStatement) statement, database); return Collections.singletonList(dataPartitionQueryParam); + } else if (statement instanceof InsertRowStatement) { + InsertRowStatement insertRowStatement = (InsertRowStatement) statement; + if (database == null) { + DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); + dataPartitionQueryParam.setDeviceID( + insertRowStatement.getDevicePath().getIDeviceIDAsFullDevice()); + dataPartitionQueryParam.setTimeList( + Collections.singletonList(insertRowStatement.getTime())); + return Collections.singletonList(dataPartitionQueryParam); + } + return computeDataPartitionParams( + Collections.singletonMap( + insertRowStatement.getDevicePath().getIDeviceIDAsFullDevice(), + Collections.singleton(insertRowStatement.getTimePartitionSlot(database))), + database); } else if (statement instanceof InsertMultiTabletsStatement) { InsertMultiTabletsStatement insertMultiTabletsStatement = (InsertMultiTabletsStatement) statement; + if (database == null) { + Map> dataPartitionQueryParamMap = new HashMap<>(); + for (InsertTabletStatement insertTabletStatement : + insertMultiTabletsStatement.getInsertTabletStatementList()) { + List timeList = + dataPartitionQueryParamMap.computeIfAbsent( + insertTabletStatement.getDevicePath().getIDeviceIDAsFullDevice(), + k -> new ArrayList<>()); + for (int i = 0; i < insertTabletStatement.getRowCount(); i++) { + timeList.add(insertTabletStatement.getTimes()[i]); + } + } + return computeDataPartitionParamsByTime(dataPartitionQueryParamMap); + } Map> dataPartitionQueryParamMap = new HashMap<>(); for (InsertTabletStatement insertTabletStatement : insertMultiTabletsStatement.getInsertTabletStatementList()) { @@ -188,13 +219,48 @@ public static List computeTreeDataPartitionParams( dataPartitionQueryParamMap.computeIfAbsent( insertTabletStatement.getDevicePath().getIDeviceIDAsFullDevice(), k -> new HashSet<>()); - timePartitionSlotSet.addAll( - insertTabletStatement.getTimePartitionSlots(getDatabaseName(statement, context))); + timePartitionSlotSet.addAll(insertTabletStatement.getTimePartitionSlots(database)); } - return computeDataPartitionParams( - dataPartitionQueryParamMap, getDatabaseName(statement, context)); + return computeDataPartitionParams(dataPartitionQueryParamMap, database); + } else if (statement instanceof InsertRowsOfOneDeviceStatement) { + final InsertRowsOfOneDeviceStatement insertRowsOfOneDeviceStatement = + (InsertRowsOfOneDeviceStatement) statement; + if (database == null) { + Map> dataPartitionQueryParamMap = new HashMap<>(); + List timeList = + dataPartitionQueryParamMap.computeIfAbsent( + insertRowsOfOneDeviceStatement.getDevicePath().getIDeviceIDAsFullDevice(), + k -> new ArrayList<>()); + for (InsertRowStatement insertRowStatement : + insertRowsOfOneDeviceStatement.getInsertRowStatementList()) { + timeList.add(insertRowStatement.getTime()); + } + return computeDataPartitionParamsByTime(dataPartitionQueryParamMap); + } + Map> dataPartitionQueryParamMap = new HashMap<>(); + Set timePartitionSlotSet = + dataPartitionQueryParamMap.computeIfAbsent( + insertRowsOfOneDeviceStatement.getDevicePath().getIDeviceIDAsFullDevice(), + k -> new HashSet<>()); + for (InsertRowStatement insertRowStatement : + insertRowsOfOneDeviceStatement.getInsertRowStatementList()) { + timePartitionSlotSet.add(insertRowStatement.getTimePartitionSlot(database)); + } + return computeDataPartitionParams(dataPartitionQueryParamMap, database); } else if (statement instanceof InsertRowsStatement) { final InsertRowsStatement insertRowsStatement = (InsertRowsStatement) statement; + if (database == null) { + Map> dataPartitionQueryParamMap = new HashMap<>(); + for (InsertRowStatement insertRowStatement : + insertRowsStatement.getInsertRowStatementList()) { + dataPartitionQueryParamMap + .computeIfAbsent( + insertRowStatement.getDevicePath().getIDeviceIDAsFullDevice(), + k -> new ArrayList<>()) + .add(insertRowStatement.getTime()); + } + return computeDataPartitionParamsByTime(dataPartitionQueryParamMap); + } Map> dataPartitionQueryParamMap = new HashMap<>(); for (InsertRowStatement insertRowStatement : insertRowsStatement.getInsertRowStatementList()) { @@ -202,22 +268,27 @@ public static List computeTreeDataPartitionParams( dataPartitionQueryParamMap.computeIfAbsent( insertRowStatement.getDevicePath().getIDeviceIDAsFullDevice(), k -> new HashSet<>()); - timePartitionSlotSet.add( - insertRowStatement.getTimePartitionSlot(getDatabaseName(statement, context))); + timePartitionSlotSet.add(insertRowStatement.getTimePartitionSlot(database)); } - return computeDataPartitionParams( - dataPartitionQueryParamMap, getDatabaseName(statement, context)); + return computeDataPartitionParams(dataPartitionQueryParamMap, database); } throw new UnsupportedOperationException("computeDataPartitionParams for " + statement); } private static DataPartitionQueryParam getTreeDataPartitionQueryParam( - InsertTabletStatement statement, MPPQueryContext context) { + InsertTabletStatement statement, String database) { DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); dataPartitionQueryParam.setDeviceID(statement.getDevicePath().getIDeviceIDAsFullDevice()); - dataPartitionQueryParam.setTimePartitionSlotList( - statement.getTimePartitionSlots(getDatabaseName(statement, context))); - dataPartitionQueryParam.setDatabaseName(getDatabaseName(statement, context)); + if (database == null) { + List timeList = new ArrayList<>(); + for (int i = 0; i < statement.getRowCount(); i++) { + timeList.add(statement.getTimes()[i]); + } + dataPartitionQueryParam.setTimeList(timeList); + } else { + dataPartitionQueryParam.setTimePartitionSlotList(statement.getTimePartitionSlots(database)); + dataPartitionQueryParam.setDatabaseName(database); + } return dataPartitionQueryParam; } @@ -240,6 +311,18 @@ public static List computeDataPartitionParams( return dataPartitionQueryParams; } + private static List computeDataPartitionParamsByTime( + Map> dataPartitionQueryParamMap) { + List dataPartitionQueryParams = new ArrayList<>(); + for (Map.Entry> entry : dataPartitionQueryParamMap.entrySet()) { + DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); + dataPartitionQueryParam.setDeviceID(entry.getKey()); + dataPartitionQueryParam.setTimeList(entry.getValue()); + dataPartitionQueryParams.add(dataPartitionQueryParam); + } + return dataPartitionQueryParams; + } + public static void validateSchema( IAnalysis analysis, InsertBaseStatement insertStatement, Runnable schemaValidation) { final long startTime = System.nanoTime(); 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 e8c9e52fc8ffe..93d8fa0af9396 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 @@ -2129,26 +2129,37 @@ private DataPartition fetchDataPartitionByDevices( Set deviceSet, ISchemaTree schemaTree, MPPQueryContext context) { long startTime = System.nanoTime(); try { - Pair, Pair> res = - 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( - Collections.emptyMap(), - CONFIG.getSeriesPartitionExecutorClass(), - CONFIG.getSeriesPartitionSlotNum()); - } - Map> sgNameToQueryParamsMap = new HashMap<>(); + Map> databaseToDevices = new HashMap<>(); for (IDeviceID deviceID : deviceSet) { - DataPartitionQueryParam queryParam = - new DataPartitionQueryParam(deviceID, res.left, res.right.left, res.right.right); - sgNameToQueryParamsMap + databaseToDevices .computeIfAbsent(schemaTree.getBelongedDatabase(deviceID), key -> new ArrayList<>()) - .add(queryParam); + .add(deviceID); } + partitionFetcher.ensureDatabaseTimePartitionConfig(databaseToDevices.keySet()); - if (res.right.left || res.right.right) { + boolean hasUnclosedTimeRange = false; + Map> sgNameToQueryParamsMap = new HashMap<>(); + for (Map.Entry> entry : databaseToDevices.entrySet()) { + String database = entry.getKey(); + Pair, Pair> res = + getTimePartitionSlotList(context.getGlobalTimeFilter(), context, database); + // there is no satisfied time range + if (res.left.isEmpty() && Boolean.FALSE.equals(res.right.left)) { + return new DataPartition( + Collections.emptyMap(), + CONFIG.getSeriesPartitionExecutorClass(), + CONFIG.getSeriesPartitionSlotNum()); + } + hasUnclosedTimeRange |= res.right.left || res.right.right; + for (IDeviceID deviceID : entry.getValue()) { + sgNameToQueryParamsMap + .computeIfAbsent(database, key -> new ArrayList<>()) + .add( + new DataPartitionQueryParam(deviceID, res.left, res.right.left, res.right.right)); + } + } + + if (hasUnclosedTimeRange) { return partitionFetcher.getDataPartitionWithUnclosedTimeRange(sgNameToQueryParamsMap); } else { return partitionFetcher.getDataPartition(sgNameToQueryParamsMap); @@ -2774,17 +2785,9 @@ public Analysis visitInsertRow(InsertRowStatement insertRowStatement, MPPQueryCo analysis.setRealStatement(realInsertStatement); if (realInsertStatement instanceof InsertRowStatement) { - InsertRowStatement realInsertRowStatement = (InsertRowStatement) realInsertStatement; - DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); - dataPartitionQueryParam.setDeviceID( - realInsertRowStatement.getDevicePath().getIDeviceIDAsFullDevice()); - dataPartitionQueryParam.setTimePartitionSlotList( - Collections.singletonList( - realInsertRowStatement.getTimePartitionSlot(context.getDatabaseName().orElse(null)))); - AnalyzeUtils.analyzeDataPartition( analysis, - Collections.singletonList(dataPartitionQueryParam), + AnalyzeUtils.computeTreeDataPartitionParams(realInsertStatement, context), context.getSession().getUserName(), partitionFetcher::getOrCreateDataPartition); } else { @@ -2865,15 +2868,9 @@ public Analysis visitInsertRowsOfOneDevice( analysis.setRealStatement(realInsertStatement); if (realInsertStatement instanceof InsertRowsOfOneDeviceStatement) { - InsertRowsOfOneDeviceStatement realStatement = - (InsertRowsOfOneDeviceStatement) realInsertStatement; - DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); - dataPartitionQueryParam.setDeviceID(realStatement.getDevicePath().getIDeviceIDAsFullDevice()); - dataPartitionQueryParam.setTimePartitionSlotList(realStatement.getTimePartitionSlots()); - AnalyzeUtils.analyzeDataPartition( analysis, - Collections.singletonList(dataPartitionQueryParam), + AnalyzeUtils.computeTreeDataPartitionParams(realInsertStatement, context), context.getSession().getUserName(), partitionFetcher::getOrCreateDataPartition); } else { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java index f93b9964c2056..06068a9e154a0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java @@ -33,8 +33,12 @@ import org.apache.iotdb.commons.partition.SchemaPartition; import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor; import org.apache.iotdb.commons.path.PathPatternTree; +import org.apache.iotdb.commons.schema.SchemaConstant; +import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq; import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionTableResp; +import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchemaResp; +import org.apache.iotdb.confignode.rpc.thrift.TGetDatabaseReq; import org.apache.iotdb.confignode.rpc.thrift.TSchemaNodeManagementReq; import org.apache.iotdb.confignode.rpc.thrift.TSchemaNodeManagementResp; import org.apache.iotdb.confignode.rpc.thrift.TSchemaPartitionReq; @@ -57,6 +61,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -316,6 +321,36 @@ public void invalidAllCache() { partitionCache.invalidAllCache(); } + @Override + public void ensureDatabaseTimePartitionConfig(Set databases) { + if (databases == null) { + return; + } + final Set missingDatabases = + databases.stream() + .filter(Objects::nonNull) + .filter(database -> !TimePartitionUtils.hasDatabaseTimePartitionConfig(database)) + .collect(Collectors.toSet()); + if (missingDatabases.isEmpty()) { + return; + } + try (final ConfigNodeClient client = + configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { + final TDatabaseSchemaResp databaseSchemaResp = + client.getMatchedDatabaseSchemas( + new TGetDatabaseReq( + Arrays.asList("root", "**"), SchemaConstant.ALL_MATCH_SCOPE_BINARY) + .setIsTableModel(false) + .setCanSeeAuditDB(true)); + if (databaseSchemaResp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + partitionCache.updateDatabaseCache(databaseSchemaResp.getDatabaseSchemaMap()); + } + } catch (final ClientManagerException | TException e) { + throw new StatementAnalyzeException( + "An error occurred when fetching database time partition config:" + e.getMessage()); + } + } + @Override public SchemaPartition getOrCreateSchemaPartition( final String database, final List deviceIDs, final String userName) { @@ -400,7 +435,18 @@ private Map> splitDataPartitionQueryParam( database = dataPartitionQueryParam.getDatabaseName(); } if (database != null) { - result.computeIfAbsent(database, key -> new ArrayList<>()).add(dataPartitionQueryParam); + final String finalDatabase = database; + if (dataPartitionQueryParam.hasTimeList()) { + dataPartitionQueryParam.setTimePartitionSlotList( + dataPartitionQueryParam.getTimeList().stream() + .map(time -> TimePartitionUtils.getTimePartitionSlot(time, finalDatabase)) + .distinct() + .collect(Collectors.toList())); + dataPartitionQueryParam.setDatabaseName(finalDatabase); + } + result + .computeIfAbsent(finalDatabase, key -> new ArrayList<>()) + .add(dataPartitionQueryParam); } } return result; @@ -487,11 +533,11 @@ private TDataPartitionReq constructDataPartitionReq( private TDataPartitionReq constructDataPartitionReqForQuery( final Map> sgNameToQueryParamsMap) { final Map> partitionSlotsMap = new HashMap<>(); - TTimeSlotList sharedTTimeSlotList = null; for (final Map.Entry> entry : sgNameToQueryParamsMap.entrySet()) { // for each sg final Map deviceToTimePartitionMap = new HashMap<>(); + TTimeSlotList sharedTTimeSlotList = null; for (final DataPartitionQueryParam queryParam : entry.getValue()) { if (sharedTTimeSlotList == null) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/IPartitionFetcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/IPartitionFetcher.java index 0549ec3964743..b13d66ba5c7cd 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/IPartitionFetcher.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/IPartitionFetcher.java @@ -32,6 +32,7 @@ import java.util.List; import java.util.Map; +import java.util.Set; public interface IPartitionFetcher { @@ -103,6 +104,10 @@ SchemaNodeManagementPartition getSchemaNodeManagementPartitionWithLevel( /** Invalid all partition cache */ void invalidAllCache(); + default void ensureDatabaseTimePartitionConfig(Set databases) { + // No-op for standalone and test implementations. + } + // ======================== Table Model Schema Partition Interface ======================== /** 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 9202ab6d56547..079aa7cc24220 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,26 +392,37 @@ private static DataPartition fetchDataPartitionByDevices( IPartitionFetcher partitionFetcher) { long startTime = System.nanoTime(); try { - String database = context.getDatabaseName().orElse(null); - Pair, Pair> res = - getTimePartitionSlotList(context.getGlobalTimeFilter(), context, database); - // there is no satisfied time range - if (res.left.isEmpty() && Boolean.FALSE.equals(res.right.left)) { - return new DataPartition( - Collections.emptyMap(), - CONFIG.getSeriesPartitionExecutorClass(), - CONFIG.getSeriesPartitionSlotNum()); - } - Map> sgNameToQueryParamsMap = new HashMap<>(); + Map> databaseToDevices = new HashMap<>(); for (IDeviceID deviceID : deviceSet) { - DataPartitionQueryParam queryParam = - new DataPartitionQueryParam(deviceID, res.left, res.right.left, res.right.right); - sgNameToQueryParamsMap + databaseToDevices .computeIfAbsent(schemaTree.getBelongedDatabase(deviceID), key -> new ArrayList<>()) - .add(queryParam); + .add(deviceID); + } + partitionFetcher.ensureDatabaseTimePartitionConfig(databaseToDevices.keySet()); + + boolean hasUnclosedTimeRange = false; + Map> sgNameToQueryParamsMap = new HashMap<>(); + for (Map.Entry> entry : databaseToDevices.entrySet()) { + String database = entry.getKey(); + Pair, Pair> res = + getTimePartitionSlotList(context.getGlobalTimeFilter(), context, database); + // there is no satisfied time range + if (res.left.isEmpty() && Boolean.FALSE.equals(res.right.left)) { + return new DataPartition( + Collections.emptyMap(), + CONFIG.getSeriesPartitionExecutorClass(), + CONFIG.getSeriesPartitionSlotNum()); + } + hasUnclosedTimeRange |= res.right.left || res.right.right; + for (IDeviceID deviceID : entry.getValue()) { + sgNameToQueryParamsMap + .computeIfAbsent(database, key -> new ArrayList<>()) + .add( + new DataPartitionQueryParam(deviceID, res.left, res.right.left, res.right.right)); + } } - if (res.right.left || res.right.right) { + if (hasUnclosedTimeRange) { return partitionFetcher.getDataPartitionWithUnclosedTimeRange(sgNameToQueryParamsMap); } else { return partitionFetcher.getDataPartition(sgNameToQueryParamsMap); diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java index 3b82179cd2400..c608a5019ecf1 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java @@ -43,9 +43,9 @@ import org.apache.iotdb.commons.pipe.config.constant.SystemConstant; import org.apache.iotdb.commons.schema.SchemaConstant; import org.apache.iotdb.commons.schema.table.Audit; -import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.commons.service.metric.PerformanceOverviewMetrics; import org.apache.iotdb.commons.utils.PathUtils; +import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema; import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchemaResp; import org.apache.iotdb.confignode.rpc.thrift.TGetDatabaseReq; @@ -590,8 +590,8 @@ public void removeFromDatabaseCache() { } } - private void fetchDatabaseAndUpdateCache(final ConfigNodeClient client, final boolean isTableModel) - throws TException { + private void fetchDatabaseAndUpdateCache( + final ConfigNodeClient client, final boolean isTableModel) throws TException { final TGetDatabaseReq req = new TGetDatabaseReq(ROOT_PATH, SchemaConstant.ALL_MATCH_SCOPE_BINARY) .setIsTableModel(isTableModel) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java index ff2d3d0e75f83..b337f9089ebf5 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/config/TableConfigTaskVisitor.java @@ -282,8 +282,8 @@ import static org.apache.iotdb.commons.schema.table.TsTable.TTL_PROPERTY; import static org.apache.iotdb.db.queryengine.plan.execution.config.metadata.relational.CreateDBTask.DATA_REGION_GROUP_NUM_KEY; import static org.apache.iotdb.db.queryengine.plan.execution.config.metadata.relational.CreateDBTask.SCHEMA_REGION_GROUP_NUM_KEY; -import static org.apache.iotdb.db.queryengine.plan.execution.config.metadata.relational.CreateDBTask.TIME_PARTITION_ORIGIN_KEY; import static org.apache.iotdb.db.queryengine.plan.execution.config.metadata.relational.CreateDBTask.TIME_PARTITION_INTERVAL_KEY; +import static org.apache.iotdb.db.queryengine.plan.execution.config.metadata.relational.CreateDBTask.TIME_PARTITION_ORIGIN_KEY; import static org.apache.iotdb.db.queryengine.plan.execution.config.metadata.relational.CreateDBTask.TTL_KEY; import static org.apache.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR; 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 d93ba0112fdb1..fa5aff2eb5bcd 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 @@ -477,7 +477,8 @@ public SettableFuture setDatabase( } } else { - TimePartitionUtils.updateDatabaseTimePartitionConfig(databaseSchema.getName(), databaseSchema); + TimePartitionUtils.updateDatabaseTimePartitionConfig( + databaseSchema.getName(), databaseSchema); future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); } } catch (final ClientManagerException | TException e) { @@ -4214,7 +4215,8 @@ public SettableFuture createDatabase( final TSStatus tsStatus = configNodeClient.setDatabase(databaseSchema); if (TSStatusCode.SUCCESS_STATUS.getStatusCode() == tsStatus.getCode()) { - TimePartitionUtils.updateDatabaseTimePartitionConfig(databaseSchema.getName(), databaseSchema); + TimePartitionUtils.updateDatabaseTimePartitionConfig( + databaseSchema.getName(), databaseSchema); future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); } else if (TSStatusCode.DATABASE_ALREADY_EXISTS.getStatusCode() == tsStatus.getCode()) { if (ifExists) { diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/PartitionCacheTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/PartitionCacheTest.java index 86b7c31a951a6..9374acc468ba8 100644 --- a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/PartitionCacheTest.java +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/PartitionCacheTest.java @@ -29,9 +29,9 @@ import org.apache.iotdb.commons.partition.DataPartition; import org.apache.iotdb.commons.partition.DataPartitionQueryParam; import org.apache.iotdb.commons.partition.SchemaPartition; +import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor; import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema; -import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor; import org.apache.iotdb.db.auth.AuthorityChecker; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; diff --git a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionQueryParam.java b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionQueryParam.java index 4282fefade1dc..0fc9266637534 100644 --- a/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionQueryParam.java +++ b/iotdb-core/node-commons/src/main/java/org/apache/iotdb/commons/partition/DataPartitionQueryParam.java @@ -31,6 +31,7 @@ public class DataPartitionQueryParam { private String databaseName; private IDeviceID deviceID; private List timePartitionSlotList = new ArrayList<>(); + private List timeList = new ArrayList<>(); // it will be set to true in query when there exist filter like: time <= XXX // (-oo, timePartitionSlotList.get(0)) @@ -75,6 +76,18 @@ public void setTimePartitionSlotList(List timePartitionSlotL this.timePartitionSlotList = timePartitionSlotList; } + public List getTimeList() { + return timeList; + } + + public void setTimeList(List timeList) { + this.timeList = timeList; + } + + public boolean hasTimeList() { + return timeList != null && !timeList.isEmpty(); + } + public boolean isNeedLeftAll() { return needLeftAll; } 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 a2dd8f9e9973b..81195de3b3fe4 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 @@ -191,6 +191,10 @@ public static void clearDatabaseTimePartitionConfigCache() { databaseConfigCache.clear(); } + public static boolean hasDatabaseTimePartitionConfig(String database) { + return database != null && databaseConfigCache.containsKey(database); + } + // Get database-specific configuration, fallback to global if not found private static DatabaseTimePartitionConfig getDatabaseConfig(String database) { if (database == 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 fddc4faec9d06..72ba28929320a 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 @@ -151,6 +151,23 @@ public void testDatabaseLevelTimePartitionFallbackToGlobal() { assertEquals(expectedSlot.getStartTime(), actualSlot.getStartTime()); } + @Test + public void testHasDatabaseTimePartitionConfig() { + TDatabaseSchema schema = new TDatabaseSchema(); + schema.setName("test.db"); + schema.setTimePartitionInterval(7200000L); + schema.setTimePartitionOrigin(2000L); + + Assert.assertFalse(TimePartitionUtils.hasDatabaseTimePartitionConfig(null)); + Assert.assertFalse(TimePartitionUtils.hasDatabaseTimePartitionConfig("test.db")); + + TimePartitionUtils.updateDatabaseTimePartitionConfig("test.db", schema); + Assert.assertTrue(TimePartitionUtils.hasDatabaseTimePartitionConfig("test.db")); + + TimePartitionUtils.removeDatabaseTimePartitionConfig("test.db"); + Assert.assertFalse(TimePartitionUtils.hasDatabaseTimePartitionConfig("test.db")); + } + @Test public void testRemoveDatabaseTimePartitionConfig() { // Create and update database configuration From d9de1a7646e62a855e7dc51ca5a50bf7410d7ff8 Mon Sep 17 00:00:00 2001 From: libo Date: Thu, 21 May 2026 13:09:21 +0800 Subject: [PATCH 09/15] Fix compatibility overloads for partition APIs --- .../load/splitter/TsFileSplitter.java | 5 ++ .../commons/utils/TimePartitionUtils.java | 81 +++++++++++++++++++ 2 files changed, 86 insertions(+) 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 d9633a3989760..38585a31dc5fa 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,6 +93,11 @@ public TsFileSplitter(File tsFile, TsFileDataConsumer consumer) { this.consumer = consumer; } + public void splitTsFileByDataPartition() + throws IOException, LoadFileException, IllegalStateException { + splitTsFileByDataPartition(null); + } + @SuppressWarnings({"squid:S3776", "squid:S6541"}) public void splitTsFileByDataPartition(String database) throws IOException, LoadFileException, IllegalStateException { 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 81195de3b3fe4..d15f0bfb1f8a1 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 @@ -214,22 +214,42 @@ public static TTimePartitionSlot getTimePartitionSlot(long time, String database return timePartitionSlot; } + public static TTimePartitionSlot getTimePartitionSlot(long time) { + return getTimePartitionSlot(time, null); + } + public static long getTimePartitionInterval(String database) { return getDatabaseConfig(database).getTimePartitionInterval(); } + public static long getTimePartitionInterval() { + return timePartitionInterval; + } + public static long getTimePartitionOrigin(String database) { return getDatabaseConfig(database).getTimePartitionOrigin(); } + public static long getTimePartitionOrigin() { + return timePartitionOrigin; + } + public static long getTimePartitionLowerBound(long time, String database) { return getTimePartitionLowerBoundInternal(time, getDatabaseConfig(database)); } + public static long getTimePartitionLowerBound(long time) { + return getTimePartitionLowerBound(time, null); + } + public static long getTimePartitionUpperBound(long time, String database) { return getTimePartitionUpperBoundInternal(time, getDatabaseConfig(database)); } + public static long getTimePartitionUpperBound(long time) { + return getTimePartitionUpperBound(time, null); + } + public static long getTimePartitionId(long time, String database) { DatabaseTimePartitionConfig config = getDatabaseConfig(database); time -= config.getTimePartitionOrigin(); @@ -238,11 +258,22 @@ public static long getTimePartitionId(long time, String database) { : time / config.getTimePartitionInterval() - 1; } + public static long getTimePartitionId(long time) { + time -= timePartitionOrigin; + return time > 0 || time % timePartitionInterval == 0 + ? time / timePartitionInterval + : time / timePartitionInterval - 1; + } + public static long getStartTimeByPartitionId(long partitionId, String database) { DatabaseTimePartitionConfig config = getDatabaseConfig(database); return (partitionId * config.getTimePartitionInterval()) + config.getTimePartitionOrigin(); } + public static long getStartTimeByPartitionId(long partitionId) { + return (partitionId * timePartitionInterval) + timePartitionOrigin; + } + public static boolean satisfyPartitionId( long startTime, long endTime, long partitionId, String database) { DatabaseTimePartitionConfig config = getDatabaseConfig(database); @@ -261,6 +292,20 @@ public static boolean satisfyPartitionId( return startPartition <= partitionId && endPartition >= partitionId; } + public static boolean satisfyPartitionId(long startTime, long endTime, long partitionId) { + long startPartition = + originMayCauseOverflow + ? getTimePartitionIdWithoutOverflow( + startTime, timePartitionOrigin, timePartitionInterval) + : getTimePartitionId(startTime); + long endPartition = + originMayCauseOverflow + ? getTimePartitionIdWithoutOverflow( + endTime, timePartitionOrigin, timePartitionInterval) + : getTimePartitionId(endTime); + return startPartition <= partitionId && endPartition >= partitionId; + } + public static boolean satisfyPartitionStartTime( Filter timeFilter, long partitionStartTime, String database) { if (timeFilter == null) { @@ -274,6 +319,17 @@ public static boolean satisfyPartitionStartTime( return timeFilter.satisfyStartEndTime(partitionStartTime, partitionEndTime); } + public static boolean satisfyPartitionStartTime(Filter timeFilter, long partitionStartTime) { + if (timeFilter == null) { + return true; + } + long partitionEndTime = + partitionStartTime >= timePartitionLowerBoundWithoutOverflow + ? Long.MAX_VALUE + : (partitionStartTime + timePartitionInterval - 1); + return timeFilter.satisfyStartEndTime(partitionStartTime, partitionEndTime); + } + public static boolean satisfyTimePartition(Filter timeFilter, long partitionId, String database) { DatabaseTimePartitionConfig config = getDatabaseConfig(database); long partitionStartTime; @@ -290,6 +346,20 @@ public static boolean satisfyTimePartition(Filter timeFilter, long partitionId, return satisfyPartitionStartTime(timeFilter, partitionStartTime, database); } + public static boolean satisfyTimePartition(Filter timeFilter, long partitionId) { + long partitionStartTime; + if (originMayCauseOverflow) { + partitionStartTime = + BigInteger.valueOf(partitionId) + .multiply(bigTimePartitionInterval) + .add(bigTimePartitionOrigin) + .longValue(); + } else { + partitionStartTime = partitionId * timePartitionInterval + timePartitionOrigin; + } + return satisfyPartitionStartTime(timeFilter, partitionStartTime); + } + public static long getEstimateTimePartitionSize(long startTime, long endTime, String database) { DatabaseTimePartitionConfig config = getDatabaseConfig(database); if (endTime > 0 && startTime < 0) { @@ -302,6 +372,17 @@ public static long getEstimateTimePartitionSize(long startTime, long endTime, St return (endTime - startTime) / config.getTimePartitionInterval() + 1; } + public static long getEstimateTimePartitionSize(long startTime, long endTime) { + if (endTime > 0 && startTime < 0) { + return BigInteger.valueOf(endTime) + .subtract(BigInteger.valueOf(startTime)) + .divide(bigTimePartitionInterval) + .longValue() + + 1; + } + return (endTime - startTime) / timePartitionInterval + 1; + } + // Helper methods for database-specific calculations private static long getTimePartitionLowerBoundInternal( long time, DatabaseTimePartitionConfig config) { From 05ea17da94b444dfa806f9d4f7c9d4ac9abccf29 Mon Sep 17 00:00:00 2001 From: libo Date: Thu, 21 May 2026 14:24:39 +0800 Subject: [PATCH 10/15] Fix compatibility overloads for partition APIs Add no-database overloads for time partition utilities and TsFileSplitter to preserve existing callers while supporting database-specific partition configuration. --- .../org/apache/iotdb/commons/utils/TimePartitionUtils.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 d15f0bfb1f8a1..6d9ef0a6ecb80 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 @@ -300,8 +300,7 @@ public static boolean satisfyPartitionId(long startTime, long endTime, long part : getTimePartitionId(startTime); long endPartition = originMayCauseOverflow - ? getTimePartitionIdWithoutOverflow( - endTime, timePartitionOrigin, timePartitionInterval) + ? getTimePartitionIdWithoutOverflow(endTime, timePartitionOrigin, timePartitionInterval) : getTimePartitionId(endTime); return startPartition <= partitionId && endPartition >= partitionId; } From b4078c7eaec2989f4e81ee8aac1188694c5b6f31 Mon Sep 17 00:00:00 2001 From: libo Date: Thu, 21 May 2026 16:33:34 +0800 Subject: [PATCH 11/15] Fix database-level time partition precision for writes Convert database time partition config from milliseconds to the current timestamp precision before caching it, and avoid double conversion during partition table cleanup. Also resolve tree-model write splitting with null session database by deriving the database from the fetched data partition, so slot calculation and region lookup stay consistent. --- .../planner/plan/node/write/InsertNode.java | 8 ++++++++ .../planner/plan/node/write/InsertRowNode.java | 8 +++++--- .../plan/node/write/InsertRowsNode.java | 12 +++++++----- .../node/write/InsertRowsOfOneDeviceNode.java | 11 +++++------ .../plan/node/write/InsertTabletNode.java | 11 +++++++---- .../node/write/RelationalInsertTabletNode.java | 6 ++++-- .../partition/SeriesPartitionTable.java | 7 +------ .../commons/utils/TimePartitionUtils.java | 11 ++++++++--- .../commons/utils/TimePartitionUtilsTest.java | 18 ++++++++++++++++++ 9 files changed, 63 insertions(+), 29 deletions(-) diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java index e4f60752d2d2e..7f8f0d3ed92d0 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java @@ -32,6 +32,7 @@ import org.apache.iotdb.db.exception.DataTypeInconsistentException; import org.apache.iotdb.db.i18n.DataNodeQueryMessages; import org.apache.iotdb.db.pipe.resource.memory.InsertNodeMemoryEstimator; +import org.apache.iotdb.db.queryengine.plan.analyze.IAnalysis; import org.apache.iotdb.db.queryengine.plan.analyze.cache.schema.DataNodeDevicePathCache; import org.apache.iotdb.db.storageengine.dataregion.memtable.AbstractMemTable; import org.apache.iotdb.db.storageengine.dataregion.memtable.DeviceIDFactory; @@ -228,6 +229,13 @@ public IDeviceID getDeviceID() { return deviceID; } + protected String getDatabaseName(final IAnalysis analysis, final IDeviceID deviceID) { + final String databaseName = analysis.getDatabaseName(); + return databaseName != null + ? databaseName + : analysis.getDataPartitionInfo().getDatabaseNameByDevice(deviceID); + } + public void setDeviceID(IDeviceID deviceID) { this.deviceID = deviceID; } 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 471392621e066..1fd662a173fb0 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 @@ -43,6 +43,7 @@ import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.exception.NotImplementedException; +import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.read.TimeValuePair; import org.apache.tsfile.utils.Binary; import org.apache.tsfile.utils.ReadWriteIOUtils; @@ -127,13 +128,14 @@ public InsertRowNode( @Override public List splitByPartition(IAnalysis analysis) { + final IDeviceID deviceID = getDeviceID(); + final String databaseName = getDatabaseName(analysis, deviceID); TTimePartitionSlot timePartitionSlot = - TimePartitionUtils.getTimePartitionSlot(time, analysis.getDatabaseName()); + TimePartitionUtils.getTimePartitionSlot(time, databaseName); this.dataRegionReplicaSet = analysis .getDataPartitionInfo() - .getDataRegionReplicaSetForWriting( - getDeviceID(), timePartitionSlot, analysis.getDatabaseName()); + .getDataRegionReplicaSetForWriting(deviceID, timePartitionSlot, databaseName); // collect redirectInfo analysis.setRedirectNodeList( Collections.singletonList( 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 0a62846981c7c..0759f3c53daf9 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 @@ -39,6 +39,7 @@ import org.apache.iotdb.db.storageengine.dataregion.wal.buffer.WALEntryValue; import org.apache.tsfile.exception.NotImplementedException; +import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.utils.ReadWriteIOUtils; import java.io.DataInputStream; @@ -273,15 +274,16 @@ public List splitByPartition(IAnalysis analysis) { for (int i = 0; i < insertRowNodeList.size(); i++) { InsertRowNode insertRowNode = insertRowNodeList.get(i); // Data region for insert row node - // each row may belong to different database, pass null for auto-detection + // Each row may belong to different database. + final IDeviceID deviceID = insertRowNode.targetPath.getIDeviceIDAsFullDevice(); + final String databaseName = getDatabaseName(analysis, deviceID); TRegionReplicaSet dataRegionReplicaSet = analysis .getDataPartitionInfo() .getDataRegionReplicaSetForWriting( - insertRowNode.targetPath.getIDeviceIDAsFullDevice(), - TimePartitionUtils.getTimePartitionSlot( - insertRowNode.getTime(), analysis.getDatabaseName()), - null); + deviceID, + TimePartitionUtils.getTimePartitionSlot(insertRowNode.getTime(), databaseName), + databaseName); // Collect redirectInfo redirectInfo.add(dataRegionReplicaSet.getDataNodeLocations().get(0).getClientRpcEndPoint()); InsertRowsNode tmpNode = splitMap.get(dataRegionReplicaSet); 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 4e45c3a9fee8d..bb011de0500d3 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 @@ -40,6 +40,7 @@ import org.apache.tsfile.enums.TSDataType; import org.apache.tsfile.exception.NotImplementedException; +import org.apache.tsfile.file.metadata.IDeviceID; import org.apache.tsfile.utils.ReadWriteIOUtils; import java.io.DataOutputStream; @@ -167,19 +168,17 @@ public List splitByPartition(IAnalysis analysis) { Map>> splitMap = new HashMap<>(); Map>> splitMapForIndex = new HashMap<>(); + final IDeviceID deviceID = targetPath.getIDeviceIDAsFullDevice(); + final String databaseName = getDatabaseName(analysis, deviceID); for (int i = 0; i < insertRowNodeList.size(); i++) { InsertRowNode insertRowNode = insertRowNodeList.get(i); TTimePartitionSlot timePartitionSlot = - TimePartitionUtils.getTimePartitionSlot( - insertRowNode.getTime(), analysis.getDatabaseName()); + TimePartitionUtils.getTimePartitionSlot(insertRowNode.getTime(), databaseName); TRegionReplicaSet dataRegionReplicaSet = analysis .getDataPartitionInfo() - .getDataRegionReplicaSetForWriting( - targetPath.getIDeviceIDAsFullDevice(), - timePartitionSlot, - analysis.getDatabaseName()); + .getDataRegionReplicaSetForWriting(deviceID, timePartitionSlot, databaseName); Map> tmpMap = splitMap.computeIfAbsent(dataRegionReplicaSet, k -> new HashMap<>()); Map> tmpIndexMap = 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 bb171e6de26d1..c699ff0aa1e65 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 @@ -237,10 +237,11 @@ public List splitByPartition(IAnalysis analysis) { return Collections.emptyList(); } + final String databaseName = getDatabaseName(analysis, getDeviceID(0)); final Map deviceIDSplitInfoMap = - collectSplitRanges(analysis.getDatabaseName()); + collectSplitRanges(databaseName); final Map> splitMap = - splitByReplicaSet(deviceIDSplitInfoMap, analysis); + splitByReplicaSet(deviceIDSplitInfoMap, analysis, databaseName); return doSplit(splitMap); } @@ -284,7 +285,9 @@ private Map collectSplitRanges(String database) { } protected Map> splitByReplicaSet( - Map deviceIDSplitInfoMap, IAnalysis analysis) { + Map deviceIDSplitInfoMap, + IAnalysis analysis, + String databaseName) { Map> splitMap = new HashMap<>(); for (Entry entry : deviceIDSplitInfoMap.entrySet()) { @@ -294,7 +297,7 @@ protected Map> splitByReplicaSet( analysis .getDataPartitionInfo() .getDataRegionReplicaSetForWriting( - deviceID, splitInfo.timePartitionSlots, analysis.getDatabaseName()); + deviceID, splitInfo.timePartitionSlots, databaseName); splitInfo.replicaSets = replicaSets; // collect redirectInfo analysis.addEndPointToRedirectNodeList( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertTabletNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertTabletNode.java index 8d24ad7736434..bf6927fafee38 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertTabletNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/RelationalInsertTabletNode.java @@ -201,7 +201,9 @@ protected InsertTabletNode getEmptySplit(int count) { } protected Map> splitByReplicaSet( - final Map deviceIDSplitInfoMap, final IAnalysis analysis) { + final Map deviceIDSplitInfoMap, + final IAnalysis analysis, + final String databaseName) { final Map> splitMap = new HashMap<>(); final Map endPointMap = new HashMap<>(); @@ -212,7 +214,7 @@ protected Map> splitByReplicaSet( analysis .getDataPartitionInfo() .getDataRegionReplicaSetForWriting( - deviceID, splitInfo.timePartitionSlots, analysis.getDatabaseName()); + deviceID, splitInfo.timePartitionSlots, databaseName); splitInfo.replicaSets = replicaSets; // collect redirectInfo endPointMap.put( 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 536352d68a1a0..d3efe2c6329e3 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 @@ -22,8 +22,6 @@ import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId; import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot; import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; -import org.apache.iotdb.commons.conf.CommonDescriptor; -import org.apache.iotdb.commons.utils.CommonDateTimeUtils; import org.apache.iotdb.commons.utils.ThriftCommonsSerDeUtils; import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.confignode.rpc.thrift.TTimeSlotList; @@ -255,10 +253,7 @@ public TConsensusGroupId getLastConsensusGroupId() { */ public List autoCleanPartitionTable( long TTL, TTimePartitionSlot currentTimeSlot, String database) { - final long timePartitionInterval = - CommonDateTimeUtils.convertMilliTimeWithPrecision( - TimePartitionUtils.getTimePartitionInterval(database), - CommonDescriptor.getInstance().getConfig().getTimestampPrecision()); + final long timePartitionInterval = TimePartitionUtils.getTimePartitionInterval(database); List removedTimePartitions = new ArrayList<>(); Iterator>> iterator = seriesPartitionMap.entrySet().iterator(); 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 6d9ef0a6ecb80..8f29791f6b4fa 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 @@ -161,12 +161,17 @@ public BigInteger getBigTimePartitionInterval() { // Update or add database-specific time partition configuration public static void updateDatabaseTimePartitionConfig(String database, TDatabaseSchema schema) { + String timestampPrecision = CommonDescriptor.getInstance().getConfig().getTimestampPrecision(); long interval = schema.isSetTimePartitionInterval() - ? schema.getTimePartitionInterval() - : timePartitionInterval; + ? CommonDateTimeUtils.convertMilliTimeWithPrecision( + schema.getTimePartitionInterval(), timestampPrecision) + : CommonDescriptor.getInstance().getConfig().getTimePartitionInterval(); long origin = - schema.isSetTimePartitionOrigin() ? schema.getTimePartitionOrigin() : timePartitionOrigin; + schema.isSetTimePartitionOrigin() + ? CommonDateTimeUtils.convertMilliTimeWithPrecision( + schema.getTimePartitionOrigin(), timestampPrecision) + : CommonDescriptor.getInstance().getConfig().getTimePartitionOrigin(); databaseConfigCache.put(database, new DatabaseTimePartitionConfig(origin, interval)); } 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 72ba28929320a..a3d89e4ba5aa4 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 @@ -36,6 +36,7 @@ public class TimePartitionUtilsTest { @Before public void setUp() { + CommonDescriptor.getInstance().getConfig().setTimestampPrecision("ms"); CommonDescriptor.getInstance().getConfig().setTimePartitionOrigin(TEST_TIME_PARTITION_ORIGIN); CommonDescriptor.getInstance() .getConfig() @@ -139,6 +140,23 @@ public void testDatabaseLevelTimePartition() { assertEquals(expectedSlot.getStartTime(), actualSlot.getStartTime()); } + @Test + public void testDatabaseLevelTimePartitionUsesTimestampPrecision() { + CommonDescriptor.getInstance().getConfig().setTimestampPrecision("ns"); + TDatabaseSchema schema = new TDatabaseSchema(); + schema.setName("test.db"); + schema.setTimePartitionInterval(7200000L); + schema.setTimePartitionOrigin(2000L); + + TimePartitionUtils.updateDatabaseTimePartitionConfig("test.db", schema); + + assertEquals(7200000_000_000L, TimePartitionUtils.getTimePartitionInterval("test.db")); + assertEquals(2000_000_000L, TimePartitionUtils.getTimePartitionOrigin("test.db")); + assertEquals( + 2000_000_000L, + TimePartitionUtils.getTimePartitionSlot(2000_000_000L + 1, "test.db").getStartTime()); + } + @Test public void testDatabaseLevelTimePartitionFallbackToGlobal() { // Test with database that doesn't have custom settings From f007ce7266ae69cc0b8accd84ed44caeae3a0f09 Mon Sep 17 00:00:00 2001 From: libo Date: Fri, 22 May 2026 14:25:05 +0800 Subject: [PATCH 12/15] Fix database time partition config cache --- .../CNPhysicalPlanGeneratorTest.java | 4 +- .../plan/analyze/AnalyzeUtils.java | 140 +++++++----------- .../plan/analyze/ClusterPartitionFetcher.java | 34 ++++- .../cache/partition/PartitionCache.java | 10 +- .../executor/ClusterConfigTaskExecutor.java | 40 ++++- .../planner/plan/node/write/InsertNode.java | 10 +- .../scheduler/load/LoadTsFileScheduler.java | 7 + .../node/write/WritePlanNodeSplitTest.java | 15 +- .../commons/utils/TimePartitionUtils.java | 18 ++- .../commons/utils/TimePartitionUtilsTest.java | 19 +++ 10 files changed, 173 insertions(+), 124 deletions(-) diff --git a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CNPhysicalPlanGeneratorTest.java b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CNPhysicalPlanGeneratorTest.java index b1b5f569f42a2..ed434715c520e 100644 --- a/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CNPhysicalPlanGeneratorTest.java +++ b/iotdb-core/confignode/src/test/java/org/apache/iotdb/confignode/persistence/CNPhysicalPlanGeneratorTest.java @@ -335,7 +335,7 @@ public void databaseWithoutTemplateGeneratorTest() throws Exception { tDatabaseSchema.setTTL(i + 1); tDatabaseSchema.setDataReplicationFactor(i); tDatabaseSchema.setSchemaReplicationFactor(i); - tDatabaseSchema.setTimePartitionInterval(i); + tDatabaseSchema.setTimePartitionInterval(i + 1); DatabaseSchemaPlan databaseSchemaPlan = new DatabaseSchemaPlan(ConfigPhysicalPlanType.CreateDatabase, tDatabaseSchema); clusterSchemaInfo.createDatabase(databaseSchemaPlan); @@ -453,7 +453,7 @@ public void templateAndDatabaseCompletedTest() throws Exception { tDatabaseSchema.setTTL(i + 1); tDatabaseSchema.setDataReplicationFactor(i); tDatabaseSchema.setSchemaReplicationFactor(i); - tDatabaseSchema.setTimePartitionInterval(i); + tDatabaseSchema.setTimePartitionInterval(i + 1); final DatabaseSchemaPlan databaseSchemaPlan = new DatabaseSchemaPlan(ConfigPhysicalPlanType.CreateDatabase, tDatabaseSchema); clusterSchemaInfo.createDatabase(databaseSchemaPlan); 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 c6d8e9a0e9206..4fc2d7d38a01c 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 @@ -133,43 +133,43 @@ public static List computeTableDataPartitionParams( String database = getDatabaseName(statement, context); if (statement instanceof InsertTabletStatement) { final InsertTabletStatement insertTabletStatement = (InsertTabletStatement) statement; - final Map> timePartitionSlotMap = new HashMap<>(); + final Map> dataPartitionQueryParamMap = new HashMap<>(); for (int i = 0; i < insertTabletStatement.getRowCount(); i++) { - timePartitionSlotMap - .computeIfAbsent(insertTabletStatement.getTableDeviceID(i), id -> new HashSet<>()) - .add(insertTabletStatement.getTimePartitionSlot(i, database)); + dataPartitionQueryParamMap + .computeIfAbsent(insertTabletStatement.getTableDeviceID(i), id -> new ArrayList<>()) + .add(insertTabletStatement.getTimes()[i]); } - return computeDataPartitionParams(timePartitionSlotMap, database); + return computeDataPartitionParamsByTime(dataPartitionQueryParamMap, database); } else if (statement instanceof InsertMultiTabletsStatement) { final InsertMultiTabletsStatement insertMultiTabletsStatement = (InsertMultiTabletsStatement) statement; - final Map> timePartitionSlotMap = new HashMap<>(); + final Map> dataPartitionQueryParamMap = new HashMap<>(); for (final InsertTabletStatement insertTabletStatement : insertMultiTabletsStatement.getInsertTabletStatementList()) { for (int i = 0; i < insertTabletStatement.getRowCount(); i++) { - timePartitionSlotMap - .computeIfAbsent(insertTabletStatement.getTableDeviceID(i), id -> new HashSet<>()) - .add(insertTabletStatement.getTimePartitionSlot(i, database)); + dataPartitionQueryParamMap + .computeIfAbsent(insertTabletStatement.getTableDeviceID(i), id -> new ArrayList<>()) + .add(insertTabletStatement.getTimes()[i]); } } - return computeDataPartitionParams(timePartitionSlotMap, database); + return computeDataPartitionParamsByTime(dataPartitionQueryParamMap, database); } else if (statement instanceof InsertRowStatement) { final InsertRowStatement insertRowStatement = (InsertRowStatement) statement; - return computeDataPartitionParams( + return computeDataPartitionParamsByTime( Collections.singletonMap( insertRowStatement.getTableDeviceID(), - Collections.singleton(insertRowStatement.getTimePartitionSlot(database))), + Collections.singletonList(insertRowStatement.getTime())), database); } else if (statement instanceof InsertRowsStatement) { final InsertRowsStatement insertRowsStatement = (InsertRowsStatement) statement; - final Map> timePartitionSlotMap = new HashMap<>(); + final Map> dataPartitionQueryParamMap = new HashMap<>(); for (final InsertRowStatement insertRowStatement : insertRowsStatement.getInsertRowStatementList()) { - timePartitionSlotMap - .computeIfAbsent(insertRowStatement.getTableDeviceID(), id -> new HashSet<>()) - .add(insertRowStatement.getTimePartitionSlot(database)); + dataPartitionQueryParamMap + .computeIfAbsent(insertRowStatement.getTableDeviceID(), id -> new ArrayList<>()) + .add(insertRowStatement.getTime()); } - return computeDataPartitionParams(timePartitionSlotMap, database); + return computeDataPartitionParamsByTime(dataPartitionQueryParamMap, database); } throw new UnsupportedOperationException( DataNodeQueryMessages.COMPUTEDATAPARTITIONPARAMS_FOR + statement); @@ -184,95 +184,51 @@ public static List computeTreeDataPartitionParams( return Collections.singletonList(dataPartitionQueryParam); } else if (statement instanceof InsertRowStatement) { InsertRowStatement insertRowStatement = (InsertRowStatement) statement; - if (database == null) { - DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); - dataPartitionQueryParam.setDeviceID( - insertRowStatement.getDevicePath().getIDeviceIDAsFullDevice()); - dataPartitionQueryParam.setTimeList( - Collections.singletonList(insertRowStatement.getTime())); - return Collections.singletonList(dataPartitionQueryParam); - } - return computeDataPartitionParams( + return computeDataPartitionParamsByTime( Collections.singletonMap( insertRowStatement.getDevicePath().getIDeviceIDAsFullDevice(), - Collections.singleton(insertRowStatement.getTimePartitionSlot(database))), + Collections.singletonList(insertRowStatement.getTime())), database); } else if (statement instanceof InsertMultiTabletsStatement) { InsertMultiTabletsStatement insertMultiTabletsStatement = (InsertMultiTabletsStatement) statement; - if (database == null) { - Map> dataPartitionQueryParamMap = new HashMap<>(); - for (InsertTabletStatement insertTabletStatement : - insertMultiTabletsStatement.getInsertTabletStatementList()) { - List timeList = - dataPartitionQueryParamMap.computeIfAbsent( - insertTabletStatement.getDevicePath().getIDeviceIDAsFullDevice(), - k -> new ArrayList<>()); - for (int i = 0; i < insertTabletStatement.getRowCount(); i++) { - timeList.add(insertTabletStatement.getTimes()[i]); - } - } - return computeDataPartitionParamsByTime(dataPartitionQueryParamMap); - } - Map> dataPartitionQueryParamMap = new HashMap<>(); + Map> dataPartitionQueryParamMap = new HashMap<>(); for (InsertTabletStatement insertTabletStatement : insertMultiTabletsStatement.getInsertTabletStatementList()) { - Set timePartitionSlotSet = + List timeList = dataPartitionQueryParamMap.computeIfAbsent( insertTabletStatement.getDevicePath().getIDeviceIDAsFullDevice(), - k -> new HashSet<>()); - timePartitionSlotSet.addAll(insertTabletStatement.getTimePartitionSlots(database)); + k -> new ArrayList<>()); + for (int i = 0; i < insertTabletStatement.getRowCount(); i++) { + timeList.add(insertTabletStatement.getTimes()[i]); + } } - return computeDataPartitionParams(dataPartitionQueryParamMap, database); + return computeDataPartitionParamsByTime(dataPartitionQueryParamMap, database); } else if (statement instanceof InsertRowsOfOneDeviceStatement) { final InsertRowsOfOneDeviceStatement insertRowsOfOneDeviceStatement = (InsertRowsOfOneDeviceStatement) statement; - if (database == null) { - Map> dataPartitionQueryParamMap = new HashMap<>(); - List timeList = - dataPartitionQueryParamMap.computeIfAbsent( - insertRowsOfOneDeviceStatement.getDevicePath().getIDeviceIDAsFullDevice(), - k -> new ArrayList<>()); - for (InsertRowStatement insertRowStatement : - insertRowsOfOneDeviceStatement.getInsertRowStatementList()) { - timeList.add(insertRowStatement.getTime()); - } - return computeDataPartitionParamsByTime(dataPartitionQueryParamMap); - } - Map> dataPartitionQueryParamMap = new HashMap<>(); - Set timePartitionSlotSet = + Map> dataPartitionQueryParamMap = new HashMap<>(); + List timeList = dataPartitionQueryParamMap.computeIfAbsent( insertRowsOfOneDeviceStatement.getDevicePath().getIDeviceIDAsFullDevice(), - k -> new HashSet<>()); + k -> new ArrayList<>()); for (InsertRowStatement insertRowStatement : insertRowsOfOneDeviceStatement.getInsertRowStatementList()) { - timePartitionSlotSet.add(insertRowStatement.getTimePartitionSlot(database)); + timeList.add(insertRowStatement.getTime()); } - return computeDataPartitionParams(dataPartitionQueryParamMap, database); + return computeDataPartitionParamsByTime(dataPartitionQueryParamMap, database); } else if (statement instanceof InsertRowsStatement) { final InsertRowsStatement insertRowsStatement = (InsertRowsStatement) statement; - if (database == null) { - Map> dataPartitionQueryParamMap = new HashMap<>(); - for (InsertRowStatement insertRowStatement : - insertRowsStatement.getInsertRowStatementList()) { - dataPartitionQueryParamMap - .computeIfAbsent( - insertRowStatement.getDevicePath().getIDeviceIDAsFullDevice(), - k -> new ArrayList<>()) - .add(insertRowStatement.getTime()); - } - return computeDataPartitionParamsByTime(dataPartitionQueryParamMap); - } - Map> dataPartitionQueryParamMap = new HashMap<>(); + Map> dataPartitionQueryParamMap = new HashMap<>(); for (InsertRowStatement insertRowStatement : insertRowsStatement.getInsertRowStatementList()) { - Set timePartitionSlotSet = - dataPartitionQueryParamMap.computeIfAbsent( + dataPartitionQueryParamMap + .computeIfAbsent( insertRowStatement.getDevicePath().getIDeviceIDAsFullDevice(), - k -> new HashSet<>()); - timePartitionSlotSet.add(insertRowStatement.getTimePartitionSlot(database)); + k -> new ArrayList<>()) + .add(insertRowStatement.getTime()); } - return computeDataPartitionParams(dataPartitionQueryParamMap, database); + return computeDataPartitionParamsByTime(dataPartitionQueryParamMap, database); } throw new UnsupportedOperationException( DataNodeQueryMessages.COMPUTEDATAPARTITIONPARAMS_FOR + statement); @@ -282,14 +238,12 @@ private static DataPartitionQueryParam getTreeDataPartitionQueryParam( InsertTabletStatement statement, String database) { DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); dataPartitionQueryParam.setDeviceID(statement.getDevicePath().getIDeviceIDAsFullDevice()); - if (database == null) { - List timeList = new ArrayList<>(); - for (int i = 0; i < statement.getRowCount(); i++) { - timeList.add(statement.getTimes()[i]); - } - dataPartitionQueryParam.setTimeList(timeList); - } else { - dataPartitionQueryParam.setTimePartitionSlotList(statement.getTimePartitionSlots(database)); + List timeList = new ArrayList<>(); + for (int i = 0; i < statement.getRowCount(); i++) { + timeList.add(statement.getTimes()[i]); + } + dataPartitionQueryParam.setTimeList(timeList); + if (database != null) { dataPartitionQueryParam.setDatabaseName(database); } return dataPartitionQueryParam; @@ -316,11 +270,19 @@ public static List computeDataPartitionParams( private static List computeDataPartitionParamsByTime( Map> dataPartitionQueryParamMap) { + return computeDataPartitionParamsByTime(dataPartitionQueryParamMap, null); + } + + private static List computeDataPartitionParamsByTime( + Map> dataPartitionQueryParamMap, String databaseName) { List dataPartitionQueryParams = new ArrayList<>(); for (Map.Entry> entry : dataPartitionQueryParamMap.entrySet()) { DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); dataPartitionQueryParam.setDeviceID(entry.getKey()); dataPartitionQueryParam.setTimeList(entry.getValue()); + if (databaseName != null) { + dataPartitionQueryParam.setDatabaseName(databaseName); + } dataPartitionQueryParams.add(dataPartitionQueryParam); } return dataPartitionQueryParams; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java index ae14ba6997d12..626e475e1fd75 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/ClusterPartitionFetcher.java @@ -34,6 +34,7 @@ import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor; import org.apache.iotdb.commons.path.PathPatternTree; import org.apache.iotdb.commons.schema.SchemaConstant; +import org.apache.iotdb.commons.utils.PathUtils; import org.apache.iotdb.commons.utils.TimePartitionUtils; import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionReq; import org.apache.iotdb.confignode.rpc.thrift.TDataPartitionTableResp; @@ -335,16 +336,17 @@ public void ensureDatabaseTimePartitionConfig(Set databases) { if (missingDatabases.isEmpty()) { return; } + final boolean needFetchTableModelDatabase = + missingDatabases.stream().anyMatch(PathUtils::isTableModelDatabase); + final boolean needFetchTreeModelDatabase = + missingDatabases.stream().anyMatch(database -> !PathUtils.isTableModelDatabase(database)); try (final ConfigNodeClient client = configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) { - final TDatabaseSchemaResp databaseSchemaResp = - client.getMatchedDatabaseSchemas( - new TGetDatabaseReq( - Arrays.asList("root", "**"), SchemaConstant.ALL_MATCH_SCOPE_BINARY) - .setIsTableModel(false) - .setCanSeeAuditDB(true)); - if (databaseSchemaResp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { - partitionCache.updateDatabaseCache(databaseSchemaResp.getDatabaseSchemaMap()); + if (needFetchTreeModelDatabase) { + updateDatabaseCache(client, false); + } + if (needFetchTableModelDatabase) { + updateDatabaseCache(client, true); } } catch (final ClientManagerException | TException e) { throw new StatementAnalyzeException( @@ -352,6 +354,18 @@ public void ensureDatabaseTimePartitionConfig(Set databases) { } } + private void updateDatabaseCache(final ConfigNodeClient client, final boolean isTableModel) + throws TException { + final TDatabaseSchemaResp databaseSchemaResp = + client.getMatchedDatabaseSchemas( + new TGetDatabaseReq(Arrays.asList("root", "**"), SchemaConstant.ALL_MATCH_SCOPE_BINARY) + .setIsTableModel(isTableModel) + .setCanSeeAuditDB(true)); + if (databaseSchemaResp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) { + partitionCache.updateDatabaseCache(databaseSchemaResp.getDatabaseSchemaMap()); + } + } + @Override public SchemaPartition getOrCreateSchemaPartition( final String database, final List deviceIDs, final String userName) { @@ -436,6 +450,10 @@ private Map> splitDataPartitionQueryParam( database = dataPartitionQueryParam.getDatabaseName(); } if (database != null) { + if (PathUtils.isTableModelDatabase(database)) { + partitionCache.checkAndAutoCreateDatabase(database, isAutoCreate, userName); + } + ensureDatabaseTimePartitionConfig(Collections.singleton(database)); final String finalDatabase = database; if (dataPartitionQueryParam.hasTimeList()) { dataPartitionQueryParam.setTimePartitionSlotList( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java index 83fd47f6902d7..b64a0bbf8ff38 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/analyze/cache/partition/PartitionCache.java @@ -366,8 +366,9 @@ private void createDatabaseAndUpdateCache( throw new IoTDBRuntimeException(tsStatus.message, tsStatus.code); } } - // Try to update database cache when all databases have already been created - if (needRefreshCacheFromConfigNode) { + // Fetch the completed schema from ConfigNode after auto-creation. The locally constructed + // schema may miss default time partition settings that ConfigNode fills in. + if (needRefreshCacheFromConfigNode || !successFullyCreatedDatabase.isEmpty()) { fetchDatabaseAndUpdateCache(client, false); } else { updateDatabaseCache(successFullyCreatedDatabaseSchema); @@ -404,8 +405,9 @@ private void createDatabaseAndUpdateCache(final String database, final String us databaseSchema.setIsTableModel(true); final TSStatus tsStatus = client.setDatabase(databaseSchema); if (TSStatusCode.SUCCESS_STATUS.getStatusCode() == tsStatus.getCode()) { - // Try to update cache by databases successfully created - updateDatabaseCache(Collections.singletonMap(database, databaseSchema)); + // Fetch the completed schema from ConfigNode after auto-creation. The locally constructed + // schema may miss default time partition settings that ConfigNode fills in. + fetchDatabaseAndUpdateCache(client, true); } else if (TSStatusCode.DATABASE_ALREADY_EXISTS.getStatusCode() == tsStatus.getCode()) { fetchDatabaseAndUpdateCache(client, true); } else { 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 0ef183fd33e16..598f33cfb18b5 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 @@ -119,6 +119,7 @@ import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRemoveReq; import org.apache.iotdb.confignode.rpc.thrift.TDataNodeRemoveResp; import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchema; +import org.apache.iotdb.confignode.rpc.thrift.TDatabaseSchemaResp; import org.apache.iotdb.confignode.rpc.thrift.TDeactivateSchemaTemplateReq; import org.apache.iotdb.confignode.rpc.thrift.TDeleteDatabasesReq; import org.apache.iotdb.confignode.rpc.thrift.TDeleteLogicalViewReq; @@ -470,16 +471,44 @@ public SettableFuture setDatabase( } } else { - TimePartitionUtils.updateDatabaseTimePartitionConfig( - databaseSchema.getName(), databaseSchema); + refreshDatabaseTimePartitionConfig(configNodeClient, databaseSchema); future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); } - } catch (final ClientManagerException | TException e) { + } catch (final IOException | ClientManagerException | TException e) { future.setException(e); } return future; } + private void refreshDatabaseTimePartitionConfig( + final ConfigNodeClient configNodeClient, final TDatabaseSchema databaseSchema) + throws IOException, TException { + TDatabaseSchema completeDatabaseSchema = databaseSchema; + if (!databaseSchema.isSetTimePartitionOrigin() + || !databaseSchema.isSetTimePartitionInterval()) { + final TGetDatabaseReq req = + new TGetDatabaseReq( + Arrays.asList(ROOT, MULTI_LEVEL_PATH_WILDCARD), ALL_MATCH_SCOPE.serialize()) + .setIsTableModel(databaseSchema.isSetIsTableModel() && databaseSchema.isIsTableModel()) + .setCanSeeAuditDB(true); + final TDatabaseSchemaResp resp = configNodeClient.getMatchedDatabaseSchemas(req); + if (resp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode() + && resp.getDatabaseSchemaMap() != null) { + completeDatabaseSchema = resp.getDatabaseSchemaMap().get(databaseSchema.getName()); + } + } + + if (completeDatabaseSchema == null) { + TimePartitionUtils.removeDatabaseTimePartitionConfig(databaseSchema.getName()); + LOGGER.warn( + "Failed to refresh time partition config for database {} after creation.", + databaseSchema.getName()); + return; + } + TimePartitionUtils.updateDatabaseTimePartitionConfig( + completeDatabaseSchema.getName(), completeDatabaseSchema); + } + @Override public SettableFuture alterDatabase( final DatabaseSchemaStatement databaseSchemaStatement) { @@ -4246,8 +4275,7 @@ public SettableFuture createDatabase( final TSStatus tsStatus = configNodeClient.setDatabase(databaseSchema); if (TSStatusCode.SUCCESS_STATUS.getStatusCode() == tsStatus.getCode()) { - TimePartitionUtils.updateDatabaseTimePartitionConfig( - databaseSchema.getName(), databaseSchema); + refreshDatabaseTimePartitionConfig(configNodeClient, databaseSchema); future.set(new ConfigTaskResult(TSStatusCode.SUCCESS_STATUS)); } else if (TSStatusCode.DATABASE_ALREADY_EXISTS.getStatusCode() == tsStatus.getCode()) { if (ifExists) { @@ -4261,7 +4289,7 @@ public SettableFuture createDatabase( } else { future.setException(new IoTDBException(tsStatus)); } - } catch (final ClientManagerException | TException e) { + } catch (final IOException | ClientManagerException | TException e) { future.setException(e); } return future; diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java index 7f8f0d3ed92d0..35d629d784582 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/planner/plan/node/write/InsertNode.java @@ -22,6 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet; import org.apache.iotdb.commons.consensus.index.ProgressIndex; import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.partition.DataPartition; import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNode; import org.apache.iotdb.commons.queryengine.plan.planner.plan.node.PlanNodeId; @@ -231,9 +232,12 @@ public IDeviceID getDeviceID() { protected String getDatabaseName(final IAnalysis analysis, final IDeviceID deviceID) { final String databaseName = analysis.getDatabaseName(); - return databaseName != null - ? databaseName - : analysis.getDataPartitionInfo().getDatabaseNameByDevice(deviceID); + final DataPartition dataPartitionInfo = analysis.getDataPartitionInfo(); + final String databaseNameInPartition = + dataPartitionInfo == null || dataPartitionInfo.isEmpty() + ? null + : dataPartitionInfo.getDatabaseNameByDevice(deviceID); + return databaseNameInPartition != null ? databaseNameInPartition : databaseName; } public void setDeviceID(IDeviceID deviceID) { 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 4da1a0249f80c..c69fd193488bf 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 @@ -182,6 +182,7 @@ public void start() { final String filePath = node.getTsFileResource().getTsFilePath(); partitionFetcher.setDatabase(getPartitionQueryDatabase(node, isGeneratedByPipe)); + partitionFetcher.ensureDatabaseTimePartitionConfig(); boolean isLoadSingleTsFileSuccess = true; boolean shouldRemoveFileFromLoadingSet = false; @@ -862,6 +863,12 @@ public void setDatabase(String database) { this.database = database; } + public void ensureDatabaseTimePartitionConfig() { + if (database != null) { + fetcher.ensureDatabaseTimePartitionConfig(Collections.singleton(database)); + } + } + public List queryDataPartition( List> slotList, String userName) { List replicaSets = new ArrayList<>(); 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 767f6bf7c0087..25c9757b27afd 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 @@ -163,6 +163,11 @@ private int getRegionIdByTime(long startTime, String database) { return (int) (4 - ((startTime - 1) / interval)); } + private String getDatabaseName(PartialPath devicePath) { + String fullPath = devicePath.getFullPath(); + return fullPath.substring(0, fullPath.indexOf(".", fullPath.indexOf(".") + 1)); + } + protected DataPartition getDataPartition( List dataPartitionQueryParamList) { Map>>> @@ -340,10 +345,8 @@ public void testInsertMultiTablets() throws IllegalPathException { DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); dataPartitionQueryParam.setDeviceID( insertTabletNode.getTargetPath().getIDeviceIDAsFullDevice()); - String fullPath = insertTabletNode.getTargetPath().getFullPath(); dataPartitionQueryParam.setTimePartitionSlotList( - insertTabletNode.getTimePartitionSlots( - fullPath.substring(0, fullPath.indexOf(".", fullPath.indexOf(".") + 1)))); + insertTabletNode.getTimePartitionSlots(getDatabaseName(insertTabletNode.getTargetPath()))); dataPartitionQueryParams.add(dataPartitionQueryParam); } @@ -358,6 +361,10 @@ public void testInsertMultiTablets() throws IllegalPathException { @Test public void testInsertRowsNode() throws IllegalPathException { + TDatabaseSchema databaseSchema = new TDatabaseSchema(); + databaseSchema.setTimePartitionInterval(150); + TimePartitionUtils.updateDatabaseTimePartitionConfig("root.sg2", databaseSchema); + initDataPartitionMap(); InsertRowsNode insertRowsNode = new InsertRowsNode(new PlanNodeId("plan node 3")); @@ -379,7 +386,7 @@ public void testInsertRowsNode() throws IllegalPathException { DataPartitionQueryParam dataPartitionQueryParam = new DataPartitionQueryParam(); dataPartitionQueryParam.setDeviceID(insertRowNode.getTargetPath().getIDeviceIDAsFullDevice()); dataPartitionQueryParam.setTimePartitionSlotList( - insertRowNode.getTimePartitionSlots("root.sg2")); + insertRowNode.getTimePartitionSlots(getDatabaseName(insertRowNode.getTargetPath()))); dataPartitionQueryParams.add(dataPartitionQueryParam); } 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 8f29791f6b4fa..9ed1b8ffab6fd 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 @@ -163,15 +163,17 @@ public BigInteger getBigTimePartitionInterval() { public static void updateDatabaseTimePartitionConfig(String database, TDatabaseSchema schema) { String timestampPrecision = CommonDescriptor.getInstance().getConfig().getTimestampPrecision(); long interval = - schema.isSetTimePartitionInterval() - ? CommonDateTimeUtils.convertMilliTimeWithPrecision( - schema.getTimePartitionInterval(), timestampPrecision) - : CommonDescriptor.getInstance().getConfig().getTimePartitionInterval(); + CommonDateTimeUtils.convertMilliTimeWithPrecision( + schema.isSetTimePartitionInterval() + ? schema.getTimePartitionInterval() + : CommonDescriptor.getInstance().getConfig().getTimePartitionInterval(), + timestampPrecision); long origin = - schema.isSetTimePartitionOrigin() - ? CommonDateTimeUtils.convertMilliTimeWithPrecision( - schema.getTimePartitionOrigin(), timestampPrecision) - : CommonDescriptor.getInstance().getConfig().getTimePartitionOrigin(); + CommonDateTimeUtils.convertMilliTimeWithPrecision( + schema.isSetTimePartitionOrigin() + ? schema.getTimePartitionOrigin() + : CommonDescriptor.getInstance().getConfig().getTimePartitionOrigin(), + timestampPrecision); databaseConfigCache.put(database, new DatabaseTimePartitionConfig(origin, interval)); } 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 a3d89e4ba5aa4..19775a5ad6099 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 @@ -157,6 +157,25 @@ public void testDatabaseLevelTimePartitionUsesTimestampPrecision() { TimePartitionUtils.getTimePartitionSlot(2000_000_000L + 1, "test.db").getStartTime()); } + @Test + public void testDatabaseLevelTimePartitionDefaultUsesTimestampPrecision() { + CommonDescriptor.getInstance().getConfig().setTimestampPrecision("us"); + TDatabaseSchema schema = new TDatabaseSchema(); + schema.setName("test.db"); + + TimePartitionUtils.updateDatabaseTimePartitionConfig("test.db", schema); + + assertEquals( + TEST_TIME_PARTITION_INTERVAL * 1000L, + TimePartitionUtils.getTimePartitionInterval("test.db")); + assertEquals( + TEST_TIME_PARTITION_ORIGIN * 1000L, TimePartitionUtils.getTimePartitionOrigin("test.db")); + assertEquals( + TEST_TIME_PARTITION_ORIGIN * 1000L, + TimePartitionUtils.getTimePartitionSlot(TEST_TIME_PARTITION_ORIGIN * 1000L + 1, "test.db") + .getStartTime()); + } + @Test public void testDatabaseLevelTimePartitionFallbackToGlobal() { // Test with database that doesn't have custom settings From 01a2bcba40d96be88f91b0ef6584209158788cef Mon Sep 17 00:00:00 2001 From: libo Date: Fri, 22 May 2026 16:20:02 +0800 Subject: [PATCH 13/15] Format database partition changes --- .../execution/config/executor/ClusterConfigTaskExecutor.java | 3 ++- .../plan/planner/node/write/WritePlanNodeSplitTest.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) 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 598f33cfb18b5..1d619c17c29ef 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 @@ -489,7 +489,8 @@ private void refreshDatabaseTimePartitionConfig( final TGetDatabaseReq req = new TGetDatabaseReq( Arrays.asList(ROOT, MULTI_LEVEL_PATH_WILDCARD), ALL_MATCH_SCOPE.serialize()) - .setIsTableModel(databaseSchema.isSetIsTableModel() && databaseSchema.isIsTableModel()) + .setIsTableModel( + databaseSchema.isSetIsTableModel() && databaseSchema.isIsTableModel()) .setCanSeeAuditDB(true); final TDatabaseSchemaResp resp = configNodeClient.getMatchedDatabaseSchemas(req); if (resp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode() 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 25c9757b27afd..0142257ff4efd 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 @@ -346,7 +346,8 @@ public void testInsertMultiTablets() throws IllegalPathException { dataPartitionQueryParam.setDeviceID( insertTabletNode.getTargetPath().getIDeviceIDAsFullDevice()); dataPartitionQueryParam.setTimePartitionSlotList( - insertTabletNode.getTimePartitionSlots(getDatabaseName(insertTabletNode.getTargetPath()))); + insertTabletNode.getTimePartitionSlots( + getDatabaseName(insertTabletNode.getTargetPath()))); dataPartitionQueryParams.add(dataPartitionQueryParam); } From fe299072686d6b58225d76c812cbda3516fcbad3 Mon Sep 17 00:00:00 2001 From: libo Date: Fri, 22 May 2026 17:31:39 +0800 Subject: [PATCH 14/15] Fix load aligned chunk split serialization --- .../iotdb/db/storageengine/load/splitter/TsFileSplitter.java | 1 - 1 file changed, 1 deletion(-) 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 38585a31dc5fa..064e12813434e 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 @@ -365,7 +365,6 @@ private void processValueChunk(TsFileSequenceReader reader, byte marker, String long[] times = pageIndex2Times.get(pageIndex); TsPrimitiveType[] values = decodeValuePage(reader, header, pageHeader, times, valueDecoder); for (AlignedChunkData alignedChunkData : alignedChunkDataList) { - alignedChunkData.setNotDecode(); alignedChunkData.writeDecodeValuePage(times, values, header.getDataType(), database); } } From 54103cadd58be58653e8689cce9f4f79f5d1dc89 Mon Sep 17 00:00:00 2001 From: libo Date: Fri, 22 May 2026 18:26:54 +0800 Subject: [PATCH 15/15] Restore TsFileManager compatibility overloads --- .../dataregion/tsfile/TsFileManager.java | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) 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 af761c065b5ba..cbbdbc647edbf 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,10 +65,23 @@ public TsFileManager(String storageGroupName, String dataRegionId, String dataRe this.dataRegionId = dataRegionId; } + public List getTsFileList(boolean sequence) { + return getTsFileList(sequence, storageGroupName); + } + public List getTsFileList(boolean sequence, String database) { return getTsFileList(sequence, null, null, database); } + /** + * @param sequence {@code true} for sequence, {@code false} for unsequence + * @param timePartitions {@code null} for all time partitions, empty for zero time partitions + */ + public List getTsFileList( + boolean sequence, List timePartitions, Filter timeFilter) { + return getTsFileList(sequence, timePartitions, timeFilter, storageGroupName); + } + /** * @param sequence {@code true} for sequence, {@code false} for unsequence * @param timePartitions {@code null} for all time partitions, empty for zero time partitions @@ -101,6 +114,16 @@ public List getTsFileList( } } + /** + * don't need to acquire lock again, caller should guarantee the lock has been acquired + * + * @return left is seq resource list, right is unSeq resource list + */ + public Pair, List> getAllTsFileListForQuery( + List timePartitions, Filter timeFilter) { + return getAllTsFileListForQuery(timePartitions, timeFilter, storageGroupName); + } + /** * don't need to acquire lock again, caller should guarantee the lock has been acquired * @@ -158,6 +181,10 @@ public Pair, List> getTsFileListSnapshot( } } + public List getTsFileList(boolean sequence, long startTime, long endTime) { + return getTsFileList(sequence, startTime, endTime, storageGroupName); + } + public List getTsFileList( boolean sequence, long startTime, long endTime, String database) { // the iteration of ConcurrentSkipListMap is not concurrent secure @@ -195,6 +222,10 @@ public TsFileResourceList getOrCreateUnsequenceListByTimePartition(long timePart } } + public Iterator getIterator(boolean sequence) { + return getIterator(sequence, storageGroupName); + } + public Iterator getIterator(boolean sequence, String database) { readLock(); try {