diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 index 338a478e78915..945ce2c975298 100644 --- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 +++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/IoTDBSqlParser.g4 @@ -38,7 +38,7 @@ statement ddlStatement : createStorageGroup | createTimeseries | createSchemaTemplate | createTimeseriesOfSchemaTemplate | createFunction | createTrigger | createContinuousQuery - | alterTimeseries | deleteStorageGroup | deleteTimeseries | deletePartition | deleteTimeseriesOfSchemaTemplate + | alterTimeseries |rewriteTimeseries | deleteStorageGroup | deleteTimeseries | deletePartition | deleteTimeseriesOfSchemaTemplate | dropFunction | dropTrigger | dropContinuousQuery | dropSchemaTemplate | setTTL | unsetTTL | startTrigger | stopTrigger | setSchemaTemplate | unsetSchemaTemplate | showStorageGroup | showDevices | showTimeseries | showChildPaths | showChildNodes @@ -49,6 +49,9 @@ ddlStatement | getRegionId | getTimeSlotList | getSeriesSlotList ; +rewriteTimeseries + : REWRITE TIMESERIES prefixPath; + dmlStatement : selectStatement | insertStatement | deleteStatement; @@ -185,6 +188,7 @@ alterClause | ADD TAGS attributePair (COMMA attributePair)* | ADD ATTRIBUTES attributePair (COMMA attributePair)* | UPSERT aliasClause? tagClause? attributeClause? + | SETTYPE attributePair (COMMA attributePair)* ; aliasClause diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 index c0438cf6caf13..c7989a6c843f3 100644 --- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 +++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlLexer.g4 @@ -117,6 +117,10 @@ CLEAR : C L E A R ; +REWRITE + : R E W R I T E + ; + CLUSTER : C L U S T E R ; @@ -666,6 +670,10 @@ UPSERT : U P S E R T ; +SETTYPE + : S E T T Y P E + ; + URI : U R I ; diff --git a/docs/UserGuide/Operate-Metadata/Timeseries.md b/docs/UserGuide/Operate-Metadata/Timeseries.md index af91e768c78f8..23189294f89b2 100644 --- a/docs/UserGuide/Operate-Metadata/Timeseries.md +++ b/docs/UserGuide/Operate-Metadata/Timeseries.md @@ -66,6 +66,18 @@ You can set different datatype, encoding, and compression for the timeseries in It is also supported to set an alias, tag, and attribute for aligned timeseries. +## Alter Timeseries + +To alter the encoding and compression of the timeseries we created before, we can use`ALTER TimeSeries SETTYPE encoding=,compression=` statement. + +The usage are as follows: + +``` +IoTDB> ALTER TimeSeries root.ln.wf01.wt01.status SETTYPE encoding=RLE +IoTDB> ALTER TimeSeries root.ln.wf01.wt01.status SETTYPE compression=SNAPPY +IoTDB> ALTER TimeSeries root.ln.wf01.wt01.status SETTYPE encoding=RLE,compression=SNAPPY +``` + ## Delete Timeseries To delete the timeseries we created before, we are able to use `(DELETE | DROP) TimeSeries ` statement. diff --git a/docs/UserGuide/Reference/SQL-Reference.md b/docs/UserGuide/Reference/SQL-Reference.md index 11276cb433bf0..863590f993dfa 100644 --- a/docs/UserGuide/Reference/SQL-Reference.md +++ b/docs/UserGuide/Reference/SQL-Reference.md @@ -188,12 +188,13 @@ Eg: IoTDB > DROP TIMESERIES root.ln.wf01.wt01.* ``` ALTER TIMESERIES fullPath alterClause alterClause - : RENAME beforeName=ID TO currentName=ID - | SET property (COMMA property)* - | DROP ID (COMMA ID)* - | ADD TAGS property (COMMA property)* - | ADD ATTRIBUTES property (COMMA property)* - | UPSERT tagClause attributeClause + : RENAME beforeName=attributeKey TO currentName=attributeKey + | SET attributePair (COMMA attributePair)* + | DROP attributeKey (COMMA attributeKey)* + | ADD TAGS attributePair (COMMA attributePair)* + | ADD ATTRIBUTES attributePair (COMMA attributePair)* + | UPSERT aliasClause? tagClause? attributeClause? + | SETTYPE attributePair (COMMA attributePair)* ; attributeClause : (ATTRIBUTES LR_BRACKET property (COMMA property)* RR_BRACKET)? @@ -207,6 +208,18 @@ Eg: ALTER timeseries root.turbine.d1.s1 DROP tag1, tag2 Eg: ALTER timeseries root.turbine.d1.s1 ADD TAGS tag3=v3, tag4=v4 Eg: ALTER timeseries root.turbine.d1.s1 ADD ATTRIBUTES attr3=v3, attr4=v4 EG: ALTER timeseries root.turbine.d1.s1 UPSERT TAGS(tag2=newV2, tag3=v3) ATTRIBUTES(attr3=v3, attr4=v4) +Eg: ALTER timeseries root.turbine.d1.s1 SETTYPE encoding=plain, compression=SNAPPY +``` + +* Rewrite Timeseries Statement +``` +REWRITE TIMESERIES prefixPath +prefixPath + : ROOT (DOT nodeName)* + ; +Eg: Rewrite Timeseries root.ln +Note: This statement can be used when "ALTER timeseries" has been called +Note: prefixPath is a storageGroup path ``` * Show All Timeseries Statement diff --git a/docs/zh/UserGuide/Reference/SQL-Reference.md b/docs/zh/UserGuide/Reference/SQL-Reference.md index aaff8ee919211..7689309c97f96 100644 --- a/docs/zh/UserGuide/Reference/SQL-Reference.md +++ b/docs/zh/UserGuide/Reference/SQL-Reference.md @@ -173,17 +173,18 @@ Eg: IoTDB > DELETE TIMESERIES root.ln.wf01.wt01.* Eg: IoTDB > DROP TIMESERIES root.ln.wf01.wt01.* ``` -* 修改时间序列标签属性语句 +* 修改时间序列标签属性、编码类型和压缩类型语句 ``` ALTER TIMESERIES fullPath alterClause alterClause - : RENAME beforeName=ID TO currentName=ID - | SET property (COMMA property)* - | DROP ID (COMMA ID)* - | ADD TAGS property (COMMA property)* - | ADD ATTRIBUTES property (COMMA property)* - | UPSERT tagClause attributeClause + : RENAME beforeName=attributeKey TO currentName=attributeKey + | SET attributePair (COMMA attributePair)* + | DROP attributeKey (COMMA attributeKey)* + | ADD TAGS attributePair (COMMA attributePair)* + | ADD ATTRIBUTES attributePair (COMMA attributePair)* + | UPSERT aliasClause? tagClause? attributeClause? + | SETTYPE attributePair (COMMA attributePair)* ; attributeClause : (ATTRIBUTES LR_BRACKET property (COMMA property)* RR_BRACKET)? @@ -197,6 +198,18 @@ Eg: ALTER timeseries root.turbine.d1.s1 DROP tag1, tag2 Eg: ALTER timeseries root.turbine.d1.s1 ADD TAGS tag3=v3, tag4=v4 Eg: ALTER timeseries root.turbine.d1.s1 ADD ATTRIBUTES attr3=v3, attr4=v4 EG: ALTER timeseries root.turbine.d1.s1 UPSERT TAGS(tag2=newV2, tag3=v3) ATTRIBUTES(attr3=v3, attr4=v4) +Eg: ALTER timeseries root.turbine.d1.s1 SETTYPE encoding=plain, compression=SNAPPY +``` + +* 重写时间序列语句 +``` +REWRITE TIMESERIES prefixPath +prefixPath + : ROOT (DOT nodeName)* + ; +Eg: Rewrite Timeseries root.ln +Note: This statement can be used when "ALTER timeseries" has been called +Note: prefixPath is a storageGroup path ``` * 显示所有时间序列语句 diff --git a/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBAlterTimeSeriesIT.java b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBAlterTimeSeriesIT.java new file mode 100644 index 0000000000000..1068699013ba6 --- /dev/null +++ b/integration/src/test/java/org/apache/iotdb/db/integration/IoTDBAlterTimeSeriesIT.java @@ -0,0 +1,123 @@ +/* + * 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.db.integration; + +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.integration.env.ConfigFactory; +import org.apache.iotdb.integration.env.EnvFactory; +import org.apache.iotdb.itbase.category.ClusterTest; +import org.apache.iotdb.itbase.category.LocalStandaloneTest; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; + +import static org.junit.Assert.assertEquals; + +@Category({LocalStandaloneTest.class, ClusterTest.class}) +public class IoTDBAlterTimeSeriesIT { + + private static final Logger logger = LoggerFactory.getLogger(IoTDBAlterTimeSeriesIT.class); + private long prevPartitionInterval; + + @Before + public void setUp() throws Exception { + prevPartitionInterval = IoTDBDescriptor.getInstance().getConfig().getPartitionInterval(); + ConfigFactory.getConfig().setPartitionInterval(1); + EnvFactory.getEnv().initBeforeTest(); + } + + @After + public void tearDown() throws Exception { + EnvFactory.getEnv().cleanAfterTest(); + ConfigFactory.getConfig().setPartitionInterval(prevPartitionInterval); + } + + @Test + public void testAlter() throws SQLException { + logger.info("test..."); + try (Connection connection = EnvFactory.getEnv().getConnection(); + Statement statement = connection.createStatement()) { + statement.execute("SET STORAGE GROUP TO root.alterTimeSeriesTest"); + try { + statement.execute( + "CREATE TIMESERIES root.alterTimeSeriesTest.s1 WITH DATATYPE=INT64,ENCODING=PLAIN"); + statement.execute( + "CREATE TIMESERIES root.alterTimeSeriesTest.s2 WITH DATATYPE=INT64,ENCODING=PLAIN"); + } catch (SQLException e) { + // ignore + } + + for (int i = 1; i <= 1000; i++) { + statement.execute( + String.format( + "INSERT INTO root.alterTimeSeriesTest(timestamp,s1,s2) VALUES (%d,%d,%d)", + i, i, i)); + } + statement.execute("FLUSH"); + ResultSet resultSetP = statement.executeQuery("show timeseries root.alterTimeSeriesTest.s1"); + while (resultSetP.next()) { + assertEquals(resultSetP.getString("encoding"), "PLAIN"); + assertEquals(resultSetP.getString("compression"), "SNAPPY"); + } + statement.execute( + "alter timeseries root.alterTimeSeriesTest.s1 settype encoding=gorilla,compression=gzip"); + + ResultSet resultSetAL = statement.executeQuery("show timeseries root.alterTimeSeriesTest.s1"); + while (resultSetAL.next()) { + assertEquals(resultSetAL.getString("encoding"), "GORILLA"); + assertEquals(resultSetAL.getString("compression"), "GZIP"); + } + for (int i = 1001; i <= 1010; i++) { + statement.execute( + String.format( + "INSERT INTO root.alterTimeSeriesTest(timestamp,s1,s2) VALUES (%d,%d,%d)", + i, i, i)); + } + + try (ResultSet resultSet = statement.executeQuery("SELECT * FROM root.alterTimeSeriesTest")) { + int cur = 1; + int count = 0; + while (resultSet.next()) { + long time = resultSet.getLong("Time"); + long s1 = resultSet.getLong("root.alterTimeSeriesTest.s1"); + long s2 = resultSet.getLong("root.alterTimeSeriesTest.s2"); + + assertEquals(cur, time); + assertEquals(cur, s1); + assertEquals(cur, s2); + cur++; + count++; + } + assertEquals(1010, count); + } + + statement.execute("REWRITE TIMESERIES root.alterTimeSeriesTest"); + } + } +} diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java b/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java index 2e58d01d60f17..f1d19aaf93b2e 100644 --- a/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java +++ b/node-commons/src/main/java/org/apache/iotdb/commons/conf/IoTDBConstant.java @@ -255,6 +255,12 @@ private IoTDBConstant() {} // compaction mods of previous version (<0.13) public static final String COMPACTION_MODIFICATION_FILE_NAME_FROM_OLD = "merge.mods"; + // alter timeseries temp file + public static final String ALTER_TMP_FILE_SUFFIX = ".alter"; + public static final String ALTER_OLD_TMP_FILE_SUFFIX = ".alter.old"; + public static final String ALTER_TMP_FILE_RESOURCE_SUFFIX = ".alter.resource"; + public static final String ALTER_OLD_TMP_FILE_RESOURCE_SUFFIX = ".alter.old.resource"; + // write ahead log public static final String WAL_FILE_PREFIX = "_"; public static final String WAL_FILE_SUFFIX = ".wal"; diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java index 6ccb3506358fd..3cf021c01bd0e 100644 --- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java +++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/RSchemaRegion.java @@ -1827,6 +1827,13 @@ public long countPathsUsingTemplate(int templateId, PathPatternTree patternTree) throw new UnsupportedOperationException(); } + @Override + public Pair alterTimeseriesEncodingCompressionType( + PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType) + throws MetadataException, IOException { + throw new UnsupportedOperationException(); + } + @Override public String toString() { return String.format("database:[%s]", storageGroupFullPath); diff --git a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RMeasurementMNode.java b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RMeasurementMNode.java index c13879c5529bd..bea424616b2cd 100644 --- a/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RMeasurementMNode.java +++ b/schema-engine-rocksdb/src/main/java/org/apache/iotdb/db/metadata/schemaregion/rocksdb/mnode/RMeasurementMNode.java @@ -29,7 +29,9 @@ import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaConstants; import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaReadWriteHandler; import org.apache.iotdb.db.metadata.schemaregion.rocksdb.RSchemaUtils; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema; import org.apache.iotdb.tsfile.write.schema.MeasurementSchema; @@ -127,6 +129,16 @@ public void setAlias(String alias) { this.alias = alias; } + @Override + public void updateSchemaInfo( + String measurementId, + TSEncoding encoding, + CompressionType compressionType, + Map props) { + // TODO not support now + throw new UnsupportedOperationException("only for alter timeSeries"); + } + @Override public boolean isPreDeleted() { return false; diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index bf77df19724cb..d3ecc9c6eb4e0 100644 --- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -1046,6 +1046,11 @@ public class IoTDBConfig { // customizedProperties, this should be empty by default. private Properties customizedProperties = new Properties(); + /** TsFileManager Rewrite lock timeout */ + private long rewriteLockWaitTimeoutInMS = (int) TimeUnit.MINUTES.toMillis(1); + + private long rewriteCandidateStatusWaitTimeoutInMs = TimeUnit.MINUTES.toMillis(10); + IoTDBConfig() {} public float getUdfMemoryBudgetInMB() { @@ -3329,6 +3334,22 @@ public String getConfigMessage() { return configMessage; } + public long getRewriteLockWaitTimeoutInMS() { + return rewriteLockWaitTimeoutInMS; + } + + public void setRewriteLockWaitTimeoutInMS(long rewriteLockWaitTimeoutInMS) { + this.rewriteLockWaitTimeoutInMS = rewriteLockWaitTimeoutInMS; + } + + public long getRewriteCandidateStatusWaitTimeoutInMs() { + return rewriteCandidateStatusWaitTimeoutInMs; + } + + public void setRewriteCandidateStatusWaitTimeoutInMs(long rewriteCandidateStatusWaitTimeoutInMs) { + this.rewriteCandidateStatusWaitTimeoutInMs = rewriteCandidateStatusWaitTimeoutInMs; + } + public long getDataRatisConsensusSnapshotTriggerThreshold() { return dataRatisConsensusSnapshotTriggerThreshold; } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java index 3818568a13b3d..97fc89befbe6b 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java @@ -26,8 +26,10 @@ import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil; import org.apache.iotdb.commons.conf.CommonDescriptor; import org.apache.iotdb.commons.consensus.DataRegionId; +import org.apache.iotdb.commons.exception.MetadataException; import org.apache.iotdb.commons.exception.ShutdownException; import org.apache.iotdb.commons.file.SystemFileFactory; +import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.commons.service.IService; import org.apache.iotdb.commons.service.ServiceType; import org.apache.iotdb.commons.utils.TestOnly; @@ -35,6 +37,7 @@ import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.consensus.statemachine.visitor.DataExecutionVisitor; +import org.apache.iotdb.db.engine.cache.AlteringRecordsCache; import org.apache.iotdb.db.engine.flush.CloseFileListener; import org.apache.iotdb.db.engine.flush.FlushListener; import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy; @@ -47,7 +50,9 @@ import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.exception.TsFileProcessorException; import org.apache.iotdb.db.exception.WriteProcessRejectException; +import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException; import org.apache.iotdb.db.exception.runtime.StorageEngineFailureException; +import org.apache.iotdb.db.metadata.LocalSchemaProcessor; import org.apache.iotdb.db.mpp.plan.planner.plan.node.PlanNode; import org.apache.iotdb.db.mpp.plan.planner.plan.node.load.LoadTsFilePieceNode; import org.apache.iotdb.db.mpp.plan.scheduler.load.LoadTsFileScheduler; @@ -61,7 +66,10 @@ import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.tsfile.exception.write.PageException; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.utils.FilePathUtils; +import org.apache.iotdb.tsfile.utils.Pair; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import org.apache.commons.io.FileUtils; @@ -546,6 +554,49 @@ private void checkResults(List> tasks, String errorMsg) { } } + /** + * alter timeseries encoding & compressionType + * + * @param fullPath + * @param curEncoding + * @param curCompressionType + * @throws StorageEngineException + */ + public void alterTimeseries( + PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType) + throws StorageEngineException, StorageGroupNotSetException { + // The alterLock is mutually exclusive with the clear operation + StorageGroupManager storageGroupManager = + processorMap.get(IoTDB.schemaProcessor.getBelongedStorageGroup(fullPath)); + if (storageGroupManager == null) { + throw new StorageEngineException("system error, StorageGroup not found"); + } + storageGroupManager.alterLock(); + // Update the ALTER state in memory + AlteringRecordsCache.getInstance().startAlter(); + try { + // Change the encoding compression type in the schema first. After that, the newly inserted + // data + // will use the new encoding compression type + Pair oldPair = + IoTDB.schemaProcessor.alterTimeseries(fullPath, curEncoding, curCompressionType); + if (oldPair == null || oldPair.left == null || oldPair.right == null) { + throw new MetadataException("system error, old type is null"); + } + curEncoding = curEncoding == null ? oldPair.left : curEncoding; + curCompressionType = curCompressionType == null ? oldPair.right : curCompressionType; + + // process alter + storageGroupManager.alterTimeseries(fullPath, curEncoding, curCompressionType); + } catch (IOException | MetadataException e) { + throw new StorageEngineException(e.getMessage()); + } finally { + if (storageGroupManager != null) { + storageGroupManager.alterUnlock(); + } + } + } + /** * merge all databases. * @@ -670,6 +721,17 @@ public void deleteDataRegion(DataRegionId regionId) { } } + /** + * Get the virtual storage group name. + * + * @return virtual storage group name, like root.sg1/0 + */ + public String getStorageGroupName(PartialPath path) throws StorageEngineException { + PartialPath deviceId = path.getDevicePath(); + DataRegion storageGroupProcessor = getProcessor(deviceId); + return storageGroupProcessor.getStorageGroupName(); + } + public DataRegion getDataRegion(DataRegionId regionId) { return dataRegionMap.get(regionId); } @@ -809,6 +871,33 @@ private void stopTimedServiceAndThrow(ScheduledExecutorService pool, String pool } } + public void rewriteTimeseries(PartialPath fullPath) + throws StorageEngineException, MetadataException { + + if (fullPath == null) { + throw new StorageEngineException("Rewrite Timeseries Failed. The storageGroupName is null"); + } + // Check altering cache + AlteringRecordsCache alteringRecordsCache = AlteringRecordsCache.getInstance(); + boolean storageGroupExsist = alteringRecordsCache.isStorageGroupExsist(fullPath.getFullPath()); + if (!storageGroupExsist) { + throw new StorageEngineException( + "Rewrite Timeseries Failed. The storage group you entered is not altering"); + } + StorageGroupManager storageGroupManager = + getStorageGroupManager( + LocalSchemaProcessor.getInstance().getStorageGroupNodeByPath(fullPath)); + if (storageGroupManager == null) { + throw new StorageEngineException( + "Rewrite Timeseries Failed. The StorageGroupManager is null"); + } + try { + storageGroupManager.rewriteTimeseries(); + } catch (Exception e) { + throw new StorageEngineException(e); + } + } + static class InstanceHolder { private static final StorageEngine INSTANCE = new StorageEngine(); diff --git a/server/src/main/java/org/apache/iotdb/db/engine/alter/RewriteTimeseriesTask.java b/server/src/main/java/org/apache/iotdb/db/engine/alter/RewriteTimeseriesTask.java new file mode 100644 index 0000000000000..02c3a7c0fbb2d --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/engine/alter/RewriteTimeseriesTask.java @@ -0,0 +1,361 @@ +/* + * 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.db.engine.alter; + +import org.apache.iotdb.commons.conf.IoTDBConstant; +import org.apache.iotdb.commons.exception.MetadataException; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.engine.alter.log.AlteringLogger; +import org.apache.iotdb.db.engine.cache.AlteringRecordsCache; +import org.apache.iotdb.db.engine.compaction.CompactionUtils; +import org.apache.iotdb.db.engine.compaction.log.TsFileIdentifier; +import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask; +import org.apache.iotdb.db.engine.storagegroup.TsFileManager; +import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator; +import org.apache.iotdb.db.engine.storagegroup.TsFileResource; +import org.apache.iotdb.db.engine.storagegroup.TsFileResourceList; +import org.apache.iotdb.db.engine.storagegroup.TsFileResourceStatus; +import org.apache.iotdb.db.exception.StorageEngineException; +import org.apache.iotdb.tsfile.common.conf.TSFileConfig; +import org.apache.iotdb.tsfile.exception.write.TsFileNotCompleteException; + +import org.apache.commons.io.FileUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.iotdb.commons.conf.IoTDBConstant.ALTER_OLD_TMP_FILE_SUFFIX; +import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX; + +public class RewriteTimeseriesTask extends AbstractCompactionTask { + + private static final Logger LOGGER = + LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME); + + private static final long waitTimeout = + IoTDBDescriptor.getInstance().getConfig().getRewriteCandidateStatusWaitTimeoutInMs(); + + private final List candidateResourceList = new ArrayList<>(32); + + private final List readyResourceList = new ArrayList<>(32); + + private final AlteringRecordsCache alteringRecordsCache = AlteringRecordsCache.getInstance(); + + private final File logFile; + + private final String logKey; + + public RewriteTimeseriesTask( + String storageGroupName, + String dataRegionId, + List timePartitions, + TsFileManager tsFileManager, + AtomicInteger currentTaskNum, + long serialId, + boolean isClearBegin, + Set doneFiles, + File logFile) { + + super(storageGroupName, dataRegionId, 0L, tsFileManager, currentTaskNum, serialId); + this.logKey = storageGroupName + dataRegionId; + this.logFile = logFile; + this.performer = new TsFileRewritePerformer(); + if (timePartitions == null || timePartitions.size() <= 0) { + LOGGER.warn("[rewriteTimeseries] {} timePartitions is null or empty!!!!!!", logKey); + return; + } + timePartitions.forEach( + curTimePartition -> + candidateResourceSelectorByTimePartition( + storageGroupName, tsFileManager, isClearBegin, doneFiles, curTimePartition)); + } + + private void candidateResourceSelectorByTimePartition( + String storageGroupName, + TsFileManager tsFileManager, + boolean isClearBegin, + Set doneFiles, + Long curTimePartition) { + // get seq tsFile list + TsFileResourceList seqTsFileResourcList = + tsFileManager.getSequenceListByTimePartition(curTimePartition); + // Gets the device list from the cache + Set devicesCache = alteringRecordsCache.getDevicesCache(storageGroupName); + if (seqTsFileResourcList != null && seqTsFileResourcList.size() > 0) { + seqTsFileResourcList.forEach( + tsFileResource -> { + Set devices = tsFileResource.getDevices(); + if (devices == null) { + return; + } + // AlteringLog filter + if (isClearBegin && doneFiles != null && doneFiles.size() > 0) { + for (TsFileIdentifier tsFileIdentifier : doneFiles) { + try { + TsFileNameGenerator.TsFileName logTsFileName = + TsFileNameGenerator.getTsFileName(tsFileIdentifier.getFilename()); + TsFileNameGenerator.TsFileName tsFileName = + TsFileNameGenerator.getTsFileName(tsFileResource.getTsFile().getName()); + // As long as time and version are the same, they are considered to be the same + // file + if (logTsFileName.getTime() == tsFileName.getTime() + && logTsFileName.getVersion() == tsFileName.getVersion()) { + LOGGER.info( + "[rewriteTimeseries] {} the file {} has been rewritten", + logKey, + tsFileResource.getTsFilePath()); + return; + } + } catch (IOException e) { + LOGGER.warn("tsfile-{} name parseFailed", tsFileIdentifier); + return; + } + } + } + // device filter + // In most scenarios, the number of devices in the tsfile file is greater than the + // number of devices to be modified + for (String device : devicesCache) { + // Looking for intersection + // TODO Use a better algorithm instead + if (devices.contains(device)) { + candidateResourceList.add(tsFileResource); + break; + } + } + }); + } + } + + /** Continuously trying to update the status to CANDIDATE for a certain amount of time */ + @Override + public void setSourceFilesToCompactionCandidate() { + + long begin = System.currentTimeMillis(); + while (candidateResourceList.size() > 0) { + List removeList = new ArrayList<>(candidateResourceList.size()); + candidateResourceList.forEach( + x -> { + if (x.isDeleted() || !x.isClosed()) { + removeList.add(x); + return; + } + if (x.getStatus() == TsFileResourceStatus.CLOSED) { + try { + x.setStatus(TsFileResourceStatus.COMPACTION_CANDIDATE); + readyResourceList.add(x); + removeList.add(x); + } catch (Exception e) { + // do nothing + } + } + }); + candidateResourceList.removeAll(removeList); + if ((System.currentTimeMillis() - begin) > waitTimeout) { + LOGGER.error("[rewriteTimeseries] {} setSourceFilesToCompactionCandidate timeout", logKey); + break; + } + } + } + + @Override + protected void doCompaction() { + + // rewrite target tsfiles + try (AlteringLogger alteringLogger = new AlteringLogger(logFile)) { + if (readyResourceList.isEmpty()) { + return; + } + int size = readyResourceList.size(); + LOGGER.info("[rewriteTimeseries] {} rewrite begin, ready resource size:{}", logKey, size); + for (int i = 0; i < size; i++) { + TsFileResource tsFileResource = readyResourceList.get(i); + if (tsFileResource == null || !tsFileResource.isClosed()) { + return; + } + rewriteDataInTsFile(tsFileResource); + // log file done + alteringLogger.doneFile(tsFileResource); + LOGGER.info( + "[rewriteTimeseries] {} rewriteDataInTsFile {} end, fileNum:{}/{}, fileSize:{}", + logKey, + tsFileResource.getTsFilePath(), + i + 1, + size, + tsFileResource.getTsFileSize()); + } + } catch (Exception e) { + LOGGER.error("[rewriteTimeseries] " + logKey + " error", e); + } finally { + // The process is complete and the logFile is deleted + if (logFile.exists()) { + try { + FileUtils.delete(logFile); + } catch (IOException e) { + LOGGER.error("[rewriteTimeseries] " + logKey + " logFile delete failed", e); + } + } + } + } + + private void rewriteDataInTsFile(TsFileResource tsFileResource) + throws IOException, StorageEngineException, InterruptedException, MetadataException, + ExecutionException { + + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "[rewriteTimeseries] {} rewriteDataInTsFile:{}, fileSize:{} start", + logKey, + tsFileResource.getTsFilePath(), + tsFileResource.getTsFileSize()); + } + // Generate the target tsFileResource + TsFileResource targetTsFileResource = + TsFileNameGenerator.generateNewAlterTsFileResource(tsFileResource); + // Data is read from the.tsfile file, re-encoded, compressed, and written to the .alter file + this.performer.setSourceFiles(Collections.singletonList(tsFileResource)); + this.performer.setTargetFiles(Collections.singletonList(targetTsFileResource)); + this.performer.setSummary(this.summary); + this.performer.perform(); + boolean hasRewrite = ((TsFileRewritePerformer) this.performer).hasRewrite(); + if (!hasRewrite) { + CompactionUtils.deleteTsFileWithoutMods(targetTsFileResource); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug( + "[rewriteTimeseries] {} tsFile:{} does not need to be rewrite", + logKey, + tsFileResource.getTsFileSize()); + } + + return; + } + // .tsfile->.alter.old .alter->.tsfile + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("[rewriteTimeseries] {} move tsfile", logKey); + } + tsFileResource.moveTsFile(TSFILE_SUFFIX, ALTER_OLD_TMP_FILE_SUFFIX); + targetTsFileResource.moveTsFile(IoTDBConstant.ALTER_TMP_FILE_SUFFIX, TSFILE_SUFFIX); + // replace + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("[rewriteTimeseries] {} replace tsfile", logKey); + } + tsFileManager.replace( + Collections.singletonList(tsFileResource), + Collections.emptyList(), + Collections.singletonList(targetTsFileResource), + timePartition, + true); + // check & delete tsfile from disk + checkAndDeleteOldTsFile(tsFileResource, targetTsFileResource, logKey); + } + + private void checkAndDeleteOldTsFile( + TsFileResource tsFileResource, TsFileResource targetTsFileResource, String logKey) + throws IOException { + // check + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("[rewriteTimeseries] {} check tsfile", logKey); + } + if (targetTsFileResource.getTsFile().exists() + && targetTsFileResource.getTsFile().length() + < TSFileConfig.MAGIC_STRING.getBytes().length * 2L + Byte.BYTES) { + // the file size is smaller than magic string and version number + throw new TsFileNotCompleteException( + String.format( + "target file %s is smaller than magic string and version number size", + targetTsFileResource)); + } + + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("[rewriteTimeseries] {} delete tsfile", logKey); + } + LOGGER.info( + "[rewriteTimeseries] {} alter {} finish, start to delete old files", + logKey, + tsFileResource.getTsFilePath()); + // delete the old files + CompactionUtils.deleteTsFileWithoutMods(tsFileResource); + } + + @Override + public boolean equalsOtherTask(AbstractCompactionTask otherTask) { + if (!(otherTask instanceof RewriteTimeseriesTask)) { + return false; + } + RewriteTimeseriesTask task = (RewriteTimeseriesTask) otherTask; + return task.getStorageGroupName().equals(this.getStorageGroupName()) + && task.getDataRegionId().equals(this.getDataRegionId()); + } + + /** copy from InnerSpaceCompactionTask */ + @Override + public boolean checkValidAndSetMerging() { + if (!tsFileManager.isAllowCompaction()) { + return false; + } + try { + for (TsFileResource resource : readyResourceList) { + if (resource.isCompacting() + || !resource.isClosed() + || !resource.getTsFile().exists() + || resource.isDeleted()) { + // this source file cannot be compacted + // release the lock of locked files, and return + resetMergingStatus(); + return false; + } + } + + for (TsFileResource resource : readyResourceList) { + resource.setStatus(TsFileResourceStatus.COMPACTING); + } + } catch (Throwable e) { + resetMergingStatus(); + throw e; + } + return true; + } + + /** set the merging status of selected files to false copy from InnerSpaceCompactionTask */ + protected void resetMergingStatus() { + for (TsFileResource resource : readyResourceList) { + try { + if (!resource.isDeleted()) { + resource.setStatus(TsFileResourceStatus.CLOSED); + } + } catch (Throwable e) { + LOGGER.error("Exception occurs when resetting resource status", e); + } + } + } + + @Override + public void resetCompactionCandidateStatusForAllSourceFiles() { + readyResourceList.forEach(x -> x.setStatus(TsFileResourceStatus.CLOSED)); + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/engine/alter/RewriteTimeseriesTaskManager.java b/server/src/main/java/org/apache/iotdb/db/engine/alter/RewriteTimeseriesTaskManager.java new file mode 100644 index 0000000000000..ea710b27c1f49 --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/engine/alter/RewriteTimeseriesTaskManager.java @@ -0,0 +1,176 @@ +/* + * 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.db.engine.alter; + +import org.apache.iotdb.commons.conf.IoTDBConstant; +import org.apache.iotdb.commons.file.SystemFileFactory; +import org.apache.iotdb.db.engine.alter.log.AlteringLogAnalyzer; +import org.apache.iotdb.db.engine.alter.log.AlteringLogger; +import org.apache.iotdb.db.engine.cache.AlteringRecordsCache; +import org.apache.iotdb.db.engine.compaction.CompactionTaskManager; +import org.apache.iotdb.db.engine.compaction.task.CompactionTaskSummary; +import org.apache.iotdb.db.engine.storagegroup.DataRegion; +import org.apache.iotdb.db.engine.storagegroup.TsFileManager; +import org.apache.iotdb.db.engine.storagegroup.dataregion.StorageGroupManager; +import org.apache.iotdb.db.exception.StorageEngineException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Future; + +/** Ensure that alterLock locks RewriteTimeseriesTask for all Dataregions */ +public class RewriteTimeseriesTaskManager { + + private static final Logger LOGGER = + LoggerFactory.getLogger(IoTDBConstant.COMPACTION_LOGGER_NAME); + + private final CompactionTaskManager compactionTaskManager = CompactionTaskManager.getInstance(); + + private AlteringRecordsCache alteringRecordsCache = AlteringRecordsCache.getInstance(); + + private RewriteTimeseriesTaskManager() {} + + public void rewriteTimeseries(DataRegion[] dataRegions, StorageGroupManager storageGroupManager) + throws Exception { + + if (storageGroupManager == null || dataRegions == null || dataRegions.length <= 0) { + return; + } + if (storageGroupManager.isAlterLocked()) { + throw new StorageEngineException("Rewrite is unfinished, try again later"); + } + // Avoid creating too many threads as a result of pressure testing + storageGroupManager.alterLock(); + try { + Thread thread = + new Thread( + () -> { + DataRegion dataRegion = dataRegions[0]; + if (dataRegion == null) { + return; + } + String storageGroupName = dataRegion.getStorageGroupName(); + storageGroupManager.alterLock(); + try { + // rewrite DataRegions + doRewrite(dataRegions); + } catch (Exception e) { + LOGGER.error("RewriteTimeseriesTaskManager Error", e); + } finally { + alteringRecordsCache.clear(storageGroupName); + storageGroupManager.alterUnlock(); + } + }, + this.getClass().getSimpleName()); + thread.start(); + } finally { + storageGroupManager.alterUnlock(); + } + } + + private void doRewrite(DataRegion[] dataRegions) throws IOException { + List tasks = new ArrayList<>(); + for (DataRegion dataRegion : dataRegions) { + if (dataRegion != null) { + List timePartitions = dataRegion.getTimePartitions(); + if (timePartitions != null && timePartitions.size() > 0) { + // alter.log analyzer + File logFile = + SystemFileFactory.INSTANCE.getFile( + dataRegion.getStorageGroupSysDir(), AlteringLogger.ALTERING_LOG_NAME); + if (!logFile.exists()) { + // there is no altering timeseries in the DataRegion + continue; + } + AlteringLogAnalyzer analyzer = new AlteringLogAnalyzer(logFile); + analyzer.analyzer(); + // clear begin + if (!analyzer.isClearBegin()) { + AlteringLogger.clearBegin(logFile); + } + TsFileManager tsFileManager = dataRegion.getTsFileManager(); + // Create tasks grouped by DataRegion TimePartition + // Share CompactionTaskManager + try { + RewriteTimeseriesTask task = + new RewriteTimeseriesTask( + dataRegion.getStorageGroupName(), + dataRegion.getDataRegionId(), + timePartitions, + tsFileManager, + CompactionTaskManager.currentTaskNum, + tsFileManager.getNextCompactionTaskId(), + analyzer.isClearBegin(), + analyzer.getDoneFiles(), + logFile); + boolean b = compactionTaskManager.addTaskToWaitingQueue(task); + if (!b) { + // addTaskToWaitingQueue failed + continue; + } + tasks.add(task); + } catch (Exception e) { + LOGGER.error("addTaskTOWaitingQueue failed", e); + } + } + } + } + + // wait until all proccess done + tasks.forEach( + task -> { + String logKey = task.getStorageGroupName() + "-" + task.getDataRegionId(); + try { + Future future = + compactionTaskManager.getCompactionTaskFutureCheckStatusMayBlock(task); + boolean isFinished = false; + if (future != null) { + CompactionTaskSummary summary = future.get(); + if (summary == null) { + LOGGER.info("[rewriteTimeseries] {} task summary is null", logKey); + return; + } + isFinished = summary.isFinished(); + } + LOGGER.info( + "[rewriteTimeseries] {} task isFinished:{}", + logKey, + future == null ? task.isTaskFinished() : isFinished); + } catch (Exception e) { + LOGGER.error("rewrite future failed " + logKey, e); + } + }); + } + + public static RewriteTimeseriesTaskManager getInstance() { + return RewriteTimeseriesTaskManagerHolder.INSTANCE; + } + + /** singleton pattern. */ + private static class RewriteTimeseriesTaskManagerHolder { + + private static final RewriteTimeseriesTaskManager INSTANCE = new RewriteTimeseriesTaskManager(); + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/engine/alter/TsFileRewritePerformer.java b/server/src/main/java/org/apache/iotdb/db/engine/alter/TsFileRewritePerformer.java new file mode 100644 index 0000000000000..2613d910603fc --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/engine/alter/TsFileRewritePerformer.java @@ -0,0 +1,485 @@ +/* + * 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.db.engine.alter; + +import org.apache.iotdb.commons.exception.MetadataException; +import org.apache.iotdb.db.engine.cache.AlteringRecordsCache; +import org.apache.iotdb.db.engine.compaction.performer.ICompactionPerformer; +import org.apache.iotdb.db.engine.compaction.task.CompactionTaskSummary; +import org.apache.iotdb.db.engine.storagegroup.TsFileResource; +import org.apache.iotdb.db.exception.StorageEngineException; +import org.apache.iotdb.tsfile.file.header.ChunkHeader; +import org.apache.iotdb.tsfile.file.metadata.AlignedChunkMetadata; +import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata; +import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; +import org.apache.iotdb.tsfile.read.TimeValuePair; +import org.apache.iotdb.tsfile.read.TsFileAlignedSeriesReaderIterator; +import org.apache.iotdb.tsfile.read.TsFileDeviceIterator; +import org.apache.iotdb.tsfile.read.TsFileSequenceReader; +import org.apache.iotdb.tsfile.read.common.Chunk; +import org.apache.iotdb.tsfile.read.common.IBatchDataIterator; +import org.apache.iotdb.tsfile.read.reader.IChunkReader; +import org.apache.iotdb.tsfile.read.reader.IPointReader; +import org.apache.iotdb.tsfile.read.reader.chunk.AlignedChunkReader; +import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader; +import org.apache.iotdb.tsfile.utils.Pair; +import org.apache.iotdb.tsfile.utils.TsPrimitiveType; +import org.apache.iotdb.tsfile.write.chunk.AlignedChunkWriterImpl; +import org.apache.iotdb.tsfile.write.chunk.ChunkWriterImpl; +import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema; +import org.apache.iotdb.tsfile.write.schema.MeasurementSchema; +import org.apache.iotdb.tsfile.write.writer.TsFileIOWriter; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** This class is used to rewrite one tsfile with current encoding & compressionType. */ +public class TsFileRewritePerformer implements ICompactionPerformer { + + private static final Logger logger = LoggerFactory.getLogger(TsFileRewritePerformer.class); + + private TsFileResource tsFileResource; + private TsFileResource targetTsFileResource; + // record the min time and max time to update the target resource + private long minStartTimestamp = Long.MAX_VALUE; + private long maxEndTimestamp = Long.MIN_VALUE; + private CompactionTaskSummary summary; + private boolean hasRewrite = true; + + private final AlteringRecordsCache alteringRecordsCache = AlteringRecordsCache.getInstance(); + + public TsFileRewritePerformer() {} + + @Override + public void perform() + throws IOException, MetadataException, StorageEngineException, InterruptedException { + this.hasRewrite = execute(); + } + + @Override + public void setTargetFiles(List targetFiles) { + if (targetFiles != null && targetFiles.size() > 0) { + this.targetTsFileResource = targetFiles.get(0); + } + } + + @Override + public void setSummary(CompactionTaskSummary summary) { + this.summary = summary; + } + + @Override + public void setSourceFiles(List files) { + if (files != null && files.size() > 0) { + this.tsFileResource = files.get(0); + } + } + + /** + * This function execute the rewrite task + * + * @return false if not rewrite + */ + private boolean execute() throws IOException, MetadataException, InterruptedException { + + tsFileResource.readLock(); + try (TsFileSequenceReader reader = new TsFileSequenceReader(tsFileResource.getTsFilePath()); + TsFileIOWriter writer = new TsFileIOWriter(targetTsFileResource.getTsFile())) { + // fast done, Many files may have been rewrite at merge time + boolean needRewrite = readFirstChunkMetaCheckNeedRewrite(reader); + if (!needRewrite) { + return false; + } + // read devices + TsFileDeviceIterator deviceIterator = reader.getAllDevicesIteratorWithIsAligned(); + while (deviceIterator.hasNext()) { + minStartTimestamp = Long.MAX_VALUE; + maxEndTimestamp = Long.MIN_VALUE; + Pair deviceInfo = deviceIterator.next(); + String device = deviceInfo.left; + boolean aligned = deviceInfo.right; + // write chunkGroup header + writer.startChunkGroup(device); + // write chunk & page data + if (aligned) { + rewriteAlgined(reader, writer, device); + } else { + rewriteNotAligned(device, reader, writer); + } + // chunkGroup end + writer.endChunkGroup(); + + targetTsFileResource.updateStartTime(device, minStartTimestamp); + targetTsFileResource.updateEndTime(device, maxEndTimestamp); + } + + targetTsFileResource.updatePlanIndexes(tsFileResource); + // write index,bloom,footer, end file + writer.endFile(); + targetTsFileResource.close(); + } catch (Exception e) { + throw new IOException(e); + } finally { + tsFileResource.readUnlock(); + } + return true; + } + + /** Read the first ChunkMeta and determine whether it needs to be rewrite */ + private boolean readFirstChunkMetaCheckNeedRewrite(TsFileSequenceReader reader) + throws IOException { + TsFileDeviceIterator deviceIterator = reader.getAllDevicesIteratorWithIsAligned(); + while (deviceIterator.hasNext()) { + Pair deviceInfo = deviceIterator.next(); + String device = deviceInfo.left; + boolean aligned = deviceInfo.right; + Map> deviceRecords = + alteringRecordsCache.getDeviceRecords(device); + if (deviceRecords == null || deviceRecords.isEmpty()) { + continue; + } + if (aligned) { + Set measurements = new HashSet<>(); + List alignedChunkMetadatas = reader.getAlignedChunkMetadata(device); + if (alignedChunkMetadatas == null || alignedChunkMetadatas.isEmpty()) { + logger.warn("[alter timeseries] device({}) alignedChunkMetadatas is null", device); + continue; + } + for (AlignedChunkMetadata alignedChunkMetadata : alignedChunkMetadatas) { + List valueChunkMetadataList = + alignedChunkMetadata.getValueChunkMetadataList(); + if (valueChunkMetadataList == null || valueChunkMetadataList.isEmpty()) { + continue; + } + for (IChunkMetadata chunkMetadata : valueChunkMetadataList) { + if (chunkMetadata == null) { + continue; + } + // Measurement may be distributed across many ChunkMetadata, using set to get the first + // one + boolean isFirst = measurements.add(chunkMetadata.getMeasurementUid()); + if (isFirst) { + Pair cacheType = + deviceRecords.get(chunkMetadata.getMeasurementUid()); + if (cacheType == null) { + continue; + } + Chunk chunk = reader.readMemChunk((ChunkMetadata) chunkMetadata); + if (isTypeDiff(cacheType, chunk)) return true; + } + } + } + } else { + Map> measurementMap = reader.readChunkMetadataInDevice(device); + if (measurementMap == null || measurementMap.isEmpty()) { + logger.warn("[alter timeseries] device({}) chunkMetadatas is null", device); + continue; + } + for (String measurement : measurementMap.keySet()) { + Pair cacheType = deviceRecords.get(measurement); + if (cacheType == null) { + continue; + } + List chunkMetadata = measurementMap.get(measurement); + if (chunkMetadata == null || chunkMetadata.isEmpty()) { + continue; + } + // read first chunk type + ChunkMetadata curChunkMetadata = chunkMetadata.get(0); + Chunk chunk = reader.readMemChunk(curChunkMetadata); + if (isTypeDiff(cacheType, chunk)) return true; + } + } + } + // If and only if the first chunk type of all measurements is the same + return false; + } + + private boolean isTypeDiff(Pair cacheType, Chunk chunk) + throws IOException { + if (chunk == null) { + throw new IOException("read chunk is null"); + } + ChunkHeader header = chunk.getHeader(); + if (header == null) { + throw new IOException("read chunk header is null"); + } + // find it + return header.getEncodingType() != cacheType.left + || header.getCompressionType() != cacheType.right; + } + + private void rewriteAlgined(TsFileSequenceReader reader, TsFileIOWriter writer, String device) + throws IOException, InterruptedException { + List alignedChunkMetadatas = reader.getAlignedChunkMetadata(device); + if (alignedChunkMetadatas == null || alignedChunkMetadatas.isEmpty()) { + logger.warn("[alter timeseries] device({}) alignedChunkMetadatas is null", device); + return; + } + Map> deviceRecords = + alteringRecordsCache.getDeviceRecords(device); + boolean isAlteringDevice = (deviceRecords != null && !deviceRecords.isEmpty()); + Pair, List>> schemaPair = + collectSchemaList(alignedChunkMetadatas, reader, deviceRecords, isAlteringDevice); + Boolean allSame = schemaPair.left; + if (!isAlteringDevice || allSame) { + // fast: rewrite chunk data to tsfile + alignedFastWrite(reader, writer, alignedChunkMetadatas); + } else { + // need to rewrite + alignedRewritePoints(reader, writer, device, alignedChunkMetadatas, schemaPair); + } + } + + private void alignedRewritePoints( + TsFileSequenceReader reader, + TsFileIOWriter writer, + String device, + List alignedChunkMetadatas, + Pair, List>> schemaPair) + throws IOException, InterruptedException { + Pair, List> listPair = schemaPair.right; + List schemaList = listPair.left; + List schemaOldList = listPair.right; + AlignedChunkWriterImpl chunkWriter = new AlignedChunkWriterImpl(schemaList); + TsFileAlignedSeriesReaderIterator readerIterator = + new TsFileAlignedSeriesReaderIterator(reader, alignedChunkMetadatas, schemaOldList); + + while (readerIterator.hasNext()) { + checkThreadInterrupted(); + Pair chunkReaderAndChunkSize = readerIterator.nextReader(); + AlignedChunkReader chunkReader = chunkReaderAndChunkSize.left; + while (chunkReader.hasNextSatisfiedPage()) { + IBatchDataIterator batchDataIterator = chunkReader.nextPageData().getBatchDataIterator(); + while (batchDataIterator.hasNext()) { + TsPrimitiveType[] pointsData = (TsPrimitiveType[]) batchDataIterator.currentValue(); + long time = batchDataIterator.currentTime(); + chunkWriter.write(time, pointsData); + targetTsFileResource.updateStartTime(device, time); + targetTsFileResource.updateEndTime(device, time); + batchDataIterator.next(); + } + } + } + chunkWriter.writeToFileWriter(writer); + } + + private void alignedFastWrite( + TsFileSequenceReader reader, + TsFileIOWriter writer, + List alignedChunkMetadatas) + throws IOException, InterruptedException { + for (AlignedChunkMetadata alignedChunkMetadata : alignedChunkMetadatas) { + // write time chunk + IChunkMetadata timeChunkMetadata = alignedChunkMetadata.getTimeChunkMetadata(); + Chunk timeChunk = reader.readMemChunk((ChunkMetadata) timeChunkMetadata); + writer.writeChunk(timeChunk, (ChunkMetadata) timeChunkMetadata); + // write value chunks + List valueChunkMetadataList = + alignedChunkMetadata.getValueChunkMetadataList(); + for (IChunkMetadata chunkMetadata : valueChunkMetadataList) { + checkThreadInterrupted(); + if (chunkMetadata == null) { + continue; + } + Chunk chunk = reader.readMemChunk((ChunkMetadata) chunkMetadata); + writer.writeChunk(chunk, (ChunkMetadata) chunkMetadata); + } + } + } + + protected Pair, List>> + collectSchemaList( + List alignedChunkMetadatas, + TsFileSequenceReader reader, + Map> deviceRecords, + boolean isAlteringDevice) + throws IOException { + List schemaList = new ArrayList<>(); + List schemaOldList = new ArrayList<>(); + Set measurementSet = new HashSet<>(); + boolean allSame = true; + for (AlignedChunkMetadata alignedChunkMetadata : alignedChunkMetadatas) { + List valueChunkMetadataList = + alignedChunkMetadata.getValueChunkMetadataList(); + for (IChunkMetadata chunkMetadata : valueChunkMetadataList) { + if (chunkMetadata == null) { + continue; + } + String measurementId = chunkMetadata.getMeasurementUid(); + if (measurementSet.contains(measurementId)) { + continue; + } + Pair alterType = null; + if (deviceRecords != null) { + alterType = deviceRecords.get(measurementId); + } + measurementSet.add(measurementId); + Chunk chunk = reader.readMemChunk((ChunkMetadata) chunkMetadata); + ChunkHeader header = chunk.getHeader(); + boolean findTarget = isAlteringDevice && alterType != null; + MeasurementSchema measurementSchema = + new MeasurementSchema( + header.getMeasurementID(), + header.getDataType(), + header.getEncodingType(), + header.getCompressionType()); + if (!findTarget) { + schemaList.add(measurementSchema); + } else { + allSame = false; + schemaList.add( + new MeasurementSchema( + header.getMeasurementID(), + header.getDataType(), + alterType.left, + alterType.right)); + } + schemaOldList.add(measurementSchema); + } + } + + schemaList.sort(Comparator.comparing(IMeasurementSchema::getMeasurementId)); + schemaOldList.sort(Comparator.comparing(IMeasurementSchema::getMeasurementId)); + return new Pair<>(allSame, new Pair<>(schemaList, schemaOldList)); + } + + private void rewriteNotAligned(String device, TsFileSequenceReader reader, TsFileIOWriter writer) + throws IOException, MetadataException { + Map> measurementMap = reader.readChunkMetadataInDevice(device); + if (measurementMap == null) { + logger.warn("[alter timeseries] device({}) measurementMap is null", device); + return; + } + Map> deviceRecords = + alteringRecordsCache.getDeviceRecords(device); + for (Map.Entry> next : measurementMap.entrySet()) { + String measurementId = next.getKey(); + List chunkMetadatas = next.getValue(); + if (chunkMetadatas == null || chunkMetadatas.isEmpty()) { + logger.warn("[alter timeseries] empty measurement({})", measurementId); + return; + } + // target chunk writer + ChunkMetadata firstChunkMetadata = chunkMetadatas.get(0); + ChunkWriterImpl chunkWriter = null; + Pair alterType = null; + if (deviceRecords != null) { + alterType = deviceRecords.get(measurementId); + } + for (ChunkMetadata chunkMetadata : chunkMetadatas) { + // old mem chunk + Chunk currentChunk = reader.readMemChunk(chunkMetadata); + ChunkHeader header = currentChunk.getHeader(); + TSEncoding curEncoding = header.getEncodingType(); + CompressionType curCompressionType = header.getCompressionType(); + if (chunkWriter == null) { + // chunkWriter init + if (alterType != null + && (alterType.left != curEncoding || alterType.right != curCompressionType)) { + curEncoding = alterType.left; + curCompressionType = alterType.right; + } + chunkWriter = + new ChunkWriterImpl( + new MeasurementSchema( + measurementId, + firstChunkMetadata.getDataType(), + curEncoding, + curCompressionType)); + } + if (alterType == null + || (chunkWriter.getEncoding() == header.getEncodingType() + && chunkWriter.getCompressionType() == header.getCompressionType())) { + // fast write chunk + writer.writeChunk(currentChunk, chunkMetadata); + continue; + } + IChunkReader chunkReader = new ChunkReader(currentChunk, null); + while (chunkReader.hasNextSatisfiedPage()) { + IPointReader batchIterator = chunkReader.nextPageData().getBatchDataIterator(); + while (batchIterator.hasNextTimeValuePair()) { + TimeValuePair timeValuePair = batchIterator.nextTimeValuePair(); + writeTimeAndValueToChunkWriter(chunkWriter, timeValuePair); + if (timeValuePair.getTimestamp() > maxEndTimestamp) { + maxEndTimestamp = timeValuePair.getTimestamp(); + } + if (timeValuePair.getTimestamp() < minStartTimestamp) { + minStartTimestamp = timeValuePair.getTimestamp(); + } + } + } + // flush + chunkWriter.writeToFileWriter(writer); + } + } + } + + /** TODO Copy it from the compaction code and extract it later into the utility class */ + private void writeTimeAndValueToChunkWriter( + ChunkWriterImpl chunkWriter, TimeValuePair timeValuePair) { + switch (chunkWriter.getDataType()) { + case TEXT: + chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getBinary()); + break; + case FLOAT: + chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getFloat()); + break; + case DOUBLE: + chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getDouble()); + break; + case BOOLEAN: + chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getBoolean()); + break; + case INT64: + chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getLong()); + break; + case INT32: + chunkWriter.write(timeValuePair.getTimestamp(), timeValuePair.getValue().getInt()); + break; + case VECTOR: + break; + default: + throw new UnsupportedOperationException("Unknown data type " + chunkWriter.getDataType()); + } + } + + public boolean hasRewrite() { + return hasRewrite; + } + + private void checkThreadInterrupted() throws InterruptedException { + if (Thread.interrupted() || summary.isCancel()) { + throw new InterruptedException( + String.format( + "[alter timeseries] rewrite for target file %s abort", + targetTsFileResource.toString())); + } + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/engine/alter/log/AlteringLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/engine/alter/log/AlteringLogAnalyzer.java new file mode 100644 index 0000000000000..ae090d4dce28e --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/engine/alter/log/AlteringLogAnalyzer.java @@ -0,0 +1,93 @@ +/* + * 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.db.engine.alter.log; + +import org.apache.iotdb.db.engine.compaction.log.TsFileIdentifier; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; +import org.apache.iotdb.tsfile.utils.Pair; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileReader; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** altering log analyzer */ +public class AlteringLogAnalyzer { + + // private static final Logger logger = LoggerFactory.getLogger(AlteringLogAnalyzer.class); + + private final File alterLogFile; + private final List>> alterList = + new ArrayList<>(4); + private boolean clearBegin = false; + private final Set doneFiles = new HashSet<>(32); + + public AlteringLogAnalyzer(File alterLogFile) { + this.alterLogFile = alterLogFile; + } + + public void analyzer() throws IOException { + + try (BufferedReader bufferedReader = new BufferedReader(new FileReader(alterLogFile))) { + + String mark; + while ((mark = bufferedReader.readLine()) != null) { + if (AlteringLogger.FLAG_ALTER_PARAM_BEGIN.equals(mark)) { + String fullPath = bufferedReader.readLine(); + if (fullPath == null) { + throw new IOException("alter.log parse fail, fullPath is null"); + } + TSEncoding encoding = TSEncoding.deserialize(Byte.parseByte(bufferedReader.readLine())); + if (encoding == null) { + throw new IOException("alter.log parse fail, encoding is null"); + } + CompressionType compressionType = + CompressionType.deserialize(Byte.parseByte(bufferedReader.readLine())); + if (compressionType == null) { + throw new IOException("alter.log parse fail, compressionType is null"); + } + alterList.add(new Pair<>(fullPath, new Pair<>(encoding, compressionType))); + } else if (AlteringLogger.FLAG_CLEAR_BEGIN.equals(mark)) { + clearBegin = true; + } else if (AlteringLogger.FLAG_DONE.equals(mark)) { + String curLineStr = bufferedReader.readLine(); + doneFiles.add(TsFileIdentifier.getFileIdentifierFromInfoString(curLineStr)); + } + } + } + } + + public List>> getAlterList() { + return alterList; + } + + public boolean isClearBegin() { + return clearBegin; + } + + public Set getDoneFiles() { + return doneFiles; + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/engine/alter/log/AlteringLogger.java b/server/src/main/java/org/apache/iotdb/db/engine/alter/log/AlteringLogger.java new file mode 100644 index 0000000000000..7e5b12abe36b6 --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/engine/alter/log/AlteringLogger.java @@ -0,0 +1,105 @@ +/* + * 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.db.engine.alter.log; + +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.engine.compaction.log.TsFileIdentifier; +import org.apache.iotdb.db.engine.storagegroup.TsFileResource; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; + +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; + +/** + * Alteringlogger records the progress of modifying the encoding compression method in the form of + * text lines in the file "alter.log". + */ +public class AlteringLogger implements AutoCloseable { + + public static final String ALTERING_LOG_NAME = "alter.log"; + public static final String FLAG_DONE = "done"; + public static final String FLAG_ALTER_PARAM_BEGIN = "apb"; + public static final String FLAG_CLEAR_BEGIN = "cb"; + + private final BufferedWriter logStream; + + public AlteringLogger(File logFile) throws IOException { + logStream = new BufferedWriter(new FileWriter(logFile, true)); + } + + @Override + public void close() throws IOException { + logStream.close(); + } + + public synchronized void addAlterParam( + PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType) + throws IOException { + if (fullPath == null || curEncoding == null || curCompressionType == null) { + throw new IOException("alter params is null"); + } + logStream.write(FLAG_ALTER_PARAM_BEGIN); + logStream.newLine(); + logStream.write(fullPath.getFullPath()); + logStream.newLine(); + logStream.write(Byte.toString(curEncoding.serialize())); + logStream.newLine(); + logStream.write(Byte.toString(curCompressionType.serialize())); + logStream.newLine(); + logStream.flush(); + } + + public static void clearBegin(File logFile) throws IOException { + + try (BufferedWriter tempLogWriter = new BufferedWriter(new FileWriter(logFile, true))) { + tempLogWriter.write(FLAG_CLEAR_BEGIN); + tempLogWriter.newLine(); + tempLogWriter.flush(); + } + } + + public void clearBegin() throws IOException { + + logStream.write(FLAG_CLEAR_BEGIN); + logStream.newLine(); + logStream.flush(); + } + + /** + * need to be thread-safe + * + * @param file + * @throws IOException + */ + public synchronized void doneFile(TsFileResource file) throws IOException { + if (file == null) { + throw new IOException("file is null"); + } + logStream.write(FLAG_DONE); + logStream.newLine(); + logStream.write( + "" + TsFileIdentifier.getFileIdentifierFromFilePath(file.getTsFile().getAbsolutePath())); + logStream.newLine(); + logStream.flush(); + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/engine/cache/AlteringRecordsCache.java b/server/src/main/java/org/apache/iotdb/db/engine/cache/AlteringRecordsCache.java new file mode 100644 index 0000000000000..809fe1ea3ec10 --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/engine/cache/AlteringRecordsCache.java @@ -0,0 +1,170 @@ +/* + * 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.db.engine.cache; + +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.db.engine.StorageEngine; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; +import org.apache.iotdb.tsfile.utils.Pair; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicBoolean; + +/** This class is used to cache Altering Timeseries. */ +public class AlteringRecordsCache { + + private static final Logger logger = LoggerFactory.getLogger(AlteringRecordsCache.class); + + // measurement->(encoding, compression) + private final Map> alteringRecords = + new ConcurrentHashMap<>(32); + // device->measurement->(encoding, compression) + private final Map>> alteringDeviceRecords = + new ConcurrentHashMap<>(32); + // sg->deviceIds + private final Map> sgDeviceMap = new ConcurrentHashMap<>(2); + + private final AtomicBoolean isAltering = new AtomicBoolean(false); + + private AlteringRecordsCache() {} + + public synchronized void startAlter() { + isAltering.set(true); + } + + public synchronized void putRecord( + String fullPath, TSEncoding encoding, CompressionType compressionType) throws Exception { + if (fullPath != null) { + PartialPath path = new PartialPath(fullPath); + String storageGroupName = StorageEngine.getInstance().getStorageGroupName(path); + putRecord(storageGroupName, fullPath, encoding, compressionType); + } + } + + /** + * only for recover + * + * @param storageGroupName + * @param fullPath + * @param encoding + * @param compressionType + * @throws Exception + */ + public synchronized void putRecord( + String storageGroupName, + String fullPath, + TSEncoding encoding, + CompressionType compressionType) + throws Exception { + if (fullPath != null) { + PartialPath path = new PartialPath(fullPath); + String device = path.getDevice(); + Set devices = + sgDeviceMap.computeIfAbsent( + storageGroupName, id -> Collections.synchronizedSet(new HashSet<>())); + devices.add(device); + Pair record = new Pair<>(encoding, compressionType); + alteringRecords.put(fullPath, record); + Map> deviceRecordMap = + alteringDeviceRecords.computeIfAbsent(device, id -> new ConcurrentHashMap<>()); + deviceRecordMap.put(path.getMeasurement(), record); + } + } + + public Set getDevicesCache(String storageGroupName) { + return sgDeviceMap.get(storageGroupName); + } + + public Pair getRecord(String fullPath) { + + if (!isAltering.get()) { + return null; + } + return alteringRecords.get(fullPath); + } + + public Map> getDeviceRecords(String device) { + return alteringDeviceRecords.get(device); + } + + public boolean isStorageGroupExsist(String storageGroupName) { + + if (storageGroupName == null) { + return false; + } + return sgDeviceMap.get(storageGroupName) != null; + } + + public static AlteringRecordsCache getInstance() { + return AlteringRecordsCacheHolder.INSTANCE; + } + + public synchronized void clear(String storageGroupName) { + + if (!isStorageGroupExsist(storageGroupName)) { + return; + } + Set devices = sgDeviceMap.get(storageGroupName); + if (devices != null) { + sgDeviceMap.remove(storageGroupName); + devices.forEach( + device -> { + Map> deviceMap = + alteringDeviceRecords.get(device); + if (deviceMap != null) { + alteringDeviceRecords.remove(device); + deviceMap.forEach( + (k, v) -> { + try { + PartialPath fullPath = new PartialPath(device, k); + alteringRecords.remove(fullPath); + } catch (IllegalPathException e) { + logger.error("fullPath error!!!!!", e); + } + }); + } + }); + } + if (sgDeviceMap.isEmpty()) { + isAltering.set(false); + } + } + + @TestOnly + public boolean isAltering() { + return isAltering.get(); + } + + /** singleton pattern. */ + private static class AlteringRecordsCacheHolder { + + private static final AlteringRecordsCache INSTANCE = new AlteringRecordsCache(); + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManager.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManager.java index edd79a608c2e8..279dce51c3301 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManager.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionTaskManager.java @@ -407,4 +407,17 @@ public Future getCompactionTaskFutureMayBlock(AbstractCom } return storageGroupTasks.get(regionWithSG).get(task); } + + public Future getCompactionTaskFutureCheckStatusMayBlock( + AbstractCompactionTask task) throws InterruptedException { + String regionWithSG = getSGWithRegionId(task.getStorageGroupName(), task.getDataRegionId()); + while (!storageGroupTasks.containsKey(regionWithSG) + || !storageGroupTasks.get(regionWithSG).containsKey(task)) { + if (task.isTaskFinished()) { + return null; + } + Thread.sleep(10); + } + return storageGroupTasks.get(regionWithSG).get(task); + } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionUtils.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionUtils.java index d4b5a33e608db..e36bef2d6ff48 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionUtils.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/CompactionUtils.java @@ -196,6 +196,17 @@ public static boolean deleteTsFile(TsFileResource seqFile) { return true; } + public static boolean deleteTsFileWithoutMods(TsFileResource seqFile) { + try { + FileReaderManager.getInstance().closeFileAndRemoveReader(seqFile.getTsFilePath()); + seqFile.removeWithoutMods(); + } catch (IOException e) { + logger.error(e.getMessage(), e); + return false; + } + return true; + } + /** Delete all modification files for source files */ public static void deleteModificationForSourceFile( Collection sourceFiles, String storageGroupName) throws IOException { diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/utils/SingleSeriesCompactionExecutor.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/utils/SingleSeriesCompactionExecutor.java index d1d4a366e7f2e..557cc22e3ca74 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/utils/SingleSeriesCompactionExecutor.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/inner/utils/SingleSeriesCompactionExecutor.java @@ -20,6 +20,7 @@ import org.apache.iotdb.commons.path.PartialPath; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.engine.cache.AlteringRecordsCache; import org.apache.iotdb.db.engine.compaction.CompactionTaskManager; import org.apache.iotdb.db.engine.compaction.constant.CompactionType; import org.apache.iotdb.db.engine.compaction.constant.ProcessChunkType; @@ -27,6 +28,8 @@ import org.apache.iotdb.db.service.metrics.recorder.CompactionMetricsRecorder; import org.apache.iotdb.tsfile.file.header.ChunkHeader; import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.read.TimeValuePair; import org.apache.iotdb.tsfile.read.TsFileSequenceReader; import org.apache.iotdb.tsfile.read.common.Chunk; @@ -144,6 +147,28 @@ public void execute() throws IOException { } // after all the chunk of this sensor is read, flush the remaining data + forceFlush(); + } + + private boolean isTypeChanged(Chunk lastChunk, Chunk currentChunk) { + + if (lastChunk == null || currentChunk == null) { + return false; + } + if (currentChunk.getHeader() == null && lastChunk.getHeader() == null) { + return false; + } + boolean encodingEqule = + currentChunk.getHeader().getEncodingType() == lastChunk.getHeader().getEncodingType(); + boolean compressionTypeEqule = + currentChunk.getHeader().getCompressionType() == lastChunk.getHeader().getCompressionType(); + if (!encodingEqule || !compressionTypeEqule) { + return true; + } + return false; + } + + private void forceFlush() throws IOException { if (cachedChunk != null) { flushChunkToFileWriter(cachedChunk, cachedChunkMetadata, true); cachedChunk = null; @@ -158,12 +183,18 @@ public void execute() throws IOException { private void constructChunkWriterFromReadChunk(Chunk chunk) { ChunkHeader chunkHeader = chunk.getHeader(); + // Gets the altering encoding/compressed timeseries record in the cache + Pair cacheRecord = + AlteringRecordsCache.getInstance().getRecord(series.getMeasurement()); + TSEncoding encoding = chunkHeader.getEncodingType(); + CompressionType compressionType = chunkHeader.getCompressionType(); + if (cacheRecord != null) { + encoding = cacheRecord.left; + compressionType = cacheRecord.right; + } this.schema = new MeasurementSchema( - series.getMeasurement(), - chunkHeader.getDataType(), - chunkHeader.getEncodingType(), - chunkHeader.getCompressionType()); + series.getMeasurement(), chunkHeader.getDataType(), encoding, compressionType); this.chunkWriter = new ChunkWriterImpl(this.schema); } @@ -188,9 +219,18 @@ private void processLargeChunk(Chunk chunk, ChunkMetadata chunkMetadata) throws writeChunkIntoChunkWriter(chunk); flushChunkWriterIfLargeEnough(); } else if (cachedChunk != null) { - // if there is a cached chunk, merge it with current chunk, then flush it - mergeWithCachedChunk(chunk, chunkMetadata); - flushCachedChunkIfLargeEnough(); + // Fix: Merging chunks can be problematic when the same measurement has different + // encoding/compression + if (!isTypeChanged(cachedChunk, chunk)) { + // if there is a cached chunk, merge it with current chunk, then flush it + mergeWithCachedChunk(chunk, chunkMetadata); + flushCachedChunkIfLargeEnough(); + } else { + // If Encoding /compression changes, force Flush to put the current Chunk into the cache + forceFlush(); + cachedChunk = chunk; + cachedChunkMetadata = chunkMetadata; + } } else { // there is no points remaining in ChunkWriter and no cached chunk // flush it to file directly @@ -206,9 +246,18 @@ private void processMiddleChunk(Chunk chunk, ChunkMetadata chunkMetadata) throws writeChunkIntoChunkWriter(chunk); flushChunkWriterIfLargeEnough(); } else if (cachedChunk != null) { - // if there is a cached chunk, merge it with current chunk - mergeWithCachedChunk(chunk, chunkMetadata); - flushCachedChunkIfLargeEnough(); + // Fix: Merging chunks can be problematic when the same measurement has different + // encoding/compression + if (!isTypeChanged(cachedChunk, chunk)) { + // if there is a cached chunk, merge it with current chunk + mergeWithCachedChunk(chunk, chunkMetadata); + flushCachedChunkIfLargeEnough(); + } else { + // If Encoding /compression changes, force Flush to put the current Chunk into the cache + forceFlush(); + cachedChunk = chunk; + cachedChunkMetadata = chunkMetadata; + } } else { // there is no points remaining in ChunkWriter and no cached chunk // cached current chunk diff --git a/server/src/main/java/org/apache/iotdb/db/engine/compaction/log/TsFileIdentifier.java b/server/src/main/java/org/apache/iotdb/db/engine/compaction/log/TsFileIdentifier.java index 2d14f96bf1c8f..dec0ca3c04e87 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/compaction/log/TsFileIdentifier.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/compaction/log/TsFileIdentifier.java @@ -23,6 +23,10 @@ import java.io.File; +import static org.apache.iotdb.commons.conf.IoTDBConstant.ALTER_OLD_TMP_FILE_SUFFIX; +import static org.apache.iotdb.commons.conf.IoTDBConstant.ALTER_TMP_FILE_SUFFIX; +import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX; + /** * This class record the logical information of files in compaction, which is used to locate file in * disk. File identifier includes whether the file is sequence, its database name, virtual database @@ -200,6 +204,30 @@ public File getFileFromDataDirs() { return null; } + public File getAlterFileFromDataDirs() { + String[] dataDirs = IoTDBDescriptor.getInstance().getConfig().getDataDirs(); + String alterFilePath = getFilePath().replace(TSFILE_SUFFIX, ALTER_TMP_FILE_SUFFIX); + for (String dataDir : dataDirs) { + File file = new File(dataDir, alterFilePath); + if (file.exists()) { + return file; + } + } + return null; + } + + public File getAlterOldFileFromDataDirs() { + String[] dataDirs = IoTDBDescriptor.getInstance().getConfig().getDataDirs(); + String alterOldFilePath = getFilePath().replace(TSFILE_SUFFIX, ALTER_OLD_TMP_FILE_SUFFIX); + for (String dataDir : dataDirs) { + File file = new File(dataDir, alterOldFilePath); + if (file.exists()) { + return file; + } + } + return null; + } + public String getFilename() { return filename; } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java index 964c423f07113..54d5b76a3ac91 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/DataRegion.java @@ -38,9 +38,13 @@ import org.apache.iotdb.db.conf.directories.DirectoryManager; import org.apache.iotdb.db.engine.StorageEngine; import org.apache.iotdb.db.engine.TsFileMetricManager; +import org.apache.iotdb.db.engine.alter.log.AlteringLogAnalyzer; +import org.apache.iotdb.db.engine.alter.log.AlteringLogger; +import org.apache.iotdb.db.engine.cache.AlteringRecordsCache; import org.apache.iotdb.db.engine.compaction.CompactionRecoverManager; import org.apache.iotdb.db.engine.compaction.CompactionScheduler; import org.apache.iotdb.db.engine.compaction.CompactionTaskManager; +import org.apache.iotdb.db.engine.compaction.log.TsFileIdentifier; import org.apache.iotdb.db.engine.compaction.task.AbstractCompactionTask; import org.apache.iotdb.db.engine.flush.CloseFileListener; import org.apache.iotdb.db.engine.flush.FlushListener; @@ -95,6 +99,8 @@ import org.apache.iotdb.rpc.RpcUtils; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer; import org.apache.iotdb.tsfile.fileSystem.fsFactory.FSFactory; import org.apache.iotdb.tsfile.read.filter.basic.Filter; @@ -130,7 +136,12 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import static org.apache.iotdb.commons.conf.IoTDBConstant.ALTER_OLD_TMP_FILE_RESOURCE_SUFFIX; +import static org.apache.iotdb.commons.conf.IoTDBConstant.ALTER_OLD_TMP_FILE_SUFFIX; +import static org.apache.iotdb.commons.conf.IoTDBConstant.ALTER_TMP_FILE_RESOURCE_SUFFIX; +import static org.apache.iotdb.commons.conf.IoTDBConstant.ALTER_TMP_FILE_SUFFIX; import static org.apache.iotdb.commons.conf.IoTDBConstant.FILE_NAME_SEPARATOR; +import static org.apache.iotdb.db.engine.storagegroup.TsFileResource.RESOURCE_SUFFIX; import static org.apache.iotdb.db.engine.storagegroup.TsFileResource.TEMP_SUFFIX; import static org.apache.iotdb.db.qp.executor.PlanExecutor.operateClearCache; import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX; @@ -267,6 +278,9 @@ public class DataRegion implements IDataRegionForQuery { private IDTable idTable; + /** used to cache Altering Timeseries */ + private AlteringRecordsCache alteringRecordsCache = AlteringRecordsCache.getInstance(); + /** * constrcut a database processor * @@ -425,12 +439,12 @@ private void recover() throws DataRegionException { try { // collect candidate TsFiles from sequential and unsequential data directory Pair, List> seqTsFilesPair = - getAllFiles(DirectoryManager.getInstance().getAllSequenceFileFolders()); + getAllFiles(DirectoryManager.getInstance().getAllSequenceFileFolders(), true); List tmpSeqTsFiles = seqTsFilesPair.left; List oldSeqTsFiles = seqTsFilesPair.right; upgradeSeqFileList.addAll(oldSeqTsFiles); Pair, List> unseqTsFilesPair = - getAllFiles(DirectoryManager.getInstance().getAllUnSequenceFileFolders()); + getAllFiles(DirectoryManager.getInstance().getAllUnSequenceFileFolders(), false); List tmpUnseqTsFiles = unseqTsFilesPair.left; List oldUnseqTsFiles = unseqTsFilesPair.right; upgradeUnseqFileList.addAll(oldUnseqTsFiles); @@ -547,6 +561,25 @@ private void recover() throws DataRegionException { throw new DataRegionException(e); } + List seqTsFileResources = tsFileManager.getTsFileList(true); + for (TsFileResource resource : seqTsFileResources) { + long timePartitionId = resource.getTimePartition(); + Map endTimeMap = new HashMap<>(); + for (String deviceId : resource.getDevices()) { + long endTime = resource.getEndTime(deviceId); + endTimeMap.put(deviceId.intern(), endTime); + } + lastFlushTimeManager.setMultiDeviceLastTime(timePartitionId, endTimeMap); + lastFlushTimeManager.setMultiDeviceFlushedTime(timePartitionId, endTimeMap); + lastFlushTimeManager.setMultiDeviceGlobalFlushedTime(endTimeMap); + } + + // recover alter records cache + try { + recoverAlter(); + } catch (IOException e) { + throw new DataRegionException(e); + } // recover and start timed compaction thread initCompaction(); @@ -557,6 +590,41 @@ private void recover() throws DataRegionException { } } + private void recoverAlter() throws DataRegionException, IOException { + + final String logKey = this.getStorageGroupPath(); + // alter.log analyzer + File logFile = + SystemFileFactory.INSTANCE.getFile(storageGroupSysDir, AlteringLogger.ALTERING_LOG_NAME); + if (!logFile.exists()) { + return; + } + AlteringLogAnalyzer analyzer = new AlteringLogAnalyzer(logFile); + analyzer.analyzer(); + List>> alterList = analyzer.getAlterList(); + if (alterList == null || alterList.size() <= 0) { + try { + logger.warn("recoverAlter-{}: An empty alter.log exists", logKey); + fsFactory.deleteIfExists(logFile); + return; + } catch (IOException e) { + throw new DataRegionException(e); + } + } + // recover alter records cache + alteringRecordsCache.startAlter(); + for (int i = 0; i < alterList.size(); i++) { + Pair> record = alterList.get(i); + try { + alteringRecordsCache.putRecord( + storageGroupName, record.left, record.right.left, record.right.right); + } catch (Exception e) { + throw new DataRegionException(e); + } + } + logger.info("recoverAlter-{}: record count is {}", logKey, alterList.size()); + } + private void updateLastFlushTime(TsFileResource resource, boolean isSeq) { // only update flush time when it is a seq file if (isSeq) { @@ -646,8 +714,8 @@ private void updateLatestFlushedTime() throws IOException { } @SuppressWarnings("squid:S3776") // Suppress high Cognitive Complexity warning - private Pair, List> getAllFiles(List folders) - throws IOException, DataRegionException { + private Pair, List> getAllFiles( + List folders, boolean isSeq) throws IOException, DataRegionException { List tsFiles = new ArrayList<>(); List upgradeFiles = new ArrayList<>(); for (String baseDir : folders) { @@ -660,6 +728,11 @@ private Pair, List> getAllFiles(List, List> getAllFiles(List(ret, upgradeRet); } + /** + * tsfile repaired
+ * 1、.tfile .alter -> del:.alter
+ * 2、.alter .alter.old -> rename:.alter.old to .tsfile del:.alter
+ */ + private void alteringFailRenames(File fileFolder) { + File[] files = fsFactory.listFilesBySuffix(fileFolder.getAbsolutePath(), ALTER_TMP_FILE_SUFFIX); + if (files != null) { + for (File tempResource : files) { + File tsFile = + fsFactory.getFile(tempResource.getPath().replace(ALTER_TMP_FILE_SUFFIX, TSFILE_SUFFIX)); + File resource = + fsFactory.getFile( + tempResource.getPath().replace(ALTER_TMP_FILE_SUFFIX, RESOURCE_SUFFIX)); + File alterResouce = + fsFactory.getFile( + tempResource + .getPath() + .replace(ALTER_TMP_FILE_SUFFIX, ALTER_TMP_FILE_RESOURCE_SUFFIX)); + File alterOldTsFile = + fsFactory.getFile( + tempResource.getPath().replace(ALTER_TMP_FILE_SUFFIX, ALTER_OLD_TMP_FILE_SUFFIX)); + File alterOldResource = + fsFactory.getFile( + tempResource + .getPath() + .replace(ALTER_TMP_FILE_SUFFIX, ALTER_OLD_TMP_FILE_RESOURCE_SUFFIX)); + if (tsFile.exists()) { + tempResource.deleteOnExit(); + alterResouce.deleteOnExit(); + continue; + } + if (alterOldTsFile.exists()) { + alterOldTsFile.renameTo(tsFile); + alterOldResource.renameTo(resource); + tempResource.deleteOnExit(); + alterResouce.deleteOnExit(); + continue; + } + } + } + } + private void continueFailedRenames(File fileFolder, String suffix) { File[] files = fsFactory.listFilesBySuffix(fileFolder.getAbsolutePath(), suffix); if (files != null) { @@ -1818,6 +1934,68 @@ private List getFileResourceListForQuery( return tsfileResourcesForQuery; } + /** + * alter timeseries encoding & compressionType
+ * 1、flush and close tsfile
+ * 2、locks
+ * 3、write temp tsfiles
+ * 4、unregister old tsfiles and release locks
+ * 5、rename temp tsfiles<br> 6、register tsfiles
+ */ + public void alter( + PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType) + throws IOException { + + // final String logKey = + // storageGroupName + "-" + dataRegionId + "-" + fullPath.getFullPath(); + // If there are still some old version tsfiles, the delete won't succeeded. + if (upgradeFileCount.get() != 0) { + throw new IOException("Alter failed. Please do not delete until the old files upgraded."); + } + if (SettleService.getINSTANCE().getFilesToBeSettledCount().get() != 0) { + throw new IOException("Alter failed. Please do not delete until the old files settled."); + } + // log and cache altering record + File logFile = + SystemFileFactory.INSTANCE.getFile(storageGroupSysDir, AlteringLogger.ALTERING_LOG_NAME); + if (!logFile.exists()) { + logFile.createNewFile(); + } + try (AlteringLogger alteringLogger = new AlteringLogger(logFile)) { + alteringLogger.addAlterParam(fullPath, curEncoding, curCompressionType); + alteringRecordsCache.putRecord(fullPath.getFullPath(), curEncoding, curCompressionType); + } catch (Exception e) { + throw new IOException("Alter failed.", e); + } + // flush & close TODO Both merge and clear will be rewritten using the new code(TSEncoding & + // CompressionType), so there is no need to waste time forcing disk flushing to modify Schema + // operations + // logger.info("[alter timeseries] {} syncCloseAllWorkingTsFileProcessors", logKey); + // syncCloseAllWorkingTsFileProcessors(); + // !! Split into alter and clear operations + } + + private boolean findUndoneResourcesAndRemove( + TsFileResource tsFileResource, Set undoneFiles) { + + if (undoneFiles == null || undoneFiles.isEmpty()) { + // all undone + return true; + } + TsFileIdentifier undoneFile = null; + for (TsFileIdentifier next : undoneFiles) { + if (next.getFilename().equals(tsFileResource.getTsFile().getName())) { + undoneFile = next; + break; + } + } + if (undoneFile != null) { + undoneFiles.remove(undoneFile); + return true; + } + return false; + } + /** Seperate tsfiles in TsFileManager to sealedList and unsealedList. */ private void separateTsFile( List sealedResource, List unsealedResource) { @@ -2913,6 +3091,14 @@ public String getStorageGroupPath() { return databaseName + File.separator + dataRegionId; } + public File getStorageGroupSysDir() { + return storageGroupSysDir; + } + + public StorageGroupInfo getStorageGroupInfo() { + return storageGroupInfo; + } + /** * Check if the data of "tsFileResource" all exist locally by comparing planIndexes in the * partition of "partitionNumber". This is available only when the IoTDB instances which generated diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileNameGenerator.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileNameGenerator.java index 8c15c99ed6c67..24b0642683519 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileNameGenerator.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileNameGenerator.java @@ -262,6 +262,26 @@ public static TsFileResource getInnerCompactionTargetFileResource( + IoTDBConstant.INNER_COMPACTION_TMP_FILE_SUFFIX)); } + public static TsFileResource generateNewAlterTsFileResource(TsFileResource oldTsFileResource) + throws IOException { + if (oldTsFileResource != null) { + TsFileName tsFileName = getTsFileName(oldTsFileResource.getTsFile().getName()); + File file = + new File( + oldTsFileResource.getTsFile().getParent(), + tsFileName.time + + FILE_NAME_SEPARATOR + + tsFileName.version + + FILE_NAME_SEPARATOR + + tsFileName.innerCompactionCnt + + FILE_NAME_SEPARATOR + + tsFileName.crossCompactionCnt + + IoTDBConstant.ALTER_TMP_FILE_SUFFIX); + return new TsFileResource(file); + } + return null; + } + public static class TsFileName { private static final String FILE_NAME_PATTERN = "(\\d+)-(\\d+)-(\\d+)-(\\d+).tsfile$"; private static final Pattern FILE_NAME_MATCHER = Pattern.compile(TsFileName.FILE_NAME_PATTERN); diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index a73ada0e331d8..c633146473065 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -531,6 +531,20 @@ public boolean remove() { return true; } + /** Remove the data file, its resource file physically. */ + public boolean removeWithoutMods() { + try { + fsFactory.deleteIfExists(file); + } catch (IOException e) { + LOGGER.error("TsFile {} cannot be deleted: {}", file, e.getMessage()); + return false; + } + if (!removeResourceFile()) { + return false; + } + return true; + } + public boolean removeResourceFile() { try { fsFactory.deleteIfExists(fsFactory.getFile(file.getPath() + RESOURCE_SUFFIX)); @@ -1087,4 +1101,28 @@ public void updateEndTime(Map times) { public boolean isFileInList() { return prev != null || next != null; } + + public void moveTsFile(String oldFileSuffix, String newFileSuffix) throws IOException { + + writeLock(); + try { + // move to target file and delete old tmp target file + if (!getTsFile().exists()) { + return; + } + File newFile = new File(getTsFilePath().replace(oldFileSuffix, newFileSuffix)); + if (!newFile.exists()) { + FSFactoryProducer.getFSFactory().moveFile(getTsFile(), newFile); + } + + // serialize xxx.tsfile.resource + setFile(newFile); + serialize(); + close(); + } catch (Exception e) { + throw e; + } finally { + writeUnlock(); + } + } } diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/LocalSchemaProcessor.java b/server/src/main/java/org/apache/iotdb/db/metadata/LocalSchemaProcessor.java index 4549b753019a8..ec50dce9b4a99 100644 --- a/server/src/main/java/org/apache/iotdb/db/metadata/LocalSchemaProcessor.java +++ b/server/src/main/java/org/apache/iotdb/db/metadata/LocalSchemaProcessor.java @@ -237,6 +237,23 @@ public void createAlignedTimeSeries(CreateAlignedTimeSeriesPlan plan) throws Met getBelongedSchemaRegionWithAutoCreate(plan.getDevicePath()).createAlignedTimeSeries(plan); } + /** + * alter timeseris encoding & compression type + * + * @param fullPath timeseries full path + * @param curEncoding + * @param curCompressionType + * @throws QueryProcessException + * @throws IOException + * @throws MetadataException + */ + public Pair alterTimeseries( + PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType) + throws MetadataException, IOException { + return getBelongedSchemaRegion(fullPath) + .alterTimeseriesEncodingCompressionType(fullPath, curEncoding, curCompressionType); + } + /** * Delete all timeseries matching the given path pattern, may cross different database. If using * prefix match, the path pattern is used to match prefix path. All timeseries start with the diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/InsertMeasurementMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/InsertMeasurementMNode.java index 285000cda2158..96fb0d518614b 100644 --- a/server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/InsertMeasurementMNode.java +++ b/server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/InsertMeasurementMNode.java @@ -29,10 +29,14 @@ import org.apache.iotdb.db.metadata.mnode.container.IMNodeContainer; import org.apache.iotdb.db.metadata.mnode.visitor.MNodeVisitor; import org.apache.iotdb.db.metadata.mtree.store.disk.cache.CacheEntry; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema; import org.apache.iotdb.tsfile.write.schema.MeasurementSchema; +import java.util.Map; + /** * Generated entity implements IMeasurementMNode interface to unify insert logic through id table * and SchemaProcessor @@ -62,6 +66,15 @@ public boolean isPreDeleted() { @Override public void setPreDeleted(boolean preDeleted) {} + @Override + public void updateSchemaInfo( + String measurementId, + TSEncoding encoding, + CompressionType compressionType, + Map props) { + throw new UnsupportedOperationException("only for alter timeSeries"); + } + @Override public IMeasurementSchema getSchema() { return schema; diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMeasurementMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMeasurementMNode.java index 5f4cccdf93795..5478467345c67 100644 --- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMeasurementMNode.java +++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/IMeasurementMNode.java @@ -19,9 +19,13 @@ package org.apache.iotdb.db.metadata.mnode; import org.apache.iotdb.commons.path.MeasurementPath; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema; +import java.util.Map; + /** This interface defines a MeasurementMNode's operation interfaces. */ public interface IMeasurementMNode extends IMNode { @@ -42,6 +46,20 @@ public interface IMeasurementMNode extends IMNode { void setOffset(long offset); + /** + * update schema info(encoding & compressionType) + * + * @param measurementId unsupported now + * @param encoding + * @param compressionType + * @param props unsupported now + */ + void updateSchemaInfo( + String measurementId, + TSEncoding encoding, + CompressionType compressionType, + Map props); + boolean isPreDeleted(); void setPreDeleted(boolean preDeleted); diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java index 9b223ba2fc8fe..eb0ead6ae6ecb 100644 --- a/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java +++ b/server/src/main/java/org/apache/iotdb/db/metadata/mnode/MeasurementMNode.java @@ -22,12 +22,17 @@ import org.apache.iotdb.db.metadata.mnode.container.IMNodeContainer; import org.apache.iotdb.db.metadata.mnode.container.MNodeContainers; import org.apache.iotdb.db.metadata.mnode.visitor.MNodeVisitor; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema; +import org.apache.iotdb.tsfile.write.schema.MeasurementSchema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Map; + public class MeasurementMNode extends MNode implements IMeasurementMNode { private static final Logger logger = LoggerFactory.getLogger(MeasurementMNode.class); @@ -124,6 +129,28 @@ public void setPreDeleted(boolean preDeleted) { this.preDeleted = preDeleted; } + @Override + public void updateSchemaInfo( + String measurementId, + TSEncoding encoding, + CompressionType compressionType, + Map props) { + + if (encoding == null) { + encoding = this.schema.getEncodingType(); + } + if (compressionType == null) { + compressionType = this.schema.getCompressor(); + } + this.schema = + new MeasurementSchema( + this.schema.getMeasurementId(), + this.schema.getType(), + encoding, + compressionType, + this.schema.getProps()); + } + @Override public R accept(MNodeVisitor visitor, C context) { return visitor.visitMeasurementMNode(this, context); diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java index 3328fd0b443b3..b31a8243484f0 100644 --- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java +++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/ISchemaRegion.java @@ -391,4 +391,17 @@ long countPathsUsingTemplate(int templateId, PathPatternTree patternTree) throws MetadataException; // endregion + + /** + * alter schema encoding & compression type + * + * @param curEncoding + * @param curCompressionType + * @throws MetadataException + * @throws IOException + * @return Pair old type + */ + Pair alterTimeseriesEncodingCompressionType( + PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType) + throws MetadataException, IOException; } diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java index 0cf41e3fd8fb8..af829c79cffb9 100644 --- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java +++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionMemoryImpl.java @@ -71,6 +71,8 @@ import org.apache.iotdb.db.metadata.utils.MetaUtils; import org.apache.iotdb.db.mpp.common.schematree.DeviceSchemaInfo; import org.apache.iotdb.db.mpp.common.schematree.MeasurementSchemaInfo; +import org.apache.iotdb.db.qp.logical.sys.AlterTimeSeriesOperator; +import org.apache.iotdb.db.qp.physical.sys.AlterTimeSeriesPlan; import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan; import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan; import org.apache.iotdb.db.query.context.QueryContext; @@ -737,6 +739,49 @@ public Map checkMeasurementExistence( return mtree.checkMeasurementExistence(devicePath, measurementList, aliasList); } + @Override + public Pair alterTimeseriesEncodingCompressionType( + PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType) + throws MetadataException, IOException { + final String logKey = fullPath.getFullPath(); + // find mnode + IMeasurementMNode measurementMNode = mtree.getMeasurementMNode(fullPath); + if (measurementMNode == null) { + throw new PathNotExistException("path not exist", true); + } + IMeasurementSchema schema = measurementMNode.getSchema(); + if (schema == null) { + throw new MetadataException("schema(" + fullPath + ") is null", false); + } + logger.info( + "[alter timeseries] {} ->old encoding:{}, compressionType:{}. cur ecoding:{}, compressionType:{}", + logKey, + schema.getEncodingType(), + schema.getCompressor(), + curEncoding, + curCompressionType); + // trigger TODO + // alter type + measurementMNode.updateSchemaInfo(null, curEncoding, curCompressionType, null); + // cache + mNodeCache.invalidate(fullPath); + // mlog + if (!isRecovering) { + writeToMLog( + new AlterTimeSeriesPlan( + fullPath, + AlterTimeSeriesOperator.AlterType.SET_TYPE, + null, + null, + null, + null, + measurementMNode.getSchema().getEncodingType(), + measurementMNode.getSchema().getCompressor())); + forceMlog(); + } + return new Pair<>(schema.getEncodingType(), schema.getCompressor()); + } + /** * Delete all timeseries matching the given path pattern. If using prefix match, the path pattern * is used to match prefix path. All timeseries start with the matched prefix path will be diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java index 1146b04f07397..0c1d1bd7bf025 100644 --- a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java +++ b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaRegionSchemaFileImpl.java @@ -1837,4 +1837,11 @@ public RecoverOperationResult visitAutoCreateDeviceMNode( } } } + + @Override + public Pair alterTimeseriesEncodingCompressionType( + PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType) + throws MetadataException, IOException { + throw new UnsupportedOperationException(); + } } diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/constant/StatementType.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/constant/StatementType.java index 7e88374da79e8..80f6eeb00adb3 100644 --- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/constant/StatementType.java +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/constant/StatementType.java @@ -150,4 +150,5 @@ public enum StatementType { SHOW_TRIGGERS, DEACTIVATE_TEMPLATE, + REWRITE_TIMESERIES } diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java index 23e897155a409..d1617b9fd26a4 100644 --- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/parser/ASTVisitor.java @@ -111,6 +111,7 @@ import org.apache.iotdb.db.mpp.plan.statement.metadata.GetRegionIdStatement; import org.apache.iotdb.db.mpp.plan.statement.metadata.GetSeriesSlotListStatement; import org.apache.iotdb.db.mpp.plan.statement.metadata.GetTimeSlotListStatement; +import org.apache.iotdb.db.mpp.plan.statement.metadata.RewriteTimeseriesStatement; import org.apache.iotdb.db.mpp.plan.statement.metadata.SetStorageGroupStatement; import org.apache.iotdb.db.mpp.plan.statement.metadata.SetTTLStatement; import org.apache.iotdb.db.mpp.plan.statement.metadata.ShowChildNodesStatement; @@ -481,6 +482,51 @@ public Statement visitAlterTimeseries(IoTDBSqlParser.AlterTimeseriesContext ctx) return alterTimeSeriesStatement; } + /** check and set datatype, encoding, compressor */ + private void checkPropsInAlterTimeSeries(AlterTimeSeriesStatement altrTimeSeriesStatement) { + Map props = altrTimeSeriesStatement.getAlterMap(); + + if (props != null + && props.containsKey(IoTDBConstant.COLUMN_TIMESERIES_ENCODING.toLowerCase())) { + String encodingString = + props.get(IoTDBConstant.COLUMN_TIMESERIES_ENCODING.toLowerCase()).toUpperCase(); + try { + altrTimeSeriesStatement.setEncoding(TSEncoding.valueOf(encodingString)); + props.remove(IoTDBConstant.COLUMN_TIMESERIES_ENCODING.toLowerCase()); + } catch (Exception e) { + throw new SemanticException(String.format("Unsupported encoding: %s", encodingString)); + } + } + + if (props != null + && props.containsKey(IoTDBConstant.COLUMN_TIMESERIES_COMPRESSION.toLowerCase())) { + String compressionString = + props.get(IoTDBConstant.COLUMN_TIMESERIES_COMPRESSION.toLowerCase()).toUpperCase(); + try { + altrTimeSeriesStatement.setCompressor(CompressionType.valueOf(compressionString)); + props.remove(IoTDBConstant.COLUMN_TIMESERIES_COMPRESSION.toLowerCase()); + } catch (Exception e) { + throw new SemanticException( + String.format("Unsupported compression: %s", compressionString)); + } + } else if (props != null + && props.containsKey(IoTDBConstant.COLUMN_TIMESERIES_COMPRESSOR.toLowerCase())) { + String compressorString = + props.get(IoTDBConstant.COLUMN_TIMESERIES_COMPRESSOR.toLowerCase()).toUpperCase(); + try { + altrTimeSeriesStatement.setCompressor(CompressionType.valueOf(compressorString)); + props.remove(IoTDBConstant.COLUMN_TIMESERIES_COMPRESSOR.toLowerCase()); + } catch (Exception e) { + throw new SemanticException(String.format("Unsupported compression: %s", compressorString)); + } + } + + if (altrTimeSeriesStatement.getCompressor() == null + && altrTimeSeriesStatement.getEncoding() == null) { + throw new SemanticException(String.format("encoding & compressor is null")); + } + } + private void parseAlterClause( IoTDBSqlParser.AlterClauseContext ctx, AlterTimeSeriesStatement alterTimeSeriesStatement) { Map alterMap = new HashMap<>(); @@ -506,6 +552,10 @@ private void parseAlterClause( // add attribute alterTimeSeriesStatement.setAlterType(AlterTimeSeriesStatement.AlterType.ADD_ATTRIBUTES); setMap(ctx, alterMap); + } else if (ctx.SETTYPE() != null) { + // alter encoding & compression + alterTimeSeriesStatement.setAlterType(AlterTimeSeriesStatement.AlterType.SET_TYPE); + setMap(ctx, alterMap); } else { // upsert alterTimeSeriesStatement.setAlterType(AlterTimeSeriesStatement.AlterType.UPSERT); @@ -520,6 +570,10 @@ private void parseAlterClause( } } alterTimeSeriesStatement.setAlterMap(alterMap); + // check type + if (ctx.SETTYPE() != null) { + checkPropsInAlterTimeSeries(alterTimeSeriesStatement); + } } public void parseAliasClause( @@ -1938,6 +1992,14 @@ public Statement visitCreateStorageGroup(IoTDBSqlParser.CreateStorageGroupContex return setStorageGroupStatement; } + @Override + public Statement visitRewriteTimeseries(IoTDBSqlParser.RewriteTimeseriesContext ctx) { + RewriteTimeseriesStatement rewriteTimeseriesStatement = new RewriteTimeseriesStatement(); + PartialPath path = parsePrefixPath(ctx.prefixPath()); + rewriteTimeseriesStatement.setStorageGroupPath(path); + return rewriteTimeseriesStatement; + } + private void parseStorageGroupAttributesClause( SetStorageGroupStatement setStorageGroupStatement, IoTDBSqlParser.StorageGroupAttributesClauseContext ctx) { diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/AlterTimeSeriesStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/AlterTimeSeriesStatement.java index d6f4198cba4fc..0c501f21c7136 100644 --- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/AlterTimeSeriesStatement.java +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/AlterTimeSeriesStatement.java @@ -23,6 +23,8 @@ import org.apache.iotdb.db.mpp.plan.constant.StatementType; import org.apache.iotdb.db.mpp.plan.statement.Statement; import org.apache.iotdb.db.mpp.plan.statement.StatementVisitor; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import java.util.Collections; import java.util.List; @@ -52,6 +54,8 @@ public class AlterTimeSeriesStatement extends Statement { private Map tagsMap; private Map attributesMap; + private TSEncoding encoding; + private CompressionType compressor; public AlterTimeSeriesStatement() { super(); @@ -111,6 +115,22 @@ public void setAttributesMap(Map attributesMap) { this.attributesMap = attributesMap; } + public TSEncoding getEncoding() { + return encoding; + } + + public void setEncoding(TSEncoding encoding) { + this.encoding = encoding; + } + + public CompressionType getCompressor() { + return compressor; + } + + public void setCompressor(CompressionType compressor) { + this.compressor = compressor; + } + @Override public R accept(StatementVisitor visitor, C context) { return visitor.visitAlterTimeseries(this, context); @@ -122,6 +142,7 @@ public enum AlterType { DROP, ADD_TAGS, ADD_ATTRIBUTES, - UPSERT + UPSERT, + SET_TYPE } } diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/RewriteTimeseriesStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/RewriteTimeseriesStatement.java new file mode 100644 index 0000000000000..6833d19fd06ae --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/metadata/RewriteTimeseriesStatement.java @@ -0,0 +1,58 @@ +/* + * 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.db.mpp.plan.statement.metadata; + +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.mpp.plan.analyze.QueryType; +import org.apache.iotdb.db.mpp.plan.constant.StatementType; +import org.apache.iotdb.db.mpp.plan.statement.IConfigStatement; +import org.apache.iotdb.db.mpp.plan.statement.Statement; + +import java.util.Collections; +import java.util.List; + +public class RewriteTimeseriesStatement extends Statement implements IConfigStatement { + private PartialPath storageGroupPath; + + public RewriteTimeseriesStatement() { + super(); + statementType = StatementType.REWRITE_TIMESERIES; + } + + public PartialPath getStorageGroupPath() { + return storageGroupPath; + } + + public void setStorageGroupPath(PartialPath storageGroupPath) { + this.storageGroupPath = storageGroupPath; + } + + @Override + public QueryType getQueryType() { + return QueryType.WRITE; + } + + @Override + public List getPaths() { + return storageGroupPath != null + ? Collections.singletonList(storageGroupPath) + : Collections.emptyList(); + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java index 6c64d769b9c16..9c07b0eaee7a7 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java @@ -35,6 +35,8 @@ import org.apache.iotdb.commons.utils.AuthUtils; import org.apache.iotdb.db.auth.AuthorityChecker; import org.apache.iotdb.db.auth.AuthorizerManager; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.engine.StorageEngine; import org.apache.iotdb.db.engine.cache.BloomFilterCache; import org.apache.iotdb.db.engine.cache.ChunkCache; import org.apache.iotdb.db.engine.cache.TimeSeriesMetadataCache; @@ -59,6 +61,7 @@ import org.apache.iotdb.db.qp.physical.crud.UDTFPlan; import org.apache.iotdb.db.qp.physical.sys.AuthorPlan; import org.apache.iotdb.db.qp.physical.sys.CountPlan; +import org.apache.iotdb.db.qp.physical.sys.RewriteTimeseriesPlan; import org.apache.iotdb.db.qp.physical.sys.ShowChildNodesPlan; import org.apache.iotdb.db.qp.physical.sys.ShowChildPathsPlan; import org.apache.iotdb.db.qp.physical.sys.ShowDevicesPlan; @@ -84,7 +87,9 @@ import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException; import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.read.common.Field; import org.apache.iotdb.tsfile.read.common.RowRecord; import org.apache.iotdb.tsfile.read.query.dataset.EmptyDataSet; @@ -177,6 +182,27 @@ public boolean processNonQuery(PhysicalPlan plan) return true; } + private boolean rewriteTimeseries(RewriteTimeseriesPlan plan) throws QueryProcessException { + + if (plan == null || plan.getPath() == null) { + throw new QueryProcessException("RewriteTimeseriesPlan is null"); + } + PartialPath fullPath = plan.getPath(); + final String logKey = fullPath.getFullPath(); + AUDIT_LOGGER.info("[rewriteTimeseries] {} begin", logKey); + if (IoTDBDescriptor.getInstance().getConfig().isClusterMode()) { + throw new QueryProcessException("This command is not supported in cluster mode"); + } + // storage alter + try { + StorageEngine.getInstance().rewriteTimeseries(fullPath); + } catch (StorageEngineException | MetadataException e) { + throw new QueryProcessException(e); + } + AUDIT_LOGGER.info("[rewriteTimeseries] {} end", logKey); + return true; + } + public static void operateClearCache() { ChunkCache.getInstance().clear(); TimeSeriesMetadataCache.getInstance().clear(); @@ -660,6 +686,32 @@ private void addRowRecordForShowQuery( listDataSet.putRecord(rowRecord); } + /** + * alter timeseris encoding & compression type + * + * @param fullPath timeseries full path + * @param curEncoding + * @param curCompressionType + * @throws QueryProcessException + * @throws IOException + * @throws MetadataException + */ + protected void alterTimeSeriesType( + PartialPath fullPath, TSEncoding curEncoding, CompressionType curCompressionType) + throws QueryProcessException, IOException, MetadataException { + + AUDIT_LOGGER.info("[alter timeseries] {} {} {}", fullPath, curEncoding, curCompressionType); + if (IoTDBDescriptor.getInstance().getConfig().isClusterMode()) { + throw new QueryProcessException("This command is not supported in cluster mode"); + } + // storage alter + try { + StorageEngine.getInstance().alterTimeseries(fullPath, curEncoding, curCompressionType); + } catch (StorageEngineException e) { + throw new QueryProcessException(e); + } + } + // high Cognitive Complexity protected QueryDataSet processAuthorQuery(AuthorPlan plan) throws QueryProcessException { AuthorType authorType = plan.getAuthorType(); diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java index 32292b5b0d90c..875d88e198f8b 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java @@ -208,6 +208,8 @@ public enum OperatorType { ACTIVATE_TEMPLATE_IN_CLUSTER, PRE_DELETE_TIMESERIES_IN_CLUSTER, - ROLLBACK_PRE_DELETE_TIMESERIES + ROLLBACK_PRE_DELETE_TIMESERIES, + + REWRITE_TIMESERIES } } diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/AlterTimeSeriesOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/AlterTimeSeriesOperator.java index 8d58fe454f453..038296a8f74b8 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/AlterTimeSeriesOperator.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/AlterTimeSeriesOperator.java @@ -25,6 +25,8 @@ import org.apache.iotdb.db.qp.physical.PhysicalPlan; import org.apache.iotdb.db.qp.physical.sys.AlterTimeSeriesPlan; import org.apache.iotdb.db.qp.strategy.PhysicalGenerator; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import java.util.Map; @@ -45,6 +47,8 @@ public class AlterTimeSeriesOperator extends Operator { private String alias; private Map tagsMap; private Map attributesMap; + private TSEncoding encoding; + private CompressionType compressor; public AlterTimeSeriesOperator(int tokenIntType) { super(tokenIntType); @@ -99,10 +103,27 @@ public void setAlias(String alias) { this.alias = alias; } + public TSEncoding getEncoding() { + return encoding; + } + + public void setEncoding(TSEncoding encoding) { + this.encoding = encoding; + } + + public CompressionType getCompressor() { + return compressor; + } + + public void setCompressor(CompressionType compressor) { + this.compressor = compressor; + } + @Override public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator) throws QueryProcessException { - return new AlterTimeSeriesPlan(path, alterType, alterMap, alias, tagsMap, attributesMap); + return new AlterTimeSeriesPlan( + path, alterType, alterMap, alias, tagsMap, attributesMap, encoding, compressor); } public enum AlterType { @@ -111,6 +132,7 @@ public enum AlterType { DROP, ADD_TAGS, ADD_ATTRIBUTES, - UPSERT + UPSERT, + SET_TYPE } } diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/RewriteTimeseriesOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/RewriteTimeseriesOperator.java new file mode 100644 index 0000000000000..c187c3eda813c --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/RewriteTimeseriesOperator.java @@ -0,0 +1,48 @@ +/* + * 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.db.qp.logical.sys; + +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.qp.logical.Operator; +import org.apache.iotdb.db.qp.physical.PhysicalPlan; +import org.apache.iotdb.db.qp.physical.sys.RewriteTimeseriesPlan; +import org.apache.iotdb.db.qp.strategy.PhysicalGenerator; + +public class RewriteTimeseriesOperator extends Operator { + + private PartialPath path; + + public RewriteTimeseriesOperator(int tokenIntType) { + super(tokenIntType); + operatorType = OperatorType.REWRITE_TIMESERIES; + } + + public PartialPath getPath() { + return path; + } + + public void setPath(PartialPath path) { + this.path = path; + } + + @Override + public PhysicalPlan generatePhysicalPlan(PhysicalGenerator generator) { + return new RewriteTimeseriesPlan(path); + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java index df59500d860c8..b54b903b9596d 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java @@ -532,6 +532,7 @@ public enum PhysicalPlanType { STOP_PIPE_SERVER, DROP_TEMPLATE, ACTIVATE_TEMPLATE_IN_CLUSTER, + REWRITE_TIMESERIES, PRE_DELETE_TIMESERIES_IN_CLUSTER, ROLLBACK_PRE_DELETE_TIMESERIES } diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/AlterTimeSeriesPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/AlterTimeSeriesPlan.java index c55edf615a98d..eb1fb2effef03 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/AlterTimeSeriesPlan.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/AlterTimeSeriesPlan.java @@ -26,6 +26,8 @@ import org.apache.iotdb.db.qp.logical.sys.AlterTimeSeriesOperator; import org.apache.iotdb.db.qp.logical.sys.AlterTimeSeriesOperator.AlterType; import org.apache.iotdb.db.qp.physical.PhysicalPlan; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import org.slf4j.Logger; @@ -50,7 +52,7 @@ public class AlterTimeSeriesPlan extends PhysicalPlan { * used when the alterType is RENAME, SET, DROP, ADD_TAGS, ADD_ATTRIBUTES. when the alterType is * RENAME, alterMap has only one entry, key is the beforeName, value is the currentName. when the * alterType is DROP, only the keySet of alterMap is useful, it contains all the key names needed - * to be removed + * to be removed. */ private Map alterMap; @@ -59,6 +61,8 @@ public class AlterTimeSeriesPlan extends PhysicalPlan { private Map tagsMap; private Map attributesMap; + private TSEncoding encoding; + private CompressionType compressor; /** used only for deserialize */ public AlterTimeSeriesPlan() { @@ -71,7 +75,9 @@ public AlterTimeSeriesPlan( Map alterMap, String alias, Map tagsMap, - Map attributesMap) { + Map attributesMap, + TSEncoding encoding, + CompressionType compressor) { super(Operator.OperatorType.ALTER_TIMESERIES); this.path = path; this.alterType = alterType; @@ -79,6 +85,8 @@ public AlterTimeSeriesPlan( this.alias = alias; this.tagsMap = tagsMap; this.attributesMap = attributesMap; + this.encoding = encoding; + this.compressor = compressor; } public PartialPath getPath() { @@ -110,6 +118,74 @@ public List getPaths() { return Collections.singletonList(path); } + public TSEncoding getEncoding() { + return encoding; + } + + public void setEncoding(TSEncoding encoding) { + this.encoding = encoding; + } + + public CompressionType getCompressor() { + return compressor; + } + + public void setCompressor(CompressionType compressor) { + this.compressor = compressor; + } + + @Override + protected void serializeImpl(ByteBuffer buffer) { + buffer.put((byte) PhysicalPlanType.ALTER_TIMESERIES.ordinal()); + byte[] bytes = path.getFullPath().getBytes(); + buffer.putInt(bytes.length); + buffer.put(bytes); + + buffer.put((byte) alterType.ordinal()); + + // alias + if (alias != null) { + buffer.put((byte) 1); + ReadWriteIOUtils.write(alias, buffer); + } else { + buffer.put((byte) 0); + } + + // alterMap + if (alterMap != null && !alterMap.isEmpty()) { + buffer.put((byte) 1); + ReadWriteIOUtils.write(alterMap, buffer); + } else { + buffer.put((byte) 0); + } + + // tagsMap + if (tagsMap != null && !tagsMap.isEmpty()) { + buffer.put((byte) 1); + ReadWriteIOUtils.write(tagsMap, buffer); + } else { + buffer.put((byte) 0); + } + + // attributesMap + if (attributesMap != null && !attributesMap.isEmpty()) { + buffer.put((byte) 1); + ReadWriteIOUtils.write(attributesMap, buffer); + } else { + buffer.put((byte) 0); + } + if (encoding != null) { + buffer.put((byte) encoding.ordinal()); + } else { + buffer.put((byte) -1); + } + if (compressor != null) { + buffer.put((byte) compressor.ordinal()); + } else { + buffer.put((byte) -1); + } + } + @Override public void serialize(DataOutputStream stream) throws IOException { stream.writeByte((byte) PhysicalPlanType.ALTER_TIMESERIES.ordinal()); @@ -150,6 +226,16 @@ public void serialize(DataOutputStream stream) throws IOException { } else { stream.write(0); } + if (encoding != null) { + stream.writeByte((byte) encoding.ordinal()); + } else { + stream.write(-1); + } + if (compressor != null) { + stream.writeByte((byte) compressor.ordinal()); + } else { + stream.write(-1); + } } @Override @@ -184,6 +270,16 @@ public void deserialize(ByteBuffer buffer) { if (buffer.get() == 1) { attributesMap = ReadWriteIOUtils.readMap(buffer); } + + // encoding + byte eb = buffer.get(); + if (eb != -1) { + encoding = TSEncoding.values()[eb]; + } + byte cb = buffer.get(); + if (cb != -1) { + compressor = CompressionType.values()[cb]; + } } @Override @@ -202,11 +298,14 @@ public boolean equals(Object o) { && Objects.equals(alterMap, that.alterMap) && Objects.equals(alias, that.alias) && Objects.equals(tagsMap, that.tagsMap) - && Objects.equals(attributesMap, that.attributesMap); + && Objects.equals(attributesMap, that.attributesMap) + && encoding == that.encoding + && compressor == that.compressor; } @Override public int hashCode() { - return Objects.hash(path, alias, alterType, alterMap, attributesMap, tagsMap); + return Objects.hash( + path, alias, alterType, alterMap, attributesMap, tagsMap, encoding, compressor); } } diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/RewriteTimeseriesPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/RewriteTimeseriesPlan.java new file mode 100644 index 0000000000000..cd7eec923d160 --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/RewriteTimeseriesPlan.java @@ -0,0 +1,100 @@ +/* + * 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.db.qp.physical.sys; + +import org.apache.iotdb.commons.exception.IllegalPathException; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.qp.logical.Operator; +import org.apache.iotdb.db.qp.physical.PhysicalPlan; + +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +public class RewriteTimeseriesPlan extends PhysicalPlan { + + private PartialPath path; + + public RewriteTimeseriesPlan() { + super(Operator.OperatorType.REWRITE_TIMESERIES); + } + + public RewriteTimeseriesPlan(PartialPath path) { + super(Operator.OperatorType.REWRITE_TIMESERIES); + this.path = path; + } + + public PartialPath getPath() { + return path; + } + + public void setPath(PartialPath path) { + this.path = path; + } + + @Override + public List getPaths() { + return path != null ? Collections.singletonList(path) : Collections.emptyList(); + } + + @Override + public void serialize(DataOutputStream stream) throws IOException { + stream.write((byte) PhysicalPlanType.REWRITE_TIMESERIES.ordinal()); + putString(stream, path.getFullPath()); + stream.writeLong(index); + } + + @Override + public void serializeImpl(ByteBuffer buffer) { + buffer.put((byte) PhysicalPlanType.REWRITE_TIMESERIES.ordinal()); + putString(buffer, path.getFullPath()); + buffer.putLong(index); + } + + @Override + public void deserialize(ByteBuffer buffer) throws IllegalPathException { + path = new PartialPath(readString(buffer)); + this.index = buffer.getLong(); + } + + @Override + public String toString() { + return "RewriteTimeseries{" + path + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RewriteTimeseriesPlan that = (RewriteTimeseriesPlan) o; + return Objects.equals(path, that.path); + } + + @Override + public int hashCode() { + return Objects.hash(path); + } +} diff --git a/server/src/main/java/org/apache/iotdb/db/utils/CommonUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/CommonUtils.java index 0c49d58670d8d..5385a033e3f92 100644 --- a/server/src/main/java/org/apache/iotdb/db/utils/CommonUtils.java +++ b/server/src/main/java/org/apache/iotdb/db/utils/CommonUtils.java @@ -266,4 +266,12 @@ private static void err(Throwable e) { System.err.println("-- StackTrace --"); System.err.println(Throwables.getStackTraceAsString(e)); } + + public static boolean equal(String s, String t) { + if (s == t) { + return true; + } else { + return s != null && t != null ? s.equals(t) : false; + } + } } diff --git a/server/src/test/java/org/apache/iotdb/db/engine/alter/AlteringLogTest.java b/server/src/test/java/org/apache/iotdb/db/engine/alter/AlteringLogTest.java new file mode 100644 index 0000000000000..de3bbc158a50d --- /dev/null +++ b/server/src/test/java/org/apache/iotdb/db/engine/alter/AlteringLogTest.java @@ -0,0 +1,141 @@ +/* + * 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.db.engine.alter; + +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.engine.alter.log.AlteringLogAnalyzer; +import org.apache.iotdb.db.engine.alter.log.AlteringLogger; +import org.apache.iotdb.db.engine.compaction.log.TsFileIdentifier; +import org.apache.iotdb.db.engine.storagegroup.TsFileResource; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; +import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer; +import org.apache.iotdb.tsfile.utils.Pair; + +import org.apache.commons.io.FileUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +public class AlteringLogTest { + + String path = AlteringLogger.ALTERING_LOG_NAME; + + String f1 = + "data" + .concat(File.separator) + .concat("data") + .concat(File.separator) + .concat("sequence") + .concat(File.separator) + .concat("root.alt1") + .concat(File.separator) + .concat("0") + .concat(File.separator) + .concat("0") + .concat(File.separator) + .concat("1-1-0-0.tsfile"); + + String f2 = + "data" + .concat(File.separator) + .concat("data") + .concat(File.separator) + .concat("sequence") + .concat(File.separator) + .concat("root.alt1") + .concat(File.separator) + .concat("0") + .concat(File.separator) + .concat("0") + .concat(File.separator) + .concat("2-2-0-0.tsfile"); + + @Before + public void setUp() throws Exception { + try { + File f = FSFactoryProducer.getFSFactory().getFile(path); + if (f.exists() && !f.delete()) { + throw new RuntimeException("can not delete " + f.getAbsolutePath()); + } + f.createNewFile(); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + } + + @Test + public void alterLogTest() { + + File f = FSFactoryProducer.getFSFactory().getFile(path); + try (AlteringLogger logger = new AlteringLogger(f)) { + PartialPath pm1 = new PartialPath("root.alt1.d1", "m1"); + PartialPath pm2 = new PartialPath("root.alt1.d2", "m1"); + logger.addAlterParam(pm1, TSEncoding.TS_2DIFF, CompressionType.GZIP); + logger.addAlterParam(pm2, TSEncoding.GORILLA, CompressionType.SNAPPY); + logger.clearBegin(); + logger.doneFile(new TsFileResource(new File(f1))); + logger.doneFile(new TsFileResource(new File(f2))); + logger.close(); + + AlteringLogAnalyzer analyzer = new AlteringLogAnalyzer(f); + analyzer.analyzer(); + + List>> alterList = analyzer.getAlterList(); + Assert.assertNotNull(alterList); + Assert.assertEquals(alterList.size(), 2); + Pair> p1 = alterList.get(0); + Pair> p2 = alterList.get(1); + Assert.assertEquals(p1.left, pm1.getFullPath()); + Assert.assertEquals(p2.left, pm2.getFullPath()); + Assert.assertEquals(p1.right.left, TSEncoding.TS_2DIFF); + Assert.assertEquals(p1.right.right, CompressionType.GZIP); + Assert.assertEquals(p2.right.left, TSEncoding.GORILLA); + Assert.assertEquals(p2.right.right, CompressionType.SNAPPY); + Assert.assertTrue(analyzer.isClearBegin()); + Set doneFiles = analyzer.getDoneFiles(); + Assert.assertNotNull(alterList); + Assert.assertEquals(alterList.size(), 2); + Iterator it = doneFiles.iterator(); + TsFileIdentifier if1 = it.next(); + TsFileIdentifier if2 = it.next(); + Assert.assertEquals(TsFileIdentifier.getFileIdentifierFromFilePath(f1), if1); + Assert.assertEquals(TsFileIdentifier.getFileIdentifierFromFilePath(f2), if2); + + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + } + + @After + public void tearDown() { + try { + FileUtils.forceDelete(new File(path)); + } catch (IOException e) { + Assert.fail(e.getMessage()); + } + } +} diff --git a/server/src/test/java/org/apache/iotdb/db/engine/alter/RewriteTimeseriesTaskTest.java b/server/src/test/java/org/apache/iotdb/db/engine/alter/RewriteTimeseriesTaskTest.java new file mode 100644 index 0000000000000..236f210e964b6 --- /dev/null +++ b/server/src/test/java/org/apache/iotdb/db/engine/alter/RewriteTimeseriesTaskTest.java @@ -0,0 +1,435 @@ +/* + * 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.db.engine.alter; + +import org.apache.iotdb.commons.file.SystemFileFactory; +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.engine.alter.log.AlteringLogAnalyzer; +import org.apache.iotdb.db.engine.alter.log.AlteringLogger; +import org.apache.iotdb.db.engine.cache.AlteringRecordsCache; +import org.apache.iotdb.db.engine.compaction.CompactionTaskManager; +import org.apache.iotdb.db.engine.compaction.task.CompactionTaskSummary; +import org.apache.iotdb.db.engine.storagegroup.TsFileManager; +import org.apache.iotdb.db.engine.storagegroup.TsFileResource; +import org.apache.iotdb.db.rescon.TsFileResourceManager; +import org.apache.iotdb.tsfile.exception.write.WriteProcessException; +import org.apache.iotdb.tsfile.file.header.ChunkHeader; +import org.apache.iotdb.tsfile.file.metadata.AlignedChunkMetadata; +import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata; +import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; +import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer; +import org.apache.iotdb.tsfile.read.TsFileDeviceIterator; +import org.apache.iotdb.tsfile.read.TsFileSequenceReader; +import org.apache.iotdb.tsfile.read.common.Chunk; +import org.apache.iotdb.tsfile.read.common.Path; +import org.apache.iotdb.tsfile.utils.FilePathUtils; +import org.apache.iotdb.tsfile.utils.MeasurementGroup; +import org.apache.iotdb.tsfile.utils.Pair; +import org.apache.iotdb.tsfile.write.TsFileWriter; +import org.apache.iotdb.tsfile.write.record.Tablet; +import org.apache.iotdb.tsfile.write.schema.MeasurementSchema; +import org.apache.iotdb.tsfile.write.schema.Schema; + +import org.apache.commons.io.FileUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +public class RewriteTimeseriesTaskTest { + + String dataRegionId = "0"; + String pathBase = + "data" + .concat(File.separator) + .concat("data") + .concat(File.separator) + .concat("sequence") + .concat(File.separator) + .concat("root.alttask1") + .concat(File.separator) + .concat(dataRegionId) + .concat(File.separator) + .concat("0") + .concat(File.separator); + String storageGroupName = "root.alttask1"; + String device = "root.alttask1.device_0"; + String alignedDevice = "root.alttask1.device_1"; + String deviceNotRewrite = "root.alttask1.device_3"; + String alignedDeviceNotRewrite = "root.alttask1.device_4"; + String sensorPrefix = "sensor_"; + // the number of rows to include in the tablet + int rowNum = 1000000; + // the number of values to include in the tablet + int sensorNum = 10; + + TSEncoding defaultEncoding = TSEncoding.TS_2DIFF; + CompressionType defaultCompressionType = CompressionType.GZIP; + TsFileManager tsFileManager = null; + int fileCount = 6; + + @Before + public void setUp() throws Exception { + try { + File storageGroupSysDir = + SystemFileFactory.INSTANCE.getFile( + FilePathUtils.regularizePath(IoTDBDescriptor.getInstance().getConfig().getSystemDir()) + + "storage_groups" + + File.separator + + File.separator + + storageGroupName, + dataRegionId); + if (!storageGroupSysDir.exists()) { + storageGroupSysDir.mkdirs(); + } + File logFile = + SystemFileFactory.INSTANCE.getFile( + storageGroupSysDir.getPath(), AlteringLogger.ALTERING_LOG_NAME); + if (!logFile.exists()) { + logFile.createNewFile(); + } + AlteringLogger alteringLogger = new AlteringLogger(logFile); + PartialPath fullPath = new PartialPath(device, sensorPrefix + "1"); + alteringLogger.addAlterParam(fullPath, TSEncoding.GORILLA, CompressionType.GZIP); + PartialPath fullPathAlign = new PartialPath(alignedDevice, sensorPrefix + "1"); + alteringLogger.addAlterParam(fullPathAlign, TSEncoding.RLE, CompressionType.GZIP); + alteringLogger.close(); + AlteringRecordsCache.getInstance() + .putRecord( + storageGroupName, fullPath.getFullPath(), TSEncoding.GORILLA, CompressionType.GZIP); + AlteringRecordsCache.getInstance() + .putRecord( + storageGroupName, fullPathAlign.getFullPath(), TSEncoding.RLE, CompressionType.GZIP); + tsFileManager = + new TsFileManager(storageGroupName, dataRegionId, storageGroupSysDir.getPath()); + for (int fnum = 1; fnum <= 3; fnum++) { + File f = FSFactoryProducer.getFSFactory().getFile(pathBase.concat(fnum + "-0-0-0.tsfile")); + if (f.exists() && !f.delete()) { + throw new RuntimeException("can not delete " + f.getAbsolutePath()); + } + + long beginTime = fnum * rowNum; + long beginValue = fnum * 1000000L; + writeFile(f, beginTime, beginValue, device, alignedDevice); + } + int fnum = 4; + File f = FSFactoryProducer.getFSFactory().getFile(pathBase.concat(fnum + "-0-0-0.tsfile")); + if (f.exists() && !f.delete()) { + throw new RuntimeException("can not delete " + f.getAbsolutePath()); + } + long beginTime = fnum * rowNum; + long beginValue = fnum * 1000000L; + writeFile(f, beginTime, beginValue, deviceNotRewrite, alignedDeviceNotRewrite); + + fnum = 5; + f = FSFactoryProducer.getFSFactory().getFile(pathBase.concat(fnum + "-0-0-0.tsfile")); + if (f.exists() && !f.delete()) { + throw new RuntimeException("can not delete " + f.getAbsolutePath()); + } + beginTime = fnum * rowNum; + beginValue = fnum * 1000000L; + writeFile(f, beginTime, beginValue, device, alignedDeviceNotRewrite); + + fnum = 6; + f = FSFactoryProducer.getFSFactory().getFile(pathBase.concat(fnum + "-0-0-0.tsfile")); + if (f.exists() && !f.delete()) { + throw new RuntimeException("can not delete " + f.getAbsolutePath()); + } + beginTime = fnum * rowNum; + beginValue = fnum * 1000000L; + writeFile(f, beginTime, beginValue, deviceNotRewrite, alignedDevice); + + } catch (Exception e) { + throw new Exception("meet error in TsFileWrite with tablet", e); + } + } + + @Test + public void taskTest() { + try { + List timepartitions = new ArrayList<>(2); + timepartitions.add(0L); + File logFile = + SystemFileFactory.INSTANCE.getFile( + tsFileManager.getStorageGroupDir(), AlteringLogger.ALTERING_LOG_NAME); + Assert.assertTrue(logFile.exists()); + AlteringLogAnalyzer analyzer = new AlteringLogAnalyzer(logFile); + analyzer.analyzer(); + // clear begin + if (!analyzer.isClearBegin()) { + AlteringLogger.clearBegin(logFile); + } + CompactionTaskManager.getInstance().start(); + RewriteTimeseriesTask task = + new RewriteTimeseriesTask( + storageGroupName, + dataRegionId, + timepartitions, + tsFileManager, + CompactionTaskManager.currentTaskNum, + tsFileManager.getNextCompactionTaskId(), + analyzer.isClearBegin(), + analyzer.getDoneFiles(), + logFile); + boolean b = CompactionTaskManager.getInstance().addTaskToWaitingQueue(task); + Assert.assertTrue(b); + Future future = + CompactionTaskManager.getInstance().getCompactionTaskFutureCheckStatusMayBlock(task); + if (future == null) { + Assert.assertTrue(task.isTaskFinished()); + Assert.assertTrue(task.isSuccess()); + } else { + CompactionTaskSummary summary = future.get(600000, TimeUnit.MILLISECONDS); + Assert.assertNotNull(summary); + Assert.assertTrue(summary.isFinished()); + Assert.assertTrue(summary.isSuccess()); + } + for (int fnum = 1; fnum <= fileCount; fnum++) { + File f = FSFactoryProducer.getFSFactory().getFile(pathBase.concat(fnum + "-0-0-0.tsfile")); + File falter = + FSFactoryProducer.getFSFactory().getFile(pathBase.concat(fnum + "-0-0-0.alter")); + File fold = + FSFactoryProducer.getFSFactory().getFile(pathBase.concat(fnum + "-0-0-0.alter.old")); + Assert.assertTrue(f.exists()); + Assert.assertTrue(new File(f.getPath() + ".resource").exists()); + Assert.assertFalse(falter.exists()); + Assert.assertFalse(fold.exists()); + Set devicesCache = + AlteringRecordsCache.getInstance().getDevicesCache(storageGroupName); + Map>> alters = new HashMap<>(); + devicesCache.forEach( + device -> { + alters.put(device, AlteringRecordsCache.getInstance().getDeviceRecords(device)); + }); + readCheck(alters, f); + } + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + } + + private void readCheck( + Map>> alters, File targetTsFile) { + try (TsFileSequenceReader reader = + new TsFileSequenceReader(targetTsFile.getAbsolutePath(), true)) { + TsFileDeviceIterator deviceIterator = reader.getAllDevicesIteratorWithIsAligned(); + + while (deviceIterator.hasNext()) { + Pair deviceInfo = deviceIterator.next(); + String device = deviceInfo.left; + boolean aligned = deviceInfo.right; + if (aligned) { + List alignedChunkMetadatas = reader.getAlignedChunkMetadata(device); + for (AlignedChunkMetadata alignedChunkMetadata : alignedChunkMetadatas) { + List valueChunkMetadataList = + alignedChunkMetadata.getValueChunkMetadataList(); + for (IChunkMetadata chunkMetadata : valueChunkMetadataList) { + Chunk chunk = reader.readMemChunk((ChunkMetadata) chunkMetadata); + ChunkHeader header = chunk.getHeader(); + Map> deviceMap = alters.get(device); + if (deviceMap != null && deviceMap.containsKey(header.getMeasurementID())) { + Pair tsEncodingCompressionTypePair = + deviceMap.get(header.getMeasurementID()); + Assert.assertEquals(header.getEncodingType(), tsEncodingCompressionTypePair.left); + Assert.assertEquals( + header.getCompressionType(), tsEncodingCompressionTypePair.right); + } else { + Assert.assertEquals(header.getEncodingType(), defaultEncoding); + Assert.assertEquals(header.getCompressionType(), defaultCompressionType); + } + } + } + } else { + Map> measurementMap = + reader.readChunkMetadataInDevice(device); + for (Map.Entry> next : measurementMap.entrySet()) { + String measurementId = next.getKey(); + List chunkMetadatas = next.getValue(); + for (ChunkMetadata chunkMetadata : chunkMetadatas) { + Chunk currentChunk = reader.readMemChunk(chunkMetadata); + ChunkHeader header = currentChunk.getHeader(); + Map> deviceMap = alters.get(device); + if (deviceMap != null && deviceMap.containsKey(measurementId)) { + Pair tsEncodingCompressionTypePair = + deviceMap.get(header.getMeasurementID()); + Assert.assertEquals(header.getEncodingType(), tsEncodingCompressionTypePair.left); + Assert.assertEquals( + header.getCompressionType(), tsEncodingCompressionTypePair.right); + } else { + Assert.assertEquals(header.getEncodingType(), defaultEncoding); + Assert.assertEquals(header.getCompressionType(), defaultCompressionType); + } + } + } + } + } + } catch (IOException e) { + Assert.fail(e.getMessage()); + } + } + + private void writeFile( + File f, long beginTime, long beginValue, String device, String alignedDevice) + throws IOException, WriteProcessException { + Schema schema = new Schema(); + List measurementSchemas = new ArrayList<>(); + // add measurements into file schema (all with INT64 data type) + for (int i = 0; i < sensorNum; i++) { + MeasurementSchema measurementSchema = + new MeasurementSchema( + sensorPrefix + (i + 1), TSDataType.INT64, defaultEncoding, defaultCompressionType); + measurementSchemas.add(measurementSchema); + schema.registerTimeseries( + new Path(device), + new MeasurementSchema( + sensorPrefix + (i + 1), TSDataType.INT64, defaultEncoding, defaultCompressionType)); + } + // add aligned measurements into file schema + List schemas = new ArrayList<>(); + List alignedMeasurementSchemas = new ArrayList<>(); + for (int i = 0; i < sensorNum; i++) { + MeasurementSchema schema1 = + new MeasurementSchema( + sensorPrefix + (i + 1), TSDataType.INT64, defaultEncoding, defaultCompressionType); + schemas.add(schema1); + alignedMeasurementSchemas.add(schema1); + } + MeasurementGroup group = new MeasurementGroup(true, schemas); + schema.registerMeasurementGroup(new Path(alignedDevice), group); + + try (TsFileWriter tsFileWriter = new TsFileWriter(f, schema)) { + + // add measurements into TSFileWriter + // construct the tablet + Tablet tablet = new Tablet(device, measurementSchemas); + long[] timestamps = tablet.timestamps; + Object[] values = tablet.values; + long timestamp = beginTime; + long value = beginValue; + for (int r = 0; r < rowNum; r++, value++) { + int row = tablet.rowSize++; + timestamps[row] = timestamp++; + for (int i = 0; i < sensorNum; i++) { + long[] sensor = (long[]) values[i]; + sensor[row] = value; + } + // write Tablet to TsFile + if (tablet.rowSize == tablet.getMaxRowNumber()) { + tsFileWriter.write(tablet); + tablet.reset(); + } + } + // write Tablet to TsFile + if (tablet.rowSize != 0) { + tsFileWriter.write(tablet); + tablet.reset(); + } + + // add aligned measurements into TSFileWriter + // construct the tablet + tablet = new Tablet(alignedDevice, alignedMeasurementSchemas); + timestamps = tablet.timestamps; + values = tablet.values; + timestamp = beginTime; + value = beginValue; + for (int r = 0; r < rowNum; r++, value++) { + int row = tablet.rowSize++; + timestamps[row] = timestamp++; + for (int i = 0; i < sensorNum; i++) { + long[] sensor = (long[]) values[i]; + sensor[row] = value; + } + // write Tablet to TsFile + if (tablet.rowSize == tablet.getMaxRowNumber()) { + tsFileWriter.writeAligned(tablet); + tablet.reset(); + } + } + // write Tablet to TsFile + if (tablet.rowSize != 0) { + tsFileWriter.writeAligned(tablet); + tablet.reset(); + } + } + Map deviceToIndex = new HashMap<>(); + deviceToIndex.put(device, 0); + deviceToIndex.put(alignedDevice, 1); + long[] startTimes = {beginTime, beginTime}; + long[] endTimes = {beginTime + rowNum - 1, beginTime + rowNum - 1}; + TsFileResource tsFileResource = new TsFileResource(f, deviceToIndex, startTimes, endTimes); + tsFileResource.serialize(); + tsFileResource.close(); + TsFileResourceManager.getInstance().registerSealedTsFileResource(tsFileResource); + tsFileManager.keepOrderInsert(tsFileResource, true); + } + + @After + public void tearDown() { + try { + File storageGroupDirFile = + SystemFileFactory.INSTANCE.getFile( + FilePathUtils.regularizePath(IoTDBDescriptor.getInstance().getConfig().getSystemDir()) + + "storage_groups" + + File.separator + + File.separator + + storageGroupName, + dataRegionId); + if (storageGroupDirFile.exists()) { + FileUtils.forceDeleteOnExit(storageGroupDirFile); + } + if (tsFileManager != null) { + List tsFileList = tsFileManager.getTsFileList(true); + tsFileList.forEach( + tsFileResource -> { + try { + tsFileResource.delete(); + } catch (IOException e) { + e.printStackTrace(); + } + }); + } else { + for (int fnum = 1; fnum <= fileCount; fnum++) { + File f = + FSFactoryProducer.getFSFactory().getFile(pathBase.concat(fnum + "-0-0-0.tsfile")); + if (f.exists()) { + FileUtils.forceDeleteOnExit(f); + } + } + } + FileUtils.forceDeleteOnExit(new File(pathBase)); + AlteringRecordsCache.getInstance().clear(storageGroupName); + } catch (IOException e) { + Assert.fail(e.getMessage()); + } + } +} diff --git a/server/src/test/java/org/apache/iotdb/db/engine/alter/TsFileRewritePerformerTest.java b/server/src/test/java/org/apache/iotdb/db/engine/alter/TsFileRewritePerformerTest.java new file mode 100644 index 0000000000000..66f19126aeaae --- /dev/null +++ b/server/src/test/java/org/apache/iotdb/db/engine/alter/TsFileRewritePerformerTest.java @@ -0,0 +1,366 @@ +/* + * 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.db.engine.alter; + +import org.apache.iotdb.commons.path.PartialPath; +import org.apache.iotdb.db.engine.cache.AlteringRecordsCache; +import org.apache.iotdb.db.engine.compaction.task.CompactionTaskSummary; +import org.apache.iotdb.db.engine.storagegroup.TsFileNameGenerator; +import org.apache.iotdb.db.engine.storagegroup.TsFileResource; +import org.apache.iotdb.tsfile.file.header.ChunkHeader; +import org.apache.iotdb.tsfile.file.metadata.AlignedChunkMetadata; +import org.apache.iotdb.tsfile.file.metadata.ChunkMetadata; +import org.apache.iotdb.tsfile.file.metadata.IChunkMetadata; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; +import org.apache.iotdb.tsfile.fileSystem.FSFactoryProducer; +import org.apache.iotdb.tsfile.read.TsFileDeviceIterator; +import org.apache.iotdb.tsfile.read.TsFileSequenceReader; +import org.apache.iotdb.tsfile.read.common.Chunk; +import org.apache.iotdb.tsfile.read.common.Path; +import org.apache.iotdb.tsfile.utils.MeasurementGroup; +import org.apache.iotdb.tsfile.utils.Pair; +import org.apache.iotdb.tsfile.write.TsFileWriter; +import org.apache.iotdb.tsfile.write.record.Tablet; +import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema; +import org.apache.iotdb.tsfile.write.schema.MeasurementSchema; +import org.apache.iotdb.tsfile.write.schema.Schema; + +import org.apache.commons.io.FileUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class TsFileRewritePerformerTest { + + String path = + "data" + .concat(File.separator) + .concat("data") + .concat(File.separator) + .concat("sequence") + .concat(File.separator) + .concat("root.alt1") + .concat(File.separator) + .concat("0") + .concat(File.separator) + .concat("0") + .concat(File.separator) + .concat("1-0-0-0.tsfile"); + String targetPath; + String storageGroupName = "root.alt1"; + String device = "root.alt1.device_0"; + String alignedDevice = "root.alt1.device_1"; + String sensorPrefix = "sensor_"; + // the number of rows to include in the tablet + int rowNum = 1000000; + // the number of values to include in the tablet + int sensorNum = 10; + + TSEncoding defaultEncoding = TSEncoding.TS_2DIFF; + CompressionType defaultCompressionType = CompressionType.GZIP; + + @Before + public void setUp() throws Exception { + try { + File f = FSFactoryProducer.getFSFactory().getFile(path); + if (f.exists() && !f.delete()) { + throw new RuntimeException("can not delete " + f.getAbsolutePath()); + } + + Schema schema = new Schema(); + + List measurementSchemas = new ArrayList<>(); + // add measurements into file schema (all with INT64 data type) + for (int i = 0; i < sensorNum; i++) { + MeasurementSchema measurementSchema = + new MeasurementSchema( + sensorPrefix + (i + 1), TSDataType.INT64, defaultEncoding, defaultCompressionType); + measurementSchemas.add(measurementSchema); + schema.registerTimeseries( + new Path(device), + new MeasurementSchema( + sensorPrefix + (i + 1), TSDataType.INT64, defaultEncoding, defaultCompressionType)); + } + // add aligned measurements into file schema + List schemas = new ArrayList<>(); + List alignedMeasurementSchemas = new ArrayList<>(); + for (int i = 0; i < sensorNum; i++) { + MeasurementSchema schema1 = + new MeasurementSchema( + sensorPrefix + (i + 1), TSDataType.INT64, defaultEncoding, defaultCompressionType); + schemas.add(schema1); + alignedMeasurementSchemas.add(schema1); + } + MeasurementGroup group = new MeasurementGroup(true, schemas); + schema.registerMeasurementGroup(new Path(alignedDevice), group); + + try (TsFileWriter tsFileWriter = new TsFileWriter(f, schema)) { + + // add measurements into TSFileWriter + // construct the tablet + Tablet tablet = new Tablet(device, measurementSchemas); + long[] timestamps = tablet.timestamps; + Object[] values = tablet.values; + long timestamp = 1; + long value = 1000000L; + for (int r = 0; r < rowNum; r++, value++) { + int row = tablet.rowSize++; + timestamps[row] = timestamp++; + for (int i = 0; i < sensorNum; i++) { + long[] sensor = (long[]) values[i]; + sensor[row] = value; + } + // write Tablet to TsFile + if (tablet.rowSize == tablet.getMaxRowNumber()) { + tsFileWriter.write(tablet); + tablet.reset(); + } + } + // write Tablet to TsFile + if (tablet.rowSize != 0) { + tsFileWriter.write(tablet); + tablet.reset(); + } + + // add aligned measurements into TSFileWriter + // construct the tablet + tablet = new Tablet(alignedDevice, alignedMeasurementSchemas); + timestamps = tablet.timestamps; + values = tablet.values; + timestamp = 1; + value = 1000000L; + for (int r = 0; r < rowNum; r++, value++) { + int row = tablet.rowSize++; + timestamps[row] = timestamp++; + for (int i = 0; i < sensorNum; i++) { + long[] sensor = (long[]) values[i]; + sensor[row] = value; + } + // write Tablet to TsFile + if (tablet.rowSize == tablet.getMaxRowNumber()) { + tsFileWriter.writeAligned(tablet); + tablet.reset(); + } + } + // write Tablet to TsFile + if (tablet.rowSize != 0) { + tsFileWriter.writeAligned(tablet); + tablet.reset(); + } + } + } catch (Exception e) { + throw new Exception("meet error in TsFileWrite with tablet", e); + } + } + + @Test + public void tsFileRewriteExcutorAlignedTest() { + + Map>> alters = new HashMap<>(); + + Map> alginMap = new HashMap<>(); + alters.put(alignedDevice, alginMap); + alginMap.put(sensorPrefix + "1", new Pair<>(TSEncoding.PLAIN, CompressionType.SNAPPY)); + + Map> notalginMap = new HashMap<>(); + alters.put(device, notalginMap); + notalginMap.put(sensorPrefix + "1", new Pair<>(TSEncoding.GORILLA, CompressionType.GZIP)); + + File targetTsFile = null; + TsFileRewritePerformer excutor = null; + try { + AlteringRecordsCache alteringRecordsCache = AlteringRecordsCache.getInstance(); + alters.forEach( + (d, m) -> { + m.forEach( + (mid, ap) -> { + PartialPath fullPath = null; + try { + fullPath = new PartialPath(d, mid); + alteringRecordsCache.putRecord( + storageGroupName, fullPath.getFullPath(), ap.left, ap.right); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + }); + }); + alteringRecordsCache.startAlter(); + File f = FSFactoryProducer.getFSFactory().getFile(path); + if (!f.exists()) { + throw new RuntimeException("tsfile not exists " + f.getAbsolutePath()); + } + TsFileResource tsFileResource = new TsFileResource(f); + tsFileResource.close(); + TsFileResource targetTsFileResource = + TsFileNameGenerator.generateNewAlterTsFileResource(tsFileResource); + excutor = new TsFileRewritePerformer(); + excutor.setSourceFiles(Collections.singletonList(tsFileResource)); + excutor.setTargetFiles(Collections.singletonList(targetTsFileResource)); + excutor.setSummary(new CompactionTaskSummary()); + excutor.perform(); + targetTsFile = targetTsFileResource.getTsFile(); + alteringRecordsCache.clear(storageGroupName); + } catch (Exception e) { + Assert.fail(e.getMessage()); + } + if (targetTsFile == null || !targetTsFile.exists()) { + throw new RuntimeException("target not exists " + targetTsFile.getAbsolutePath()); + } + targetPath = targetTsFile.getAbsolutePath(); + Assert.assertFalse(AlteringRecordsCache.getInstance().isAltering()); + readCheck(alters, targetTsFile); + } + + private void readCheck( + Map>> alters, File targetTsFile) { + try (TsFileSequenceReader reader = + new TsFileSequenceReader(targetTsFile.getAbsolutePath(), true)) { + TsFileDeviceIterator deviceIterator = reader.getAllDevicesIteratorWithIsAligned(); + + while (deviceIterator.hasNext()) { + Pair deviceInfo = deviceIterator.next(); + String device = deviceInfo.left; + boolean aligned = deviceInfo.right; + if (aligned) { + List alignedChunkMetadatas = reader.getAlignedChunkMetadata(device); + for (AlignedChunkMetadata alignedChunkMetadata : alignedChunkMetadatas) { + List valueChunkMetadataList = + alignedChunkMetadata.getValueChunkMetadataList(); + for (IChunkMetadata chunkMetadata : valueChunkMetadataList) { + Chunk chunk = reader.readMemChunk((ChunkMetadata) chunkMetadata); + ChunkHeader header = chunk.getHeader(); + Map> deviceMap = alters.get(device); + if (deviceMap != null && deviceMap.containsKey(header.getMeasurementID())) { + Pair tsEncodingCompressionTypePair = + deviceMap.get(header.getMeasurementID()); + Assert.assertEquals(header.getEncodingType(), tsEncodingCompressionTypePair.left); + Assert.assertEquals( + header.getCompressionType(), tsEncodingCompressionTypePair.right); + } else { + Assert.assertEquals(header.getEncodingType(), defaultEncoding); + Assert.assertEquals(header.getCompressionType(), defaultCompressionType); + } + } + } + } else { + Map> measurementMap = + reader.readChunkMetadataInDevice(device); + for (Map.Entry> next : measurementMap.entrySet()) { + String measurementId = next.getKey(); + List chunkMetadatas = next.getValue(); + for (ChunkMetadata chunkMetadata : chunkMetadatas) { + Chunk currentChunk = reader.readMemChunk(chunkMetadata); + ChunkHeader header = currentChunk.getHeader(); + Map> deviceMap = alters.get(device); + if (deviceMap != null && deviceMap.containsKey(measurementId)) { + Pair tsEncodingCompressionTypePair = + deviceMap.get(header.getMeasurementID()); + Assert.assertEquals(header.getEncodingType(), tsEncodingCompressionTypePair.left); + Assert.assertEquals( + header.getCompressionType(), tsEncodingCompressionTypePair.right); + } else { + Assert.assertEquals(header.getEncodingType(), defaultEncoding); + Assert.assertEquals(header.getCompressionType(), defaultCompressionType); + } + } + } + } + } + } catch (IOException e) { + Assert.fail(e.getMessage()); + } finally { + if (targetPath != null) { + try { + File file = new File(targetPath); + if (file.exists()) { + FileUtils.forceDelete(file); + } + } catch (IOException e) { + Assert.fail(e.getMessage()); + } + } + } + } + + @After + public void tearDown() { + try { + FileUtils.forceDelete(new File(path)); + if (targetPath != null) { + try { + File file = new File(targetPath); + if (file.exists()) { + FileUtils.forceDelete(file); + } + } catch (IOException e) { + Assert.fail(e.getMessage()); + } + } + } catch (IOException e) { + Assert.fail(e.getMessage()); + } + } + + protected List collectSchemaList( + List alignedChunkMetadatas, TsFileSequenceReader reader) + throws IOException { + List schemaList = new ArrayList<>(); + Set measurementSet = new HashSet<>(); + for (AlignedChunkMetadata alignedChunkMetadata : alignedChunkMetadatas) { + List valueChunkMetadataList = + alignedChunkMetadata.getValueChunkMetadataList(); + for (IChunkMetadata chunkMetadata : valueChunkMetadataList) { + if (chunkMetadata == null) { + continue; + } + String measurementId = chunkMetadata.getMeasurementUid(); + if (measurementSet.contains(measurementId)) { + continue; + } + measurementSet.add(measurementId); + Chunk chunk = reader.readMemChunk((ChunkMetadata) chunkMetadata); + ChunkHeader header = chunk.getHeader(); + MeasurementSchema measurementSchema = + new MeasurementSchema( + header.getMeasurementID(), + header.getDataType(), + header.getEncodingType(), + header.getCompressionType()); + schemaList.add(measurementSchema); + } + } + + schemaList.sort(Comparator.comparing(IMeasurementSchema::getMeasurementId)); + return schemaList; + } +} diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ChunkWriterImpl.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ChunkWriterImpl.java index b32a4d6a2b379..ee8e9e6958db0 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ChunkWriterImpl.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/chunk/ChunkWriterImpl.java @@ -24,7 +24,9 @@ import org.apache.iotdb.tsfile.exception.write.PageException; import org.apache.iotdb.tsfile.file.header.ChunkHeader; import org.apache.iotdb.tsfile.file.header.PageHeader; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics; import org.apache.iotdb.tsfile.utils.Binary; import org.apache.iotdb.tsfile.utils.PublicBAOS; @@ -374,6 +376,14 @@ public TSDataType getDataType() { return measurementSchema.getType(); } + public TSEncoding getEncoding() { + return measurementSchema.getEncodingType(); + } + + public CompressionType getCompressionType() { + return measurementSchema.getCompressor(); + } + /** * write the page header and data into the PageWriter's output stream. @NOTE: for upgrading * 0.11/v2 to 0.12/v3 TsFile