This is an automated email from the ASF dual-hosted git repository.
haonan pushed a commit to branch rel/1.2
in repository https://gitbox.apache.org/repos/asf/iotdb.git
The following commit(s) were added to refs/heads/rel/1.2 by this push:
new 964bdce7b8e [IOTDB-5999] Maintain system.properties (#10170) (#10210)
964bdce7b8e is described below
commit 964bdce7b8ec02526921a89f204535326504b98b
Author: YongzaoDan <[email protected]>
AuthorDate: Mon Jun 19 17:59:36 2023 +0800
[IOTDB-5999] Maintain system.properties (#10170) (#10210)
---
.../iotdb/confignode/conf/ConfigNodeConfig.java | 11 ----
.../confignode/conf/ConfigNodeDescriptor.java | 7 ---
.../confignode/conf/ConfigNodeStartupCheck.java | 16 +++++
.../confignode/conf/SystemPropertiesUtils.java | 58 +++++++++---------
.../iotdb/confignode/manager/ConfigManager.java | 20 ++++++-
.../partition/GreedyPartitionAllocator.java | 4 +-
.../iotdb/confignode/manager/node/NodeManager.java | 5 +-
.../manager/partition/PartitionManager.java | 6 +-
.../thrift/ConfigNodeRPCServiceProcessor.java | 3 +-
.../confignode/it/utils/ConfigNodeTestUtils.java | 2 +
.../iotdb/session/it/IoTDBSessionComplexIT.java | 8 +--
.../src/main/thrift/confignode.thrift | 5 ++
.../resources/conf/iotdb-common.properties | 4 --
.../apache/iotdb/commons/conf/CommonConfig.java | 55 ++++++++++++++++-
.../iotdb/commons/conf/CommonDescriptor.java | 23 ++++++++
.../iotdb/commons/utils/CommonDateTimeUtils.java | 42 +++++++++++++
.../java/org/apache/iotdb/db/conf/IoTDBConfig.java | 63 --------------------
.../org/apache/iotdb/db/conf/IoTDBDescriptor.java | 25 +-------
.../org/apache/iotdb/db/conf/IoTDBStartCheck.java | 69 +---------------------
.../org/apache/iotdb/db/engine/StorageEngine.java | 2 +-
.../iotdb/db/engine/storagegroup/DataRegion.java | 13 ++--
.../metadata/cache/DataNodeLastCacheManager.java | 4 +-
.../db/metadata/metric/SchemaEngineMemMetric.java | 3 +-
.../db/metadata/metric/SchemaMetricManager.java | 4 +-
.../db/metadata/rescon/SchemaResourceManager.java | 6 +-
.../db/metadata/schemaregion/SchemaEngine.java | 9 ++-
.../apache/iotdb/db/metadata/tag/TagLogFile.java | 3 +-
.../apache/iotdb/db/metadata/tag/TagManager.java | 21 ++++---
.../operator/process/last/LastQueryUtil.java | 4 +-
.../config/metadata/GetTimeSlotListTask.java | 4 +-
.../pipe/connector/v1/IoTDBThriftConnectorV1.java | 3 +-
.../pipe/connector/v1/IoTDBThriftReceiverV1.java | 9 ++-
.../pipe/connector/v2/IoTDBThriftConnectorV2.java | 3 +-
.../influxdb/handler/QueryHandlerFactory.java | 3 +-
.../influxdb/meta/InfluxDBMetaManagerFactory.java | 3 +-
.../rest/v1/impl/GrafanaApiServiceImpl.java | 3 +-
.../rest/v2/impl/GrafanaApiServiceImpl.java | 3 +-
.../apache/iotdb/db/service/IoTDBShutdownHook.java | 2 +-
.../service/thrift/impl/ClientRPCServiceImpl.java | 2 +-
.../org/apache/iotdb/db/utils/DateTimeUtils.java | 24 ++------
.../apache/iotdb/db/utils/TimePartitionUtils.java | 4 +-
.../SizeTieredCompactionSelectorTest.java | 5 +-
.../db/engine/storagegroup/DataRegionTest.java | 20 ++++---
.../iotdb/db/engine/storagegroup/TTLTest.java | 7 ++-
.../mtree/schemafile/AliasIndexPageTest.java | 6 +-
.../mtree/schemafile/InternalPageTest.java | 6 +-
.../mtree/schemafile/SchemaFileLogTest.java | 6 +-
.../metadata/mtree/schemafile/SchemaFileTest.java | 6 +-
.../mtree/schemafile/WrappedSegmentTest.java | 6 +-
.../schemaRegion/AbstractSchemaRegionTest.java | 9 ++-
.../mpp/plan/analyze/QueryTimePartitionTest.java | 62 +++++++++----------
.../plan/node/write/WritePlanNodeSplitTest.java | 7 ++-
.../org/apache/iotdb/db/qp/sql/ASTVisitorTest.java | 12 ++--
.../iotdb/db/tools/PBTreeFileSketchTest.java | 6 +-
.../iotdb/db/tools/TsFileAndModSettleToolTest.java | 12 ++--
55 files changed, 370 insertions(+), 358 deletions(-)
diff --git
a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java
b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java
index 6bc95fd034a..f9d641ca840 100644
---
a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java
+++
b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeConfig.java
@@ -177,9 +177,6 @@ public class ConfigNodeConfig {
/** External temporary lib directory for storing downloaded pipe JAR files */
private String pipeTemporaryLibDir = pipeDir + File.separator +
IoTDBConstant.TMP_FOLDER_NAME;
- /** Time partition interval in milliseconds */
- private long timePartitionInterval = 604_800_000;
-
/** Procedure Evict ttl */
private int procedureCompletedEvictTTL = 800;
@@ -419,14 +416,6 @@ public class ConfigNodeConfig {
this.seriesPartitionExecutorClass = seriesPartitionExecutorClass;
}
- public long getTimePartitionInterval() {
- return timePartitionInterval;
- }
-
- public void setTimePartitionInterval(long timePartitionInterval) {
- this.timePartitionInterval = timePartitionInterval;
- }
-
public int getCnRpcMaxConcurrentClientNum() {
return rpcMaxConcurrentClientNum;
}
diff --git
a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
index 1a6e3b2d342..9864c382941 100644
---
a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
+++
b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeDescriptor.java
@@ -332,13 +332,6 @@ public class ConfigNodeDescriptor {
conf.setPipeDir(properties.getProperty("pipe_lib_dir",
conf.getPipeDir()).trim());
- conf.setTimePartitionInterval(
- Long.parseLong(
- properties
- .getProperty(
- "time_partition_interval",
String.valueOf(conf.getTimePartitionInterval()))
- .trim()));
-
conf.setHeartbeatIntervalInMs(
Long.parseLong(
properties
diff --git
a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeStartupCheck.java
b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeStartupCheck.java
index 47c0c26f55c..96198e96205 100644
---
a/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeStartupCheck.java
+++
b/confignode/src/main/java/org/apache/iotdb/confignode/conf/ConfigNodeStartupCheck.java
@@ -19,6 +19,8 @@
package org.apache.iotdb.confignode.conf;
+import org.apache.iotdb.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.commons.exception.ConfigurationException;
import org.apache.iotdb.commons.exception.StartupException;
@@ -44,6 +46,7 @@ public class ConfigNodeStartupCheck extends StartupChecks {
private static final Logger LOGGER =
LoggerFactory.getLogger(ConfigNodeStartupCheck.class);
private static final ConfigNodeConfig CONF =
ConfigNodeDescriptor.getInstance().getConf();
+ private static final CommonConfig COMMON_CONFIG =
CommonDescriptor.getInstance().getConfig();
private static final int CONFIGNODE_PORTS = 2;
@@ -165,6 +168,19 @@ public class ConfigNodeStartupCheck extends StartupChecks {
if (CONF.getDefaultDataRegionGroupNumPerDatabase() <= 0) {
throw new ConfigurationException("The default_data_region_group_num
should be positive");
}
+
+ // Check time partition interval
+ if (COMMON_CONFIG.getTimePartitionInterval() <= 0) {
+ throw new ConfigurationException("The time_partition_interval should be
positive");
+ }
+
+ // Check timestamp precision
+ String timestampPrecision = COMMON_CONFIG.getTimestampPrecision();
+ if (!("ms".equals(timestampPrecision)
+ || "us".equals(timestampPrecision)
+ || "ns".equals(timestampPrecision))) {
+ throw new ConfigurationException("The timestamp_precision should be ms,
us or ns");
+ }
}
private void createDirsIfNecessary() throws IOException {
diff --git
a/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java
b/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java
index 5a304fe610d..b9ac61a07e4 100644
---
a/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java
+++
b/confignode/src/main/java/org/apache/iotdb/confignode/conf/SystemPropertiesUtils.java
@@ -19,8 +19,10 @@
package org.apache.iotdb.confignode.conf;
import org.apache.iotdb.common.rpc.thrift.TConfigNodeLocation;
+import org.apache.iotdb.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
+import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.commons.exception.BadNodeUrlException;
-import org.apache.iotdb.commons.exception.ConfigurationException;
import org.apache.iotdb.commons.utils.NodeUrlUtils;
import org.slf4j.Logger;
@@ -49,6 +51,7 @@ public class SystemPropertiesUtils {
+ ConfigNodeConstant.SYSTEM_FILE_NAME);
private static final ConfigNodeConfig conf =
ConfigNodeDescriptor.getInstance().getConf();
+ private static final CommonConfig COMMON_CONFIG =
CommonDescriptor.getInstance().getConfig();
/**
* Check if the ConfigNode is restarted
@@ -64,31 +67,29 @@ public class SystemPropertiesUtils {
* interface when restarted
*
* @throws IOException When read the confignode-system.properties file failed
- * @throws ConfigurationException When some system parameters are
inconsistent
*/
- public static void checkSystemProperties() throws IOException,
ConfigurationException {
+ public static void checkSystemProperties() throws IOException {
Properties systemProperties = getSystemProperties();
boolean needReWrite = false;
+ final String format =
+ "[SystemProperties] The parameter \"{}\" can't be modified after first
startup."
+ + " Your configuration: {} will be forced update to: {}";
- // Startup configuration
+ // Cluster configuration
String clusterName = systemProperties.getProperty(CLUSTER_NAME, null);
- if (clusterName != null && !clusterName.equals(conf.getClusterName())) {
- throw new ConfigurationException(
- CLUSTER_NAME,
- conf.getClusterName(),
- clusterName,
- "cluster_name can't be modified after first startup");
+ if (clusterName == null) {
+ needReWrite = true;
+ } else if (!clusterName.equals(conf.getClusterName())) {
+ LOGGER.warn(format, CLUSTER_NAME, conf.getClusterName(), clusterName);
+ conf.setClusterName(clusterName);
}
String internalAddress =
systemProperties.getProperty("cn_internal_address", null);
if (internalAddress == null) {
needReWrite = true;
} else if (!internalAddress.equals(conf.getInternalAddress())) {
- throw new ConfigurationException(
- "cn_internal_address",
- conf.getInternalAddress(),
- internalAddress,
- "cn_internal_address can't be modified after first startup");
+ LOGGER.warn(format, "cn_internal_address", conf.getInternalAddress(),
internalAddress);
+ conf.setInternalAddress(internalAddress);
}
if (systemProperties.getProperty("cn_internal_port", null) == null) {
@@ -96,11 +97,8 @@ public class SystemPropertiesUtils {
} else {
int internalPort =
Integer.parseInt(systemProperties.getProperty("cn_internal_port"));
if (internalPort != conf.getInternalPort()) {
- throw new ConfigurationException(
- "cn_internal_port",
- String.valueOf(conf.getInternalPort()),
- String.valueOf(internalPort),
- "cn_internal_port can't be modified after first startup");
+ LOGGER.warn(format, "cn_internal_port", conf.getInternalPort(),
internalPort);
+ conf.setInternalPort(internalPort);
}
}
@@ -109,17 +107,11 @@ public class SystemPropertiesUtils {
} else {
int consensusPort =
Integer.parseInt(systemProperties.getProperty("cn_consensus_port"));
if (consensusPort != conf.getConsensusPort()) {
- throw new ConfigurationException(
- "cn_consensus_port",
- String.valueOf(conf.getConsensusPort()),
- String.valueOf(consensusPort),
- "cn_consensus_port can't be modified after first startup");
+ LOGGER.warn(format, "cn_consensus_port", conf.getConsensusPort(),
consensusPort);
+ conf.setConsensusPort(consensusPort);
}
}
- final String format =
- "[SystemProperties] The parameter \"{}\" can't be modified after first
startup."
- + " Your configuration: {} will be forced update to: {}";
// Consensus protocol configuration
String configNodeConsensusProtocolClass =
systemProperties.getProperty("config_node_consensus_protocol_class",
null);
@@ -222,6 +214,8 @@ public class SystemPropertiesUtils {
public static void storeSystemParameters() throws IOException {
Properties systemProperties = getSystemProperties();
+ systemProperties.setProperty("iotdb_version", IoTDBConstant.VERSION);
+
// Cluster configuration
systemProperties.setProperty("cluster_name", conf.getClusterName());
LOGGER.info("[SystemProperties] store cluster_name: {}",
conf.getClusterName());
@@ -252,6 +246,14 @@ public class SystemPropertiesUtils {
"series_partition_slot_num", String.valueOf(conf.getSeriesSlotNum()));
systemProperties.setProperty(
"series_partition_executor_class",
conf.getSeriesPartitionExecutorClass());
+ systemProperties.setProperty(
+ "time_partition_interval",
String.valueOf(COMMON_CONFIG.getTimePartitionInterval()));
+ systemProperties.setProperty("timestamp_precision",
COMMON_CONFIG.getTimestampPrecision());
+
+ // DataNode Functions
+ systemProperties.setProperty("schema_engine_mode",
COMMON_CONFIG.getSchemaEngineMode());
+ systemProperties.setProperty(
+ "tag_attribute_total_size",
String.valueOf(COMMON_CONFIG.getTagAttributeTotalSize()));
storeSystemProperties(systemProperties);
}
diff --git
a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
index 18880cf9597..aa9c46f6db6 100644
---
a/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
+++
b/confignode/src/main/java/org/apache/iotdb/confignode/manager/ConfigManager.java
@@ -475,7 +475,7 @@ public class ConfigManager implements IManager {
clusterParameters.setSeriesPartitionSlotNum(CONF.getSeriesSlotNum());
clusterParameters.setSeriesPartitionExecutorClass(CONF.getSeriesPartitionExecutorClass());
clusterParameters.setDefaultTTL(COMMON_CONF.getDefaultTTLInMs());
-
clusterParameters.setTimePartitionInterval(CONF.getTimePartitionInterval());
+
clusterParameters.setTimePartitionInterval(COMMON_CONF.getTimePartitionInterval());
clusterParameters.setDataReplicationFactor(CONF.getDataReplicationFactor());
clusterParameters.setSchemaReplicationFactor(CONF.getSchemaReplicationFactor());
clusterParameters.setDataRegionPerDataNode(CONF.getDataRegionPerDataNode());
@@ -483,6 +483,8 @@ public class ConfigManager implements IManager {
clusterParameters.setDiskSpaceWarningThreshold(COMMON_CONF.getDiskSpaceWarningThreshold());
clusterParameters.setReadConsistencyLevel(CONF.getReadConsistencyLevel());
clusterParameters.setTimestampPrecision(COMMON_CONF.getTimestampPrecision());
+ clusterParameters.setSchemaEngineMode(COMMON_CONF.getSchemaEngineMode());
+
clusterParameters.setTagAttributeTotalSize(COMMON_CONF.getTagAttributeTotalSize());
return clusterParameters;
}
@@ -1072,7 +1074,7 @@ public class ConfigManager implements IManager {
!= CommonDescriptor.getInstance().getConfig().getDefaultTTLInMs()) {
return errorStatus.setMessage(errorPrefix + "default_ttl" + errorSuffix);
}
- if (clusterParameters.getTimePartitionInterval() !=
CONF.getTimePartitionInterval()) {
+ if (clusterParameters.getTimePartitionInterval() !=
COMMON_CONF.getTimePartitionInterval()) {
return errorStatus.setMessage(errorPrefix + "time_partition_interval" +
errorSuffix);
}
@@ -1095,10 +1097,22 @@ public class ConfigManager implements IManager {
}
if (clusterParameters.getDiskSpaceWarningThreshold()
- !=
CommonDescriptor.getInstance().getConfig().getDiskSpaceWarningThreshold()) {
+ != COMMON_CONF.getDiskSpaceWarningThreshold()) {
return errorStatus.setMessage(errorPrefix +
"disk_space_warning_threshold" + errorSuffix);
}
+ if
(!clusterParameters.getTimestampPrecision().equals(COMMON_CONF.getTimestampPrecision()))
{
+ return errorStatus.setMessage(errorPrefix + "timestamp_precision" +
errorSuffix);
+ }
+
+ if
(!clusterParameters.getSchemaEngineMode().equals(COMMON_CONF.getSchemaEngineMode()))
{
+ return errorStatus.setMessage(errorPrefix + "schema_engine_mode" +
errorSuffix);
+ }
+
+ if (clusterParameters.getTagAttributeTotalSize() !=
COMMON_CONF.getTagAttributeTotalSize()) {
+ return errorStatus.setMessage(errorPrefix + "tag_attribute_total_size" +
errorSuffix);
+ }
+
return null;
}
diff --git
a/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/partition/GreedyPartitionAllocator.java
b/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/partition/GreedyPartitionAllocator.java
index 3ad5cb0a5f1..fae897091df 100644
---
a/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/partition/GreedyPartitionAllocator.java
+++
b/confignode/src/main/java/org/apache/iotdb/confignode/manager/load/balancer/partition/GreedyPartitionAllocator.java
@@ -22,6 +22,7 @@ import org.apache.iotdb.common.rpc.thrift.TConsensusGroupId;
import org.apache.iotdb.common.rpc.thrift.TConsensusGroupType;
import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.partition.DataPartitionTable;
import org.apache.iotdb.commons.partition.SchemaPartitionTable;
import org.apache.iotdb.commons.partition.SeriesPartitionTable;
@@ -45,7 +46,8 @@ public class GreedyPartitionAllocator implements
IPartitionAllocator {
private static final ConfigNodeConfig CONF =
ConfigNodeDescriptor.getInstance().getConf();
private static final boolean ENABLE_DATA_PARTITION_INHERIT_POLICY =
CONF.isEnableDataPartitionInheritPolicy();
- private static final long TIME_PARTITION_INTERVAL =
CONF.getTimePartitionInterval();
+ private static final long TIME_PARTITION_INTERVAL =
+ CommonDescriptor.getInstance().getConfig().getTimePartitionInterval();
private final IManager configManager;
diff --git
a/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java
b/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java
index 8dcf95789d8..8fb6694fc0e 100644
---
a/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java
+++
b/confignode/src/main/java/org/apache/iotdb/confignode/manager/node/NodeManager.java
@@ -135,9 +135,12 @@ public class NodeManager {
globalConfig.setSeriesPartitionSlotNum(configNodeConfig.getSeriesSlotNum());
globalConfig.setSeriesPartitionExecutorClass(
configNodeConfig.getSeriesPartitionExecutorClass());
-
globalConfig.setTimePartitionInterval(configNodeConfig.getTimePartitionInterval());
+
globalConfig.setTimePartitionInterval(commonConfig.getTimePartitionInterval());
globalConfig.setReadConsistencyLevel(configNodeConfig.getReadConsistencyLevel());
globalConfig.setDiskSpaceWarningThreshold(commonConfig.getDiskSpaceWarningThreshold());
+ globalConfig.setTimestampPrecision(commonConfig.getTimestampPrecision());
+ globalConfig.setSchemaEngineMode(commonConfig.getSchemaEngineMode());
+
globalConfig.setTagAttributeTotalSize(commonConfig.getTagAttributeTotalSize());
dataSet.setGlobalConfig(globalConfig);
}
diff --git
a/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionManager.java
b/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionManager.java
index 1d9d4c95756..34f018e23d4 100644
---
a/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionManager.java
+++
b/confignode/src/main/java/org/apache/iotdb/confignode/manager/partition/PartitionManager.java
@@ -28,6 +28,8 @@ import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
import org.apache.iotdb.commons.cluster.RegionRoleType;
import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil;
+import org.apache.iotdb.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.partition.DataPartitionTable;
import org.apache.iotdb.commons.partition.SchemaPartitionTable;
import org.apache.iotdb.commons.partition.executor.SeriesPartitionExecutor;
@@ -117,6 +119,8 @@ public class PartitionManager {
private static final RegionGroupExtensionPolicy
DATA_REGION_GROUP_EXTENSION_POLICY =
CONF.getDataRegionGroupExtensionPolicy();
+ private static final CommonConfig COMMON_CONFIG =
CommonDescriptor.getInstance().getConfig();
+
private final IManager configManager;
private final PartitionInfo partitionInfo;
@@ -865,7 +869,7 @@ public class PartitionManager {
if (req.isSetTimeStamp()) {
plan.setTimeSlotId(
new TTimePartitionSlot(
- req.getTimeStamp() - req.getTimeStamp() %
CONF.getTimePartitionInterval()));
+ req.getTimeStamp() - req.getTimeStamp() %
COMMON_CONFIG.getTimePartitionInterval()));
}
return (GetRegionIdResp) getConsensusManager().read(plan).getDataset();
}
diff --git
a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
index 7b657c1080d..313f0ddbd0e 100644
---
a/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
+++
b/confignode/src/main/java/org/apache/iotdb/confignode/service/thrift/ConfigNodeRPCServiceProcessor.java
@@ -309,7 +309,8 @@ public class ConfigNodeRPCServiceProcessor implements
IConfigNodeRPCService.Ifac
}
if (!databaseSchema.isSetTimePartitionInterval()) {
-
databaseSchema.setTimePartitionInterval(CONFIG_NODE_CONFIG.getTimePartitionInterval());
+ databaseSchema.setTimePartitionInterval(
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval());
} else if (databaseSchema.getTimePartitionInterval() <= 0) {
errorResp =
new TSStatus(TSStatusCode.DATABASE_CONFIG_ERROR.getStatusCode())
diff --git
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java
index e1588956e2f..2e8a704f2dd 100644
---
a/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java
+++
b/integration-test/src/test/java/org/apache/iotdb/confignode/it/utils/ConfigNodeTestUtils.java
@@ -213,6 +213,8 @@ public class ConfigNodeTestUtils {
clusterParameters.setDiskSpaceWarningThreshold(0.05);
clusterParameters.setReadConsistencyLevel("strong");
clusterParameters.setTimestampPrecision("ms");
+ clusterParameters.setSchemaEngineMode("Memory");
+ clusterParameters.setTagAttributeTotalSize(700);
return clusterParameters;
}
diff --git
a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionComplexIT.java
b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionComplexIT.java
index e44c40ce992..c79537ed86c 100644
---
a/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionComplexIT.java
+++
b/integration-test/src/test/java/org/apache/iotdb/session/it/IoTDBSessionComplexIT.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.session.it;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.isession.ISession;
import org.apache.iotdb.isession.SessionDataSet;
import org.apache.iotdb.it.env.EnvFactory;
@@ -266,7 +266,7 @@ public class IoTDBSessionComplexIT {
private void insertRecords(ISession session, List<String> deviceIdList)
throws IoTDBConnectionException, StatementExecutionException {
- long timePartition =
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval();
+ long timePartition =
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval();
List<String> measurements = new ArrayList<>();
measurements.add("s1");
@@ -315,7 +315,7 @@ public class IoTDBSessionComplexIT {
private void insertMultiTablets(ISession session, List<String> deviceIdList)
throws IoTDBConnectionException, StatementExecutionException {
- long timePartition =
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval();
+ long timePartition =
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval();
List<MeasurementSchema> schemaList = new ArrayList<>();
schemaList.add(new MeasurementSchema("s1", TSDataType.INT64));
schemaList.add(new MeasurementSchema("s2", TSDataType.INT64));
@@ -343,7 +343,7 @@ public class IoTDBSessionComplexIT {
private void insertRecordsOfOneDevice(ISession session, String deviceId)
throws IoTDBConnectionException, StatementExecutionException {
- long timePartition =
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval();
+ long timePartition =
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval();
List<String> measurements = new ArrayList<>();
measurements.add("s1");
diff --git a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift
b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift
index 66b402ba5b7..8ad6fc9ac76 100644
--- a/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift
+++ b/iotdb-protocol/thrift-confignode/src/main/thrift/confignode.thrift
@@ -37,6 +37,9 @@ struct TGlobalConfig {
5: required i64 timePartitionInterval
6: required string readConsistencyLevel
7: required double diskSpaceWarningThreshold
+ 8: optional string timestampPrecision
+ 9: optional string schemaEngineMode
+ 10: optional i32 tagAttributeTotalSize
}
struct TRatisConfig {
@@ -353,6 +356,8 @@ struct TClusterParameters {
13: required string seriesPartitionExecutorClass
14: required double diskSpaceWarningThreshold
15: required string timestampPrecision
+ 16: optional string schemaEngineMode
+ 17: optional i32 tagAttributeTotalSize
}
struct TConfigNodeRegisterReq {
diff --git a/node-commons/src/assembly/resources/conf/iotdb-common.properties
b/node-commons/src/assembly/resources/conf/iotdb-common.properties
index 1f91d8aae69..f288c88d634 100644
--- a/node-commons/src/assembly/resources/conf/iotdb-common.properties
+++ b/node-commons/src/assembly/resources/conf/iotdb-common.properties
@@ -803,10 +803,6 @@ cluster_name=defaultCluster
# And it is also used as the default compressor of time column in aligned
timeseries.
# compressor=SNAPPY
-# Maximum degree of a metadataIndex node, default value is 256
-# Datatype: int
-# max_degree_of_index_node=256
-
# time interval in minute for calculating query frequency
# Datatype: int
# frequency_interval_in_minute=1
diff --git
a/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java
b/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java
index b01128cefcc..d4f768fc0c4 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonConfig.java
@@ -130,6 +130,10 @@ public class CommonConfig {
/** Ip and port of target ML node. */
private TEndPoint targetMLNodeEndPoint = new TEndPoint("127.0.0.1", 10810);
+ /** Time partition interval in milliseconds */
+ private long timePartitionInterval = 604_800_000;
+
+ /** This variable set timestamp precision as millisecond, microsecond or
nanosecond. */
private String timestampPrecision = "ms";
/** Pipe related */
@@ -140,7 +144,7 @@ public class CommonConfig {
*/
private String pipeHardlinkTsFileDirName = "pipe";
- /** The maximum number of threads that can be used to execute subtasks in
PipeSubtaskExecutor */
+ /** The maximum number of threads that can be used to execute subtasks in
PipeSubtaskExecutor. */
private int pipeSubtaskExecutorMaxThreadNum = 5;
private int pipeSubtaskExecutorBasicCheckPointIntervalByConsumedEventCount =
10_000;
@@ -161,6 +165,15 @@ public class CommonConfig {
private long pipeMetaSyncerInitialSyncDelayMinutes = 3;
private long pipeMetaSyncerSyncIntervalMinutes = 3;
+ /** whether to use persistent schema mode. */
+ private String schemaEngineMode = "Memory";
+
+ /** Whether to enable Last cache. */
+ private boolean lastCacheEnable = true;
+
+ // max size for tag and attribute of one time series
+ private int tagAttributeTotalSize = 700;
+
CommonConfig() {}
public void updatePath(String homeDir) {
@@ -413,7 +426,23 @@ public class CommonConfig {
isStopping = stopping;
}
+ public long getTimePartitionInterval() {
+ return timePartitionInterval;
+ }
+
+ public void setTimePartitionInterval(long timePartitionInterval) {
+ this.timePartitionInterval = timePartitionInterval;
+ }
+
public void setTimestampPrecision(String timestampPrecision) {
+ if (!("ms".equals(timestampPrecision)
+ || "us".equals(timestampPrecision)
+ || "ns".equals(timestampPrecision))) {
+ logger.error(
+ "Wrong timestamp precision, please set as: ms, us or ns ! Current
is: {}",
+ timestampPrecision);
+ System.exit(-1);
+ }
this.timestampPrecision = timestampPrecision;
}
@@ -550,4 +579,28 @@ public class CommonConfig {
this.pipeSubtaskExecutorPendingQueueMaxBlockingTimeMs =
pipeSubtaskExecutorPendingQueueMaxBlockingTimeMs;
}
+
+ public String getSchemaEngineMode() {
+ return schemaEngineMode;
+ }
+
+ public void setSchemaEngineMode(String schemaEngineMode) {
+ this.schemaEngineMode = schemaEngineMode;
+ }
+
+ public boolean isLastCacheEnable() {
+ return lastCacheEnable;
+ }
+
+ public void setLastCacheEnable(boolean lastCacheEnable) {
+ this.lastCacheEnable = lastCacheEnable;
+ }
+
+ public int getTagAttributeTotalSize() {
+ return tagAttributeTotalSize;
+ }
+
+ public void setTagAttributeTotalSize(int tagAttributeTotalSize) {
+ this.tagAttributeTotalSize = tagAttributeTotalSize;
+ }
}
diff --git
a/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java
b/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java
index d2410d1cba3..2ab38b9dbb4 100644
---
a/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java
+++
b/node-commons/src/main/java/org/apache/iotdb/commons/conf/CommonDescriptor.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.commons.conf;
import org.apache.iotdb.commons.enums.HandleSystemErrorStrategy;
import org.apache.iotdb.commons.exception.BadNodeUrlException;
+import org.apache.iotdb.commons.utils.CommonDateTimeUtils;
import org.apache.iotdb.commons.utils.NodeUrlUtils;
import org.apache.iotdb.confignode.rpc.thrift.TGlobalConfig;
@@ -212,6 +213,22 @@ public class CommonDescriptor {
"Illegal target MLNode endpoint url format in config file: {}, use
default configuration.",
endPointUrl);
}
+
+ config.setSchemaEngineMode(
+ properties.getProperty("schema_engine_mode",
String.valueOf(config.getSchemaEngineMode())));
+
+ config.setLastCacheEnable(
+ Boolean.parseBoolean(
+ properties.getProperty(
+ "enable_last_cache",
Boolean.toString(config.isLastCacheEnable()))));
+ if (config.getSchemaEngineMode().equals("Rocksdb_based")) {
+ config.setLastCacheEnable(false);
+ }
+
+ config.setTagAttributeTotalSize(
+ Integer.parseInt(
+ properties.getProperty(
+ "tag_attribute_total_size",
String.valueOf(config.getTagAttributeTotalSize()))));
}
private void loadPipeProps(Properties properties) {
@@ -300,6 +317,12 @@ public class CommonDescriptor {
}
public void loadGlobalConfig(TGlobalConfig globalConfig) {
+ config.setTimePartitionInterval(
+ CommonDateTimeUtils.convertMilliTimeWithPrecision(
+ globalConfig.timePartitionInterval,
config.getTimestampPrecision()));
+ config.setTimestampPrecision(globalConfig.timestampPrecision);
+ config.setSchemaEngineMode(globalConfig.schemaEngineMode);
+ config.setTagAttributeTotalSize(globalConfig.tagAttributeTotalSize);
config.setDiskSpaceWarningThreshold(globalConfig.getDiskSpaceWarningThreshold());
}
}
diff --git
a/node-commons/src/main/java/org/apache/iotdb/commons/utils/CommonDateTimeUtils.java
b/node-commons/src/main/java/org/apache/iotdb/commons/utils/CommonDateTimeUtils.java
new file mode 100644
index 00000000000..3d65058abc9
--- /dev/null
+++
b/node-commons/src/main/java/org/apache/iotdb/commons/utils/CommonDateTimeUtils.java
@@ -0,0 +1,42 @@
+/*
+ * 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.commons.utils;
+
+public class CommonDateTimeUtils {
+
+ public CommonDateTimeUtils() {
+ // Empty constructor
+ }
+
+ public static long convertMilliTimeWithPrecision(long milliTime, String
timePrecision) {
+ long result = milliTime;
+ switch (timePrecision) {
+ case "ns":
+ result = milliTime * 1000_000L;
+ break;
+ case "us":
+ result = milliTime * 1000L;
+ break;
+ default:
+ break;
+ }
+ return result;
+ }
+}
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 004bef1d107..7c3bd482d40 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
@@ -175,9 +175,6 @@ public class IoTDBConfig {
/** When inserting rejected exceeds this, throw an exception. Unit:
millisecond */
private int maxWaitingTimeWhenInsertBlockedInMs = 10000;
- /** this variable set timestamp precision as millisecond, microsecond or
nanosecond */
- private String timestampPrecision = "ms";
-
// region Write Ahead Log Configuration
/** Write mode of wal */
private volatile WALMode walMode = WALMode.ASYNC;
@@ -566,9 +563,6 @@ public class IoTDBConfig {
*/
private boolean enableQueryMemoryEstimation = true;
- /** Whether to enable Last cache */
- private boolean lastCacheEnable = true;
-
/** Cache size of {@code checkAndGetDataTypeCache}. */
private int mRemoteSchemaCacheSize = 100000;
@@ -759,9 +753,6 @@ public class IoTDBConfig {
/** The default value of primitive array size in array pool */
private int primitiveArraySize = 64;
- /** Time partition interval in milliseconds */
- private long timePartitionInterval = 604_800_000;
-
/**
* Level of TimeIndex, which records the start time and end time of
TsFileResource. Currently,
* DEVICE_TIME_INDEX and FILE_TIME_INDEX are supported, and could not be
changed after first set.
@@ -772,9 +763,6 @@ public class IoTDBConfig {
// wait for 60 second by default.
private int thriftServerAwaitTimeForStopService = 60;
- // max size for tag and attribute of one time series
- private int tagAttributeTotalSize = 700;
-
// Interval num of tag and attribute records when force flushing to disk
private int tagAttributeFlushInterval = 1000;
@@ -850,9 +838,6 @@ public class IoTDBConfig {
*/
private boolean enableIDTableLogFile = false;
- /** whether to use persistent schema mode */
- private String schemaEngineMode = "Memory";
-
/** the memory used for metadata cache when using persistent schema */
private int cachedMNodeSizeInPBTreeMode = -1;
@@ -1201,14 +1186,6 @@ public class IoTDBConfig {
this.defaultFillInterval = defaultFillInterval;
}
- public long getTimePartitionInterval() {
- return timePartitionInterval;
- }
-
- public void setTimePartitionInterval(long timePartitionInterval) {
- this.timePartitionInterval = timePartitionInterval;
- }
-
public TimeIndexLevel getTimeIndexLevel() {
return timeIndexLevel;
}
@@ -1378,22 +1355,6 @@ public class IoTDBConfig {
this.rpcPort = rpcPort;
}
- public String getTimestampPrecision() {
- return timestampPrecision;
- }
-
- public void setTimestampPrecision(String timestampPrecision) {
- if (!("ms".equals(timestampPrecision)
- || "us".equals(timestampPrecision)
- || "ns".equals(timestampPrecision))) {
- logger.error(
- "Wrong timestamp precision, please set as: ms, us or ns ! Current
is: {}",
- timestampPrecision);
- System.exit(-1);
- }
- this.timestampPrecision = timestampPrecision;
- }
-
public boolean isEnableDiscardOutOfOrderData() {
return enableDiscardOutOfOrderData;
}
@@ -2195,14 +2156,6 @@ public class IoTDBConfig {
this.enableQueryMemoryEstimation = enableQueryMemoryEstimation;
}
- public boolean isLastCacheEnabled() {
- return lastCacheEnable;
- }
-
- public void setEnableLastCache(boolean lastCacheEnable) {
- this.lastCacheEnable = lastCacheEnable;
- }
-
public boolean isAutoCreateSchemaEnabled() {
return enableAutoCreateSchema;
}
@@ -2518,14 +2471,6 @@ public class IoTDBConfig {
this.mqttMaxMessageSize = mqttMaxMessageSize;
}
- public int getTagAttributeTotalSize() {
- return tagAttributeTotalSize;
- }
-
- public void setTagAttributeTotalSize(int tagAttributeTotalSize) {
- this.tagAttributeTotalSize = tagAttributeTotalSize;
- }
-
public int getTagAttributeFlushInterval() {
return tagAttributeFlushInterval;
}
@@ -2925,14 +2870,6 @@ public class IoTDBConfig {
this.enableIDTableLogFile = enableIDTableLogFile;
}
- public String getSchemaEngineMode() {
- return schemaEngineMode;
- }
-
- public void setSchemaEngineMode(String schemaEngineMode) {
- this.schemaEngineMode = schemaEngineMode;
- }
-
public int getCachedMNodeSizeInPBTreeMode() {
return cachedMNodeSizeInPBTreeMode;
}
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
index edf1b124da0..3f26f47866f 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
@@ -299,9 +299,6 @@ public class IoTDBDescriptor {
.getProperty("mlnode_rpc_port",
Integer.toString(conf.getMLNodePort()))
.trim()));
- conf.setTimestampPrecision(
- properties.getProperty("timestamp_precision",
conf.getTimestampPrecision()).trim());
-
conf.setBufferedArraysMemoryProportion(
Double.parseDouble(
properties
@@ -782,11 +779,6 @@ public class IoTDBDescriptor {
properties.getProperty(
"default_fill_interval",
String.valueOf(conf.getDefaultFillInterval()))));
- conf.setTagAttributeTotalSize(
- Integer.parseInt(
- properties.getProperty(
- "tag_attribute_total_size",
String.valueOf(conf.getTagAttributeTotalSize()))));
-
conf.setTagAttributeFlushInterval(
Integer.parseInt(
properties.getProperty(
@@ -870,18 +862,6 @@ public class IoTDBDescriptor {
properties.getProperty(
"enable_id_table_log_file",
String.valueOf(conf.isEnableIDTableLogFile()))));
- conf.setSchemaEngineMode(
- properties.getProperty("schema_engine_mode",
String.valueOf(conf.getSchemaEngineMode())));
-
- conf.setEnableLastCache(
- Boolean.parseBoolean(
- properties.getProperty(
- "enable_last_cache",
Boolean.toString(conf.isLastCacheEnabled()))));
-
- if (conf.getSchemaEngineMode().equals("Rocksdb_based")) {
- conf.setEnableLastCache(false);
- }
-
conf.setCachedMNodeSizeInPBTreeMode(
Integer.parseInt(
properties.getProperty(
@@ -1936,7 +1916,7 @@ public class IoTDBDescriptor {
conf.setContinuousQueryMinimumEveryInterval(
DateTimeUtils.convertDurationStrToLong(
properties.getProperty("continuous_query_minimum_every_interval",
"1s"),
- conf.getTimestampPrecision()));
+
CommonDescriptor.getInstance().getConfig().getTimestampPrecision()));
}
public void loadClusterProps(Properties properties) {
@@ -2042,9 +2022,6 @@ public class IoTDBDescriptor {
public void loadGlobalConfig(TGlobalConfig globalConfig) {
conf.setSeriesPartitionExecutorClass(globalConfig.getSeriesPartitionExecutorClass());
conf.setSeriesPartitionSlotNum(globalConfig.getSeriesPartitionSlotNum());
- conf.setTimePartitionInterval(
- DateTimeUtils.convertMilliTimeWithPrecision(
- globalConfig.timePartitionInterval, conf.getTimestampPrecision()));
conf.setReadConsistencyLevel(globalConfig.getReadConsistencyLevel());
}
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBStartCheck.java
b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBStartCheck.java
index 20680724bae..49254e93b6b 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBStartCheck.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBStartCheck.java
@@ -29,7 +29,6 @@ import org.apache.iotdb.consensus.ConsensusFactory;
import org.apache.iotdb.db.conf.directories.DirectoryChecker;
import org.apache.iotdb.db.wal.utils.WALMode;
import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
-import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
import org.apache.commons.io.FileUtils;
import org.slf4j.Logger;
@@ -71,43 +70,8 @@ public class IoTDBStartCheck {
// region params need checking, determined when first start
private static final String SYSTEM_PROPERTIES_STRING = "System properties:";
- private static final String TIMESTAMP_PRECISION_STRING =
"timestamp_precision";
- private static final String PARTITION_INTERVAL_STRING =
"time_partition_interval";
- private static final String TSFILE_FILE_SYSTEM_STRING = "tsfile_storage_fs";
- private static final String TAG_ATTRIBUTE_SIZE_STRING =
"tag_attribute_total_size";
- private static final String TAG_ATTRIBUTE_FLUSH_INTERVAL =
"tag_attribute_flush_interval";
- private static final String MAX_DEGREE_OF_INDEX_STRING =
"max_degree_of_index_node";
private static final String DATA_REGION_NUM = "data_region_num";
- private static final String ENABLE_ID_TABLE = "enable_id_table";
- private static final String ENABLE_ID_TABLE_LOG_FILE =
"enable_id_table_log_file";
- private static final String SCHEMA_ENGINE_MODE = "schema_engine_mode";
- private static final String TIME_ENCODER_KEY = "time_encoder";
- // Immutable system parameters
- private static final Map<String, Supplier<String>> constantParamValueTable =
new HashMap<>();
-
- static {
- constantParamValueTable.put(TIMESTAMP_PRECISION_STRING,
config::getTimestampPrecision);
- constantParamValueTable.put(
- PARTITION_INTERVAL_STRING, () ->
String.valueOf(config.getTimePartitionInterval()));
- constantParamValueTable.put(
- TSFILE_FILE_SYSTEM_STRING, () ->
config.getTsFileStorageFs().toString());
- constantParamValueTable.put(
- TAG_ATTRIBUTE_SIZE_STRING, () ->
String.valueOf(config.getTagAttributeTotalSize()));
- constantParamValueTable.put(
- TAG_ATTRIBUTE_FLUSH_INTERVAL, () ->
String.valueOf(config.getTagAttributeFlushInterval()));
- constantParamValueTable.put(
- MAX_DEGREE_OF_INDEX_STRING,
- () ->
String.valueOf(TSFileDescriptor.getInstance().getConfig().getMaxDegreeOfIndexNode()));
- constantParamValueTable.put(DATA_REGION_NUM, () ->
String.valueOf(config.getDataRegionNum()));
- constantParamValueTable.put(ENABLE_ID_TABLE, () ->
String.valueOf(config.isEnableIDTable()));
- constantParamValueTable.put(
- ENABLE_ID_TABLE_LOG_FILE, () ->
String.valueOf(config.isEnableIDTableLogFile()));
- constantParamValueTable.put(
- SCHEMA_ENGINE_MODE, () ->
String.valueOf(config.getSchemaEngineMode()));
- constantParamValueTable.put(
- TIME_ENCODER_KEY,
TSFileDescriptor.getInstance().getConfig()::getTimeEncoder);
- }
// endregion
// region params don't need checking and can be updated
private static final String INTERNAL_ADDRESS = "dn_internal_address";
@@ -154,9 +118,7 @@ public class IoTDBStartCheck {
}
private String getVal(String paramName) {
- if (constantParamValueTable.containsKey(paramName)) {
- return constantParamValueTable.get(paramName).get();
- } else if (variableParamValueTable.containsKey(paramName)) {
+ if (variableParamValueTable.containsKey(paramName)) {
return variableParamValueTable.get(paramName).get();
} else {
return null;
@@ -184,28 +146,7 @@ public class IoTDBStartCheck {
SystemFileFactory.INSTANCE.getFile(
IoTDBStartCheck.SCHEMA_DIR + File.separator + PROPERTIES_FILE_NAME
+ ".tmp");
- // Check time stamp precision
- String timestampPrecision = getVal(TIMESTAMP_PRECISION_STRING);
- if (!("ms".equals(timestampPrecision)
- || "us".equals(timestampPrecision)
- || "ns".equals(timestampPrecision))) {
- logger.error(
- "Wrong {}, please set as: ms, us or ns ! Current is: {}",
- TIMESTAMP_PRECISION_STRING,
- timestampPrecision);
- System.exit(-1);
- }
-
- // check partition interval
- if (Long.parseLong(getVal(PARTITION_INTERVAL_STRING)) <= 0) {
- logger.error("Time partition interval must larger than 0!");
- System.exit(-1);
- }
-
systemProperties.put(IOTDB_VERSION_STRING, () -> IoTDBConstant.VERSION);
- for (String param : constantParamValueTable.keySet()) {
- systemProperties.put(param, () -> getVal(param));
- }
for (String param : variableParamValueTable.keySet()) {
systemProperties.put(param, () -> getVal(param));
}
@@ -414,7 +355,7 @@ public class IoTDBStartCheck {
}
/** Check all immutable properties */
- private void checkImmutableSystemProperties() throws ConfigurationException,
IOException {
+ private void checkImmutableSystemProperties() throws IOException {
for (Entry<String, Supplier<String>> entry : systemProperties.entrySet()) {
if (!properties.containsKey(entry.getKey())) {
upgradePropertiesFileFromBrokenFile();
@@ -422,12 +363,6 @@ public class IoTDBStartCheck {
}
}
- for (String param : constantParamValueTable.keySet()) {
- if (!(properties.getProperty(param).equals(getVal(param)))) {
- throwException(param, getVal(param));
- }
- }
-
// load configuration from system properties only when start as Data node
if (properties.containsKey(IoTDBConstant.CLUSTER_NAME)) {
config.setClusterName(properties.getProperty(IoTDBConstant.CLUSTER_NAME));
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 b656d5a523f..b44d51aa72e 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
@@ -151,7 +151,7 @@ public class StorageEngine implements IService {
}
private static void initTimePartition() {
- timePartitionInterval =
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval();
+ timePartitionInterval =
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval();
}
public static long getTimePartitionInterval() {
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 5526a74b9fd..de9a822facf 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
@@ -33,6 +33,7 @@ import org.apache.iotdb.commons.file.SystemFileFactory;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.service.metric.MetricService;
import org.apache.iotdb.commons.service.metric.PerformanceOverviewMetrics;
+import org.apache.iotdb.commons.utils.CommonDateTimeUtils;
import org.apache.iotdb.commons.utils.TestOnly;
import org.apache.iotdb.consensus.ConsensusFactory;
import org.apache.iotdb.db.conf.IoTDBConfig;
@@ -1148,7 +1149,7 @@ public class DataRegion implements IDataRegionForQuery {
}
private void tryToUpdateBatchInsertLastCache(InsertTabletNode node, long
latestFlushedTime) {
- if (!IoTDBDescriptor.getInstance().getConfig().isLastCacheEnabled()
+ if (!CommonDescriptor.getInstance().getConfig().isLastCacheEnable()
||
(config.getDataRegionConsensusProtocolClass().equals(ConsensusFactory.IOT_CONSENSUS)
&& node.isSyncFromLeaderWhenUsingIoTConsensus())) {
// disable updating last cache on follower
@@ -1202,7 +1203,7 @@ public class DataRegion implements IDataRegionForQuery {
}
private void tryToUpdateInsertLastCache(InsertRowNode node, long
latestFlushedTime) {
- if (!IoTDBDescriptor.getInstance().getConfig().isLastCacheEnabled()
+ if (!CommonDescriptor.getInstance().getConfig().isLastCacheEnable()
||
(config.getDataRegionConsensusProtocolClass().equals(ConsensusFactory.IOT_CONSENSUS)
&& node.isSyncFromLeaderWhenUsingIoTConsensus())) {
// disable updating last cache on follower
@@ -1633,7 +1634,7 @@ public class DataRegion implements IDataRegionForQuery {
resource.getTsFilePath(),
new Date(ttlLowerBound),
dataTTL,
- config.getTimestampPrecision());
+ CommonDescriptor.getInstance().getConfig().getTimestampPrecision());
} finally {
resource.writeUnlock();
}
@@ -2357,7 +2358,7 @@ public class DataRegion implements IDataRegionForQuery {
}
private void resetLastCacheWhenLoadingTsFile() throws IllegalPathException {
- if (!IoTDBDescriptor.getInstance().getConfig().isLastCacheEnabled()) {
+ if (!CommonDescriptor.getInstance().getConfig().isLastCacheEnable()) {
return;
}
DataNodeSchemaCache.getInstance().takeWriteLock();
@@ -2744,8 +2745,8 @@ public class DataRegion implements IDataRegionForQuery {
public void setDataTTLWithTimePrecisionCheck(long dataTTL) {
if (dataTTL != Long.MAX_VALUE) {
dataTTL =
- DateTimeUtils.convertMilliTimeWithPrecision(
- dataTTL,
IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision());
+ CommonDateTimeUtils.convertMilliTimeWithPrecision(
+ dataTTL,
CommonDescriptor.getInstance().getConfig().getTimestampPrecision());
}
this.dataTTL = dataTTL;
}
diff --git
a/server/src/main/java/org/apache/iotdb/db/metadata/cache/DataNodeLastCacheManager.java
b/server/src/main/java/org/apache/iotdb/db/metadata/cache/DataNodeLastCacheManager.java
index 7417160f47e..1ac491e5275 100644
---
a/server/src/main/java/org/apache/iotdb/db/metadata/cache/DataNodeLastCacheManager.java
+++
b/server/src/main/java/org/apache/iotdb/db/metadata/cache/DataNodeLastCacheManager.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.metadata.cache;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import
org.apache.iotdb.db.metadata.cache.lastCache.container.ILastCacheContainer;
import org.apache.iotdb.tsfile.read.TimeValuePair;
@@ -30,7 +30,7 @@ public class DataNodeLastCacheManager {
private static final Logger logger =
LoggerFactory.getLogger(DataNodeLastCacheManager.class);
private static final boolean CACHE_ENABLED =
- IoTDBDescriptor.getInstance().getConfig().isLastCacheEnabled();
+ CommonDescriptor.getInstance().getConfig().isLastCacheEnable();
/**
* get the last cache value from time series
diff --git
a/server/src/main/java/org/apache/iotdb/db/metadata/metric/SchemaEngineMemMetric.java
b/server/src/main/java/org/apache/iotdb/db/metadata/metric/SchemaEngineMemMetric.java
index 389df823127..999a68e2e5b 100644
---
a/server/src/main/java/org/apache/iotdb/db/metadata/metric/SchemaEngineMemMetric.java
+++
b/server/src/main/java/org/apache/iotdb/db/metadata/metric/SchemaEngineMemMetric.java
@@ -18,6 +18,7 @@
*/
package org.apache.iotdb.db.metadata.metric;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.service.metric.enums.Metric;
import org.apache.iotdb.commons.service.metric.enums.Tag;
import org.apache.iotdb.consensus.ConsensusFactory;
@@ -93,7 +94,7 @@ public class SchemaEngineMemMetric implements
ISchemaEngineMetric {
Tag.NAME.toString(),
REGION_NUMBER);
metricService.gauge(
-
SchemaEngineMode.valueOf(IoTDBDescriptor.getInstance().getConfig().getSchemaEngineMode())
+
SchemaEngineMode.valueOf(CommonDescriptor.getInstance().getConfig().getSchemaEngineMode())
.getCode(),
Metric.SCHEMA_ENGINE.toString(),
MetricLevel.IMPORTANT,
diff --git
a/server/src/main/java/org/apache/iotdb/db/metadata/metric/SchemaMetricManager.java
b/server/src/main/java/org/apache/iotdb/db/metadata/metric/SchemaMetricManager.java
index 37327c7a690..f62832a2234 100644
---
a/server/src/main/java/org/apache/iotdb/db/metadata/metric/SchemaMetricManager.java
+++
b/server/src/main/java/org/apache/iotdb/db/metadata/metric/SchemaMetricManager.java
@@ -18,8 +18,8 @@
*/
package org.apache.iotdb.db.metadata.metric;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.service.metric.MetricService;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.metadata.mtree.store.disk.cache.CacheMemoryManager;
import org.apache.iotdb.db.metadata.rescon.ISchemaEngineStatistics;
import org.apache.iotdb.db.metadata.schemaregion.ISchemaRegion;
@@ -35,7 +35,7 @@ public class SchemaMetricManager {
private SchemaMetricManager() {}
public void init(ISchemaEngineStatistics engineStatistics) {
- if
(IoTDBDescriptor.getInstance().getConfig().getSchemaEngineMode().equals("Memory"))
{
+ if
(CommonDescriptor.getInstance().getConfig().getSchemaEngineMode().equals("Memory"))
{
engineMetric = new
SchemaEngineMemMetric(engineStatistics.getAsMemSchemaEngineStatistics());
} else {
SchemaEngineCachedMetric schemaEngineCachedMetric =
diff --git
a/server/src/main/java/org/apache/iotdb/db/metadata/rescon/SchemaResourceManager.java
b/server/src/main/java/org/apache/iotdb/db/metadata/rescon/SchemaResourceManager.java
index 457b5a894f0..28e85fd5e5d 100644
---
a/server/src/main/java/org/apache/iotdb/db/metadata/rescon/SchemaResourceManager.java
+++
b/server/src/main/java/org/apache/iotdb/db/metadata/rescon/SchemaResourceManager.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.metadata.rescon;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.db.metadata.mtree.store.disk.cache.CacheMemoryManager;
import org.apache.iotdb.db.metadata.schemaregion.SchemaEngineMode;
@@ -28,7 +28,7 @@ public class SchemaResourceManager {
private SchemaResourceManager() {}
public static void initSchemaResource(ISchemaEngineStatistics
engineStatistics) {
- if (IoTDBDescriptor.getInstance()
+ if (CommonDescriptor.getInstance()
.getConfig()
.getSchemaEngineMode()
.equals(SchemaEngineMode.PB_Tree.toString())) {
@@ -37,7 +37,7 @@ public class SchemaResourceManager {
}
public static void clearSchemaResource() {
- if (IoTDBDescriptor.getInstance()
+ if (CommonDescriptor.getInstance()
.getConfig()
.getSchemaEngineMode()
.equals(SchemaEngineMode.PB_Tree.toString())) {
diff --git
a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaEngine.java
b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaEngine.java
index eef8b139424..a28ddb2dc93 100644
---
a/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaEngine.java
+++
b/server/src/main/java/org/apache/iotdb/db/metadata/schemaregion/SchemaEngine.java
@@ -21,6 +21,7 @@ package org.apache.iotdb.db.metadata.schemaregion;
import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
import org.apache.iotdb.commons.concurrent.threadpool.ScheduledExecutorUtil;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.consensus.SchemaRegionId;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.exception.MetadataException;
@@ -91,9 +92,11 @@ public class SchemaEngine {
}
public void init() {
- logger.info("used schema engine mode: {}.", config.getSchemaEngineMode());
+ logger.info(
+ "used schema engine mode: {}.",
+ CommonDescriptor.getInstance().getConfig().getSchemaEngineMode());
- schemaRegionLoader.init(config.getSchemaEngineMode());
+
schemaRegionLoader.init(CommonDescriptor.getInstance().getConfig().getSchemaEngineMode());
initSchemaEngineStatistics();
SchemaResourceManager.initSchemaResource(schemaEngineStatistics);
@@ -190,7 +193,7 @@ public class SchemaEngine {
}
private void initSchemaEngineStatistics() {
- if
(IoTDBDescriptor.getInstance().getConfig().getSchemaEngineMode().equals("Memory"))
{
+ if
(CommonDescriptor.getInstance().getConfig().getSchemaEngineMode().equals("Memory"))
{
schemaEngineStatistics = new MemSchemaEngineStatistics();
} else {
schemaEngineStatistics = new CachedSchemaEngineStatistics();
diff --git
a/server/src/main/java/org/apache/iotdb/db/metadata/tag/TagLogFile.java
b/server/src/main/java/org/apache/iotdb/db/metadata/tag/TagLogFile.java
index 07d47d84000..ac3dffd8ba1 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/tag/TagLogFile.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/tag/TagLogFile.java
@@ -18,6 +18,7 @@
*/
package org.apache.iotdb.db.metadata.tag;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.file.SystemFileFactory;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -48,7 +49,7 @@ public class TagLogFile implements AutoCloseable {
+ "Please enlarge tag_attribute_total_size in
iotdb-common.properties";
private static final int MAX_LENGTH =
- IoTDBDescriptor.getInstance().getConfig().getTagAttributeTotalSize();
+ CommonDescriptor.getInstance().getConfig().getTagAttributeTotalSize();
private static final int RECORD_FLUSH_INTERVAL =
IoTDBDescriptor.getInstance().getConfig().getTagAttributeFlushInterval();
diff --git
a/server/src/main/java/org/apache/iotdb/db/metadata/tag/TagManager.java
b/server/src/main/java/org/apache/iotdb/db/metadata/tag/TagManager.java
index 428e0ebb7ee..1e1d74ae416 100644
--- a/server/src/main/java/org/apache/iotdb/db/metadata/tag/TagManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/metadata/tag/TagManager.java
@@ -18,6 +18,8 @@
*/
package org.apache.iotdb.db.metadata.tag;
+import org.apache.iotdb.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.file.SystemFileFactory;
import org.apache.iotdb.commons.path.PartialPath;
@@ -64,6 +66,7 @@ public class TagManager {
private static final Logger logger =
LoggerFactory.getLogger(TagManager.class);
private static IoTDBConfig config =
IoTDBDescriptor.getInstance().getConfig();
+ private static final CommonConfig COMMON_CONFIG =
CommonDescriptor.getInstance().getConfig();
private String sgSchemaDirPath;
private TagLogFile tagLogFile;
@@ -127,7 +130,7 @@ public class TagManager {
public boolean recoverIndex(long offset, IMeasurementMNode<?>
measurementMNode)
throws IOException {
- Map<String, String> tags =
tagLogFile.readTag(config.getTagAttributeTotalSize(), offset);
+ Map<String, String> tags =
tagLogFile.readTag(COMMON_CONFIG.getTagAttributeTotalSize(), offset);
if (tags == null || tags.isEmpty()) {
return false;
} else {
@@ -292,7 +295,7 @@ public class TagManager {
return;
}
Map<String, String> tagMap =
- tagLogFile.readTag(config.getTagAttributeTotalSize(),
node.getOffset());
+ tagLogFile.readTag(COMMON_CONFIG.getTagAttributeTotalSize(),
node.getOffset());
if (tagMap != null) {
for (Map.Entry<String, String> entry : tagMap.entrySet()) {
if (tagIndex.containsKey(entry.getKey())
@@ -338,7 +341,7 @@ public class TagManager {
throws MetadataException, IOException {
Pair<Map<String, String>, Map<String, String>> pair =
- tagLogFile.read(config.getTagAttributeTotalSize(),
leafMNode.getOffset());
+ tagLogFile.read(COMMON_CONFIG.getTagAttributeTotalSize(),
leafMNode.getOffset());
if (tagsMap != null) {
for (Map.Entry<String, String> entry : tagsMap.entrySet()) {
@@ -402,7 +405,7 @@ public class TagManager {
throws MetadataException, IOException {
Pair<Map<String, String>, Map<String, String>> pair =
- tagLogFile.read(config.getTagAttributeTotalSize(),
leafMNode.getOffset());
+ tagLogFile.read(COMMON_CONFIG.getTagAttributeTotalSize(),
leafMNode.getOffset());
for (Map.Entry<String, String> entry : attributesMap.entrySet()) {
String key = entry.getKey();
@@ -430,7 +433,7 @@ public class TagManager {
throws MetadataException, IOException {
Pair<Map<String, String>, Map<String, String>> pair =
- tagLogFile.read(config.getTagAttributeTotalSize(),
leafMNode.getOffset());
+ tagLogFile.read(COMMON_CONFIG.getTagAttributeTotalSize(),
leafMNode.getOffset());
for (Map.Entry<String, String> entry : tagsMap.entrySet()) {
String key = entry.getKey();
@@ -459,7 +462,7 @@ public class TagManager {
Set<String> keySet, PartialPath fullPath, IMeasurementMNode<?> leafMNode)
throws MetadataException, IOException {
Pair<Map<String, String>, Map<String, String>> pair =
- tagLogFile.read(config.getTagAttributeTotalSize(),
leafMNode.getOffset());
+ tagLogFile.read(COMMON_CONFIG.getTagAttributeTotalSize(),
leafMNode.getOffset());
Map<String, String> deleteTag = new HashMap<>();
for (String key : keySet) {
@@ -531,7 +534,7 @@ public class TagManager {
throws MetadataException, IOException {
// tags, attributes
Pair<Map<String, String>, Map<String, String>> pair =
- tagLogFile.read(config.getTagAttributeTotalSize(),
leafMNode.getOffset());
+ tagLogFile.read(COMMON_CONFIG.getTagAttributeTotalSize(),
leafMNode.getOffset());
Map<String, String> oldTagValue = new HashMap<>();
Map<String, String> newTagValue = new HashMap<>();
@@ -601,7 +604,7 @@ public class TagManager {
throws MetadataException, IOException {
// tags, attributes
Pair<Map<String, String>, Map<String, String>> pair =
- tagLogFile.read(config.getTagAttributeTotalSize(),
leafMNode.getOffset());
+ tagLogFile.read(COMMON_CONFIG.getTagAttributeTotalSize(),
leafMNode.getOffset());
// current name has existed
if (pair.left.containsKey(newKey) || pair.right.containsKey(newKey)) {
@@ -663,7 +666,7 @@ public class TagManager {
public Pair<Map<String, String>, Map<String, String>> readTagFile(long
tagFileOffset)
throws IOException {
- return tagLogFile.read(config.getTagAttributeTotalSize(), tagFileOffset);
+ return tagLogFile.read(COMMON_CONFIG.getTagAttributeTotalSize(),
tagFileOffset);
}
/**
diff --git
a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/last/LastQueryUtil.java
b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/last/LastQueryUtil.java
index f8bb75f0885..c674fc44f86 100644
---
a/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/last/LastQueryUtil.java
+++
b/server/src/main/java/org/apache/iotdb/db/mpp/execution/operator/process/last/LastQueryUtil.java
@@ -18,7 +18,7 @@
*/
package org.apache.iotdb.db.mpp.execution.operator.process.last;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.db.mpp.aggregation.Aggregator;
import org.apache.iotdb.db.mpp.aggregation.LastValueDescAccumulator;
import org.apache.iotdb.db.mpp.aggregation.MaxTimeDescAccumulator;
@@ -43,7 +43,7 @@ import java.util.List;
public class LastQueryUtil {
private static final boolean CACHE_ENABLED =
- IoTDBDescriptor.getInstance().getConfig().isLastCacheEnabled();
+ CommonDescriptor.getInstance().getConfig().isLastCacheEnable();
public static TsBlockBuilder createTsBlockBuilder() {
return new TsBlockBuilder(ImmutableList.of(TSDataType.TEXT,
TSDataType.TEXT, TSDataType.TEXT));
diff --git
a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/GetTimeSlotListTask.java
b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/GetTimeSlotListTask.java
index f07abe7025a..9962d01d5bf 100644
---
a/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/GetTimeSlotListTask.java
+++
b/server/src/main/java/org/apache/iotdb/db/mpp/plan/execution/config/metadata/GetTimeSlotListTask.java
@@ -20,8 +20,8 @@
package org.apache.iotdb.db.mpp.plan.execution.config.metadata;
import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.confignode.rpc.thrift.TGetTimeSlotListResp;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.mpp.common.header.ColumnHeader;
import org.apache.iotdb.db.mpp.common.header.ColumnHeaderConstant;
import org.apache.iotdb.db.mpp.common.header.DatasetHeader;
@@ -63,7 +63,7 @@ public class GetTimeSlotListTask implements IConfigTask {
.getColumnBuilder(0)
.writeLong(
timePartitionSlot.getStartTime()
- /
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval());
+ /
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval());
builder
.getColumnBuilder(1)
.writeBinary(new
Binary(DateTimeUtils.convertLongToDate(timePartitionSlot.getStartTime())));
diff --git
a/server/src/main/java/org/apache/iotdb/db/pipe/connector/v1/IoTDBThriftConnectorV1.java
b/server/src/main/java/org/apache/iotdb/db/pipe/connector/v1/IoTDBThriftConnectorV1.java
index f0c4d54d929..bb18acda62d 100644
---
a/server/src/main/java/org/apache/iotdb/db/pipe/connector/v1/IoTDBThriftConnectorV1.java
+++
b/server/src/main/java/org/apache/iotdb/db/pipe/connector/v1/IoTDBThriftConnectorV1.java
@@ -106,7 +106,8 @@ public class IoTDBThriftConnectorV1 implements
PipeConnector {
try {
final TPipeTransferResp resp =
client.pipeTransfer(
-
PipeTransferHandshakeReq.toTPipeTransferReq(IOTDB_CONFIG.getTimestampPrecision()));
+ PipeTransferHandshakeReq.toTPipeTransferReq(
+
CommonDescriptor.getInstance().getConfig().getTimestampPrecision()));
if (resp.getStatus().getCode() !=
TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
throw new PipeException(String.format("Handshake error, result status
%s.", resp.status));
}
diff --git
a/server/src/main/java/org/apache/iotdb/db/pipe/connector/v1/IoTDBThriftReceiverV1.java
b/server/src/main/java/org/apache/iotdb/db/pipe/connector/v1/IoTDBThriftReceiverV1.java
index 5ab2b19c5a7..cc803c33b95 100644
---
a/server/src/main/java/org/apache/iotdb/db/pipe/connector/v1/IoTDBThriftReceiverV1.java
+++
b/server/src/main/java/org/apache/iotdb/db/pipe/connector/v1/IoTDBThriftReceiverV1.java
@@ -20,6 +20,7 @@
package org.apache.iotdb.db.pipe.connector.v1;
import org.apache.iotdb.common.rpc.thrift.TSStatus;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.mpp.plan.Coordinator;
@@ -93,11 +94,15 @@ public class IoTDBThriftReceiverV1 implements
IoTDBThriftReceiver {
}
private TPipeTransferResp handleTransferHandshake(PipeTransferHandshakeReq
req) {
- if
(!IOTDB_CONFIG.getTimestampPrecision().equals(req.getTimestampPrecision())) {
+ if (!CommonDescriptor.getInstance()
+ .getConfig()
+ .getTimestampPrecision()
+ .equals(req.getTimestampPrecision())) {
String msg =
String.format(
"IoTDB receiver's timestamp precision %s, connector's timestamp
precision %s. validation fails.",
- IOTDB_CONFIG.getTimestampPrecision(),
req.getTimestampPrecision());
+
CommonDescriptor.getInstance().getConfig().getTimestampPrecision(),
+ req.getTimestampPrecision());
LOGGER.warn(msg);
return new
TPipeTransferResp(RpcUtils.getStatus(TSStatusCode.PIPE_HANDSHAKE_ERROR, msg));
}
diff --git
a/server/src/main/java/org/apache/iotdb/db/pipe/connector/v2/IoTDBThriftConnectorV2.java
b/server/src/main/java/org/apache/iotdb/db/pipe/connector/v2/IoTDBThriftConnectorV2.java
index 8dffd1d544c..edba5374f75 100644
---
a/server/src/main/java/org/apache/iotdb/db/pipe/connector/v2/IoTDBThriftConnectorV2.java
+++
b/server/src/main/java/org/apache/iotdb/db/pipe/connector/v2/IoTDBThriftConnectorV2.java
@@ -145,7 +145,8 @@ public class IoTDBThriftConnectorV2 implements
PipeConnector {
firstNodeUrl.getPort())) {
final TPipeTransferResp resp =
client.pipeTransfer(
-
PipeTransferHandshakeReq.toTPipeTransferReq(IOTDB_CONFIG.getTimestampPrecision()));
+ PipeTransferHandshakeReq.toTPipeTransferReq(
+
CommonDescriptor.getInstance().getConfig().getTimestampPrecision()));
if (resp.getStatus().getCode() !=
TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
throw new PipeException(String.format("Handshake error, result status
%s.", resp.status));
}
diff --git
a/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/handler/QueryHandlerFactory.java
b/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/handler/QueryHandlerFactory.java
index b944faf14ce..77c47cf57f2 100644
---
a/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/handler/QueryHandlerFactory.java
+++
b/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/handler/QueryHandlerFactory.java
@@ -18,6 +18,7 @@
*/
package org.apache.iotdb.db.protocol.influxdb.handler;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.service.thrift.impl.ClientRPCServiceImpl;
@@ -34,7 +35,7 @@ public class QueryHandlerFactory {
.getConfig()
.getRpcImplClassName()
.equals(ClientRPCServiceImpl.class.getName())) {
- if
("Tag".equals(IoTDBDescriptor.getInstance().getConfig().getSchemaEngineMode()))
{
+ if
("Tag".equals(CommonDescriptor.getInstance().getConfig().getSchemaEngineMode()))
{
return new TagQueryHandler();
}
return new NewQueryHandler();
diff --git
a/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/meta/InfluxDBMetaManagerFactory.java
b/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/meta/InfluxDBMetaManagerFactory.java
index 5efa5728ed7..106fe986d96 100644
---
a/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/meta/InfluxDBMetaManagerFactory.java
+++
b/server/src/main/java/org/apache/iotdb/db/protocol/influxdb/meta/InfluxDBMetaManagerFactory.java
@@ -18,6 +18,7 @@
*/
package org.apache.iotdb.db.protocol.influxdb.meta;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.service.thrift.impl.ClientRPCServiceImpl;
@@ -34,7 +35,7 @@ public class InfluxDBMetaManagerFactory {
.getConfig()
.getRpcImplClassName()
.equals(ClientRPCServiceImpl.class.getName())) {
- if
("Tag".equals(IoTDBDescriptor.getInstance().getConfig().getSchemaEngineMode()))
{
+ if
("Tag".equals(CommonDescriptor.getInstance().getConfig().getSchemaEngineMode()))
{
return TagInfluxDBMetaManager.getInstance();
}
return NewInfluxDBMetaManager.getInstance();
diff --git
a/server/src/main/java/org/apache/iotdb/db/protocol/rest/v1/impl/GrafanaApiServiceImpl.java
b/server/src/main/java/org/apache/iotdb/db/protocol/rest/v1/impl/GrafanaApiServiceImpl.java
index 4301457b28a..a29116a990f 100644
---
a/server/src/main/java/org/apache/iotdb/db/protocol/rest/v1/impl/GrafanaApiServiceImpl.java
+++
b/server/src/main/java/org/apache/iotdb/db/protocol/rest/v1/impl/GrafanaApiServiceImpl.java
@@ -17,6 +17,7 @@
package org.apache.iotdb.db.protocol.rest.v1.impl;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -72,7 +73,7 @@ public class GrafanaApiServiceImpl extends GrafanaApiService {
SCHEMA_FETCHER = ClusterSchemaFetcher.getInstance();
authorizationHandler = new AuthorizationHandler();
- switch (IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision())
{
+ switch
(CommonDescriptor.getInstance().getConfig().getTimestampPrecision()) {
case "ns":
timePrecision = 1000000;
break;
diff --git
a/server/src/main/java/org/apache/iotdb/db/protocol/rest/v2/impl/GrafanaApiServiceImpl.java
b/server/src/main/java/org/apache/iotdb/db/protocol/rest/v2/impl/GrafanaApiServiceImpl.java
index 1512b56a6f8..2a0a408df98 100644
---
a/server/src/main/java/org/apache/iotdb/db/protocol/rest/v2/impl/GrafanaApiServiceImpl.java
+++
b/server/src/main/java/org/apache/iotdb/db/protocol/rest/v2/impl/GrafanaApiServiceImpl.java
@@ -17,6 +17,7 @@
package org.apache.iotdb.db.protocol.rest.v2.impl;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.conf.IoTDBConfig;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
@@ -72,7 +73,7 @@ public class GrafanaApiServiceImpl extends GrafanaApiService {
SCHEMA_FETCHER = ClusterSchemaFetcher.getInstance();
authorizationHandler = new AuthorizationHandler();
- switch (IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision())
{
+ switch
(CommonDescriptor.getInstance().getConfig().getTimestampPrecision()) {
case "ns":
timePrecision = 1000000;
break;
diff --git
a/server/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java
b/server/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java
index 72b8e1344c2..27af592a5f3 100644
--- a/server/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java
+++ b/server/src/main/java/org/apache/iotdb/db/service/IoTDBShutdownHook.java
@@ -53,7 +53,7 @@ public class IoTDBShutdownHook extends Thread {
@Override
public void run() {
// close rocksdb if possible to avoid lose data
- if
(SchemaEngineMode.valueOf(IoTDBDescriptor.getInstance().getConfig().getSchemaEngineMode())
+ if
(SchemaEngineMode.valueOf(CommonDescriptor.getInstance().getConfig().getSchemaEngineMode())
.equals(SchemaEngineMode.Rocksdb_based)) {
SchemaEngine.getInstance().clear();
}
diff --git
a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/ClientRPCServiceImpl.java
b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/ClientRPCServiceImpl.java
index c1a0412e5c1..8a1f9e323d9 100644
---
a/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/ClientRPCServiceImpl.java
+++
b/server/src/main/java/org/apache/iotdb/db/service/thrift/impl/ClientRPCServiceImpl.java
@@ -735,7 +735,7 @@ public class ClientRPCServiceImpl implements
IClientRPCServiceWithHandler {
properties.getSupportedTimeAggregationOperations().add(IoTDBConstant.MAX_TIME);
properties.getSupportedTimeAggregationOperations().add(IoTDBConstant.MIN_TIME);
properties.setTimestampPrecision(
- IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision());
+ CommonDescriptor.getInstance().getConfig().getTimestampPrecision());
properties.setMaxConcurrentClientNum(
IoTDBDescriptor.getInstance().getConfig().getRpcMaxConcurrentClientNum());
properties.setIsReadOnly(CommonDescriptor.getInstance().getConfig().isReadOnly());
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/DateTimeUtils.java
b/server/src/main/java/org/apache/iotdb/db/utils/DateTimeUtils.java
index c7df46fbe94..e056f6138eb 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/DateTimeUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/DateTimeUtils.java
@@ -18,6 +18,7 @@
*/
package org.apache.iotdb.db.utils;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.utils.TestOnly;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.query.control.SessionManager;
@@ -454,7 +455,7 @@ public class DateTimeUtils {
str,
toZoneOffset(zoneId),
0,
- IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision());
+ CommonDescriptor.getInstance().getConfig().getTimestampPrecision());
}
public static long convertDatetimeStrToLong(
@@ -532,7 +533,7 @@ public class DateTimeUtils {
public static long convertDurationStrToLong(long currentTime, String
duration) {
return convertDurationStrToLong(
- currentTime, duration,
IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision());
+ currentTime, duration,
CommonDescriptor.getInstance().getConfig().getTimestampPrecision());
}
/**
@@ -653,7 +654,7 @@ public class DateTimeUtils {
public static long currentTime() {
long startupNano =
IoTDBDescriptor.getInstance().getConfig().getStartUpNanosecond();
- String timePrecision =
IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision();
+ String timePrecision =
CommonDescriptor.getInstance().getConfig().getTimestampPrecision();
switch (timePrecision) {
case "ns":
return System.currentTimeMillis() * 1000_000 + (System.nanoTime() -
startupNano) % 1000_000;
@@ -665,7 +666,7 @@ public class DateTimeUtils {
}
public static String convertLongToDate(long timestamp) {
- String timePrecision =
IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision();
+ String timePrecision =
CommonDescriptor.getInstance().getConfig().getTimestampPrecision();
switch (timePrecision) {
case "ns":
timestamp /= 1000_000;
@@ -686,21 +687,6 @@ public class DateTimeUtils {
return ZonedDateTime.ofInstant(Instant.ofEpochMilli(millisecond),
ZoneId.systemDefault());
}
- public static long convertMilliTimeWithPrecision(long milliTime, String
timePrecision) {
- long result = milliTime;
- switch (timePrecision) {
- case "ns":
- result = milliTime * 1000_000L;
- break;
- case "us":
- result = milliTime * 1000L;
- break;
- default:
- break;
- }
- return result;
- }
-
public enum DurationUnit {
y,
mo,
diff --git
a/server/src/main/java/org/apache/iotdb/db/utils/TimePartitionUtils.java
b/server/src/main/java/org/apache/iotdb/db/utils/TimePartitionUtils.java
index 763b7e0d5ab..058ae2fe6a1 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/TimePartitionUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/TimePartitionUtils.java
@@ -19,14 +19,14 @@
package org.apache.iotdb.db.utils;
import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.utils.TestOnly;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.conf.ServerConfigConsistent;
public class TimePartitionUtils {
@ServerConfigConsistent
public static long timePartitionInterval =
- IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval();
+ CommonDescriptor.getInstance().getConfig().getTimePartitionInterval();
public static TTimePartitionSlot getTimePartition(long time) {
TTimePartitionSlot timePartitionSlot = new TTimePartitionSlot();
diff --git
a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/sizetiered/SizeTieredCompactionSelectorTest.java
b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/sizetiered/SizeTieredCompactionSelectorTest.java
index 5cd387cefb9..a4e42bf94c8 100644
---
a/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/sizetiered/SizeTieredCompactionSelectorTest.java
+++
b/server/src/test/java/org/apache/iotdb/db/engine/compaction/inner/sizetiered/SizeTieredCompactionSelectorTest.java
@@ -19,6 +19,7 @@
package org.apache.iotdb.db.engine.compaction.inner.sizetiered;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.engine.compaction.schedule.CompactionTaskManager;
import
org.apache.iotdb.db.engine.compaction.selector.impl.SizeTieredCompactionSelector;
@@ -40,8 +41,8 @@ public class SizeTieredCompactionSelectorTest {
@Test
public void testSubmitWhenNextTimePartitionExists() {
long originPartitionInterval =
- IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval();
-
IoTDBDescriptor.getInstance().getConfig().setTimePartitionInterval(1000000);
+ CommonDescriptor.getInstance().getConfig().getTimePartitionInterval();
+
CommonDescriptor.getInstance().getConfig().setTimePartitionInterval(1000000);
List<TsFileResource> resources = new ArrayList<>();
for (int i = 0; i < 100; ++i) {
diff --git
a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/DataRegionTest.java
b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/DataRegionTest.java
index 01f311d1337..0088a8b2ec9 100644
---
a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/DataRegionTest.java
+++
b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/DataRegionTest.java
@@ -19,6 +19,7 @@
package org.apache.iotdb.db.engine.storagegroup;
+import org.apache.iotdb.commons.conf.CommonConfig;
import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.consensus.DataRegionId;
import org.apache.iotdb.commons.exception.IllegalPathException;
@@ -75,6 +76,7 @@ import java.util.concurrent.atomic.AtomicInteger;
public class DataRegionTest {
private static final IoTDBConfig config =
IoTDBDescriptor.getInstance().getConfig();
+ private static final CommonConfig COMMON_CONFIG =
CommonDescriptor.getInstance().getConfig();
private static final Logger logger =
LoggerFactory.getLogger(DataRegionTest.class);
private String storageGroup = "root.vehicle.d0";
@@ -413,9 +415,9 @@ public class DataRegionTest {
public void testEnableDiscardOutOfOrderDataForInsertTablet1()
throws QueryProcessException, IllegalPathException, IOException,
WriteProcessException {
boolean defaultEnableDiscard = config.isEnableDiscardOutOfOrderData();
- long defaultTimePartition = config.getTimePartitionInterval();
+ long defaultTimePartition = COMMON_CONFIG.getTimePartitionInterval();
config.setEnableDiscardOutOfOrderData(true);
- config.setTimePartitionInterval(100000);
+ COMMON_CONFIG.setTimePartitionInterval(100000);
String[] measurements = new String[2];
measurements[0] = "s0";
@@ -494,16 +496,16 @@ public class DataRegionTest {
}
config.setEnableDiscardOutOfOrderData(defaultEnableDiscard);
- config.setTimePartitionInterval(defaultTimePartition);
+ COMMON_CONFIG.setTimePartitionInterval(defaultTimePartition);
}
@Test
public void testEnableDiscardOutOfOrderDataForInsertTablet2()
throws QueryProcessException, IllegalPathException, IOException,
WriteProcessException {
boolean defaultEnableDiscard = config.isEnableDiscardOutOfOrderData();
- long defaultTimePartition = config.getTimePartitionInterval();
+ long defaultTimePartition = COMMON_CONFIG.getTimePartitionInterval();
config.setEnableDiscardOutOfOrderData(true);
- config.setTimePartitionInterval(1200000);
+ COMMON_CONFIG.setTimePartitionInterval(1200000);
String[] measurements = new String[2];
measurements[0] = "s0";
@@ -582,16 +584,16 @@ public class DataRegionTest {
}
config.setEnableDiscardOutOfOrderData(defaultEnableDiscard);
- config.setTimePartitionInterval(defaultTimePartition);
+ COMMON_CONFIG.setTimePartitionInterval(defaultTimePartition);
}
@Test
public void testEnableDiscardOutOfOrderDataForInsertTablet3()
throws QueryProcessException, IllegalPathException, IOException,
WriteProcessException {
boolean defaultEnableDiscard = config.isEnableDiscardOutOfOrderData();
- long defaultTimePartition = config.getTimePartitionInterval();
+ long defaultTimePartition = COMMON_CONFIG.getTimePartitionInterval();
config.setEnableDiscardOutOfOrderData(true);
- config.setTimePartitionInterval(1000000);
+ COMMON_CONFIG.setTimePartitionInterval(1000000);
String[] measurements = new String[2];
measurements[0] = "s0";
@@ -670,7 +672,7 @@ public class DataRegionTest {
}
config.setEnableDiscardOutOfOrderData(defaultEnableDiscard);
- config.setTimePartitionInterval(defaultTimePartition);
+ COMMON_CONFIG.setTimePartitionInterval(defaultTimePartition);
}
@Test
diff --git
a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java
b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java
index c2392bff9ec..8b34eab75b5 100644
--- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java
@@ -20,6 +20,7 @@
package org.apache.iotdb.db.engine.storagegroup;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.conf.IoTDBConstant;
import org.apache.iotdb.commons.consensus.DataRegionId;
import org.apache.iotdb.commons.exception.IllegalPathException;
@@ -82,8 +83,8 @@ public class TTLTest {
@Before
public void setUp() throws MetadataException, DataRegionException {
- prevPartitionInterval =
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval();
-
IoTDBDescriptor.getInstance().getConfig().setTimePartitionInterval(86400000);
+ prevPartitionInterval =
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval();
+
CommonDescriptor.getInstance().getConfig().setTimePartitionInterval(86400000);
EnvironmentUtils.envSetUp();
dataRegion =
new DataRegion(
@@ -98,7 +99,7 @@ public class TTLTest {
public void tearDown() throws IOException, StorageEngineException {
dataRegion.syncCloseAllWorkingTsFileProcessors();
EnvironmentUtils.cleanEnv();
-
IoTDBDescriptor.getInstance().getConfig().setTimePartitionInterval(prevPartitionInterval);
+
CommonDescriptor.getInstance().getConfig().setTimePartitionInterval(prevPartitionInterval);
}
@Test
diff --git
a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/AliasIndexPageTest.java
b/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/AliasIndexPageTest.java
index 32aba0b4c29..5a1f992c026 100644
---
a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/AliasIndexPageTest.java
+++
b/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/AliasIndexPageTest.java
@@ -18,8 +18,8 @@
*/
package org.apache.iotdb.db.metadata.mtree.schemafile;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.metadata.mtree.store.disk.schemafile.ISchemaPage;
import org.apache.iotdb.db.metadata.mtree.store.disk.schemafile.ISegment;
import
org.apache.iotdb.db.metadata.mtree.store.disk.schemafile.SchemaFileConfig;
@@ -36,7 +36,7 @@ import java.nio.ByteBuffer;
public class AliasIndexPageTest {
@Before
public void setUp() {
- IoTDBDescriptor.getInstance()
+ CommonDescriptor.getInstance()
.getConfig()
.setSchemaEngineMode(SchemaEngineMode.PB_Tree.toString());
EnvironmentUtils.envSetUp();
@@ -45,7 +45,7 @@ public class AliasIndexPageTest {
@After
public void tearDown() throws Exception {
EnvironmentUtils.cleanEnv();
- IoTDBDescriptor.getInstance()
+ CommonDescriptor.getInstance()
.getConfig()
.setSchemaEngineMode(SchemaEngineMode.Memory.toString());
}
diff --git
a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/InternalPageTest.java
b/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/InternalPageTest.java
index 6a6b0a1449d..5f145b6f3fa 100644
---
a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/InternalPageTest.java
+++
b/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/InternalPageTest.java
@@ -18,8 +18,8 @@
*/
package org.apache.iotdb.db.metadata.mtree.schemafile;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.exception.MetadataException;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.metadata.mtree.store.disk.schemafile.ISchemaPage;
import org.apache.iotdb.db.metadata.mtree.store.disk.schemafile.ISegment;
import
org.apache.iotdb.db.metadata.mtree.store.disk.schemafile.SchemaFileConfig;
@@ -36,7 +36,7 @@ import java.nio.ByteBuffer;
public class InternalPageTest {
@Before
public void setUp() {
- IoTDBDescriptor.getInstance()
+ CommonDescriptor.getInstance()
.getConfig()
.setSchemaEngineMode(SchemaEngineMode.PB_Tree.toString());
EnvironmentUtils.envSetUp();
@@ -45,7 +45,7 @@ public class InternalPageTest {
@After
public void tearDown() throws Exception {
EnvironmentUtils.cleanEnv();
- IoTDBDescriptor.getInstance()
+ CommonDescriptor.getInstance()
.getConfig()
.setSchemaEngineMode(SchemaEngineMode.Memory.toString());
}
diff --git
a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/SchemaFileLogTest.java
b/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/SchemaFileLogTest.java
index fbbd7ab3f89..bc822554012 100644
---
a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/SchemaFileLogTest.java
+++
b/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/SchemaFileLogTest.java
@@ -18,10 +18,10 @@
*/
package org.apache.iotdb.db.metadata.mtree.schemafile;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.metadata.MetadataConstant;
import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
import org.apache.iotdb.db.metadata.mnode.schemafile.factory.CacheMNodeFactory;
@@ -56,7 +56,7 @@ public class SchemaFileLogTest {
@Before
public void setUp() {
- IoTDBDescriptor.getInstance()
+ CommonDescriptor.getInstance()
.getConfig()
.setSchemaEngineMode(SchemaEngineMode.PB_Tree.toString());
EnvironmentUtils.envSetUp();
@@ -65,7 +65,7 @@ public class SchemaFileLogTest {
@After
public void tearDown() throws Exception {
EnvironmentUtils.cleanEnv();
- IoTDBDescriptor.getInstance()
+ CommonDescriptor.getInstance()
.getConfig()
.setSchemaEngineMode(SchemaEngineMode.Memory.toString());
}
diff --git
a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/SchemaFileTest.java
b/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/SchemaFileTest.java
index d9ac2c84628..38786528b8c 100644
---
a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/SchemaFileTest.java
+++
b/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/SchemaFileTest.java
@@ -18,13 +18,13 @@
*/
package org.apache.iotdb.db.metadata.mtree.schemafile;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.schema.node.IMNode;
import org.apache.iotdb.commons.schema.node.role.IDatabaseMNode;
import org.apache.iotdb.commons.schema.node.role.IMeasurementMNode;
import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory;
import org.apache.iotdb.commons.utils.PathUtils;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
import
org.apache.iotdb.db.metadata.mnode.schemafile.container.ICachedMNodeContainer;
import org.apache.iotdb.db.metadata.mnode.schemafile.factory.CacheMNodeFactory;
@@ -66,7 +66,7 @@ public class SchemaFileTest {
@Before
public void setUp() {
- IoTDBDescriptor.getInstance()
+ CommonDescriptor.getInstance()
.getConfig()
.setSchemaEngineMode(SchemaEngineMode.PB_Tree.toString());
EnvironmentUtils.envSetUp();
@@ -75,7 +75,7 @@ public class SchemaFileTest {
@After
public void tearDown() throws Exception {
EnvironmentUtils.cleanEnv();
- IoTDBDescriptor.getInstance()
+ CommonDescriptor.getInstance()
.getConfig()
.setSchemaEngineMode(SchemaEngineMode.Memory.toString());
}
diff --git
a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/WrappedSegmentTest.java
b/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/WrappedSegmentTest.java
index daf2ff3ad35..be5c5ef15bd 100644
---
a/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/WrappedSegmentTest.java
+++
b/server/src/test/java/org/apache/iotdb/db/metadata/mtree/schemafile/WrappedSegmentTest.java
@@ -18,9 +18,9 @@
*/
package org.apache.iotdb.db.metadata.mtree.schemafile;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
import org.apache.iotdb.db.metadata.mnode.schemafile.factory.CacheMNodeFactory;
import org.apache.iotdb.db.metadata.mtree.store.disk.schemafile.ISegment;
@@ -46,7 +46,7 @@ public class WrappedSegmentTest {
@Before
public void setUp() {
- IoTDBDescriptor.getInstance()
+ CommonDescriptor.getInstance()
.getConfig()
.setSchemaEngineMode(SchemaEngineMode.PB_Tree.toString());
EnvironmentUtils.envSetUp();
@@ -55,7 +55,7 @@ public class WrappedSegmentTest {
@After
public void tearDown() throws Exception {
EnvironmentUtils.cleanEnv();
- IoTDBDescriptor.getInstance()
+ CommonDescriptor.getInstance()
.getConfig()
.setSchemaEngineMode(SchemaEngineMode.Memory.toString());
}
diff --git
a/server/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/AbstractSchemaRegionTest.java
b/server/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/AbstractSchemaRegionTest.java
index 11b61d494ae..451277f8335 100644
---
a/server/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/AbstractSchemaRegionTest.java
+++
b/server/src/test/java/org/apache/iotdb/db/metadata/schemaRegion/AbstractSchemaRegionTest.java
@@ -19,6 +19,8 @@
package org.apache.iotdb.db.metadata.schemaRegion;
+import org.apache.iotdb.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.consensus.SchemaRegionId;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.db.conf.IoTDBConfig;
@@ -41,6 +43,7 @@ import java.util.List;
public abstract class AbstractSchemaRegionTest {
private static final IoTDBConfig config =
IoTDBDescriptor.getInstance().getConfig();
+ private static final CommonConfig COMMON_CONFIG =
CommonDescriptor.getInstance().getConfig();
private SchemaRegionTestParams rawConfig;
@@ -64,10 +67,10 @@ public abstract class AbstractSchemaRegionTest {
rawConfig =
new SchemaRegionTestParams(
"Raw-Config",
- config.getSchemaEngineMode(),
+ COMMON_CONFIG.getSchemaEngineMode(),
config.getCachedMNodeSizeInPBTreeMode(),
config.isClusterMode());
- config.setSchemaEngineMode(testParams.schemaEngineMode);
+ COMMON_CONFIG.setSchemaEngineMode(testParams.schemaEngineMode);
config.setCachedMNodeSizeInPBTreeMode(testParams.cachedMNodeSize);
config.setClusterMode(testParams.isClusterMode);
SchemaEngine.getInstance().init();
@@ -77,7 +80,7 @@ public abstract class AbstractSchemaRegionTest {
public void tearDown() throws Exception {
SchemaEngine.getInstance().clear();
cleanEnv();
- config.setSchemaEngineMode(rawConfig.schemaEngineMode);
+ COMMON_CONFIG.setSchemaEngineMode(rawConfig.schemaEngineMode);
config.setCachedMNodeSizeInPBTreeMode(rawConfig.cachedMNodeSize);
config.setClusterMode(rawConfig.isClusterMode);
}
diff --git
a/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/QueryTimePartitionTest.java
b/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/QueryTimePartitionTest.java
index c7951571a0a..a97026974f6 100644
---
a/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/QueryTimePartitionTest.java
+++
b/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/QueryTimePartitionTest.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.mpp.plan.analyze;
import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.tsfile.read.common.TimeRange;
import org.apache.iotdb.tsfile.read.filter.TimeFilter;
import org.apache.iotdb.tsfile.read.filter.operator.AndFilter;
@@ -397,16 +397,17 @@ public class QueryTimePartitionTest {
new AndFilter(
TimeFilter.gt(0),
TimeFilter.ltEq(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() * 3 + 1)));
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() * 3
+ + 1)));
expected =
Arrays.asList(
new TTimePartitionSlot(0),
new TTimePartitionSlot(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval()),
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval()),
new TTimePartitionSlot(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() * 2),
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() * 2),
new TTimePartitionSlot(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() * 3));
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() * 3));
assertEquals(expected.size(), res.left.size());
for (int i = 0; i < expected.size(); i++) {
assertEquals(expected.get(i), res.left.get(i));
@@ -420,14 +421,14 @@ public class QueryTimePartitionTest {
getTimePartitionSlotList(
new AndFilter(
TimeFilter.gtEq(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() - 1),
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() - 1),
TimeFilter.lt(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() + 1)));
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() + 1)));
expected =
Arrays.asList(
new TTimePartitionSlot(0),
new TTimePartitionSlot(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval()));
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval()));
assertEquals(expected.size(), res.left.size());
for (int i = 0; i < expected.size(); i++) {
assertEquals(expected.get(i), res.left.get(i));
@@ -440,13 +441,13 @@ public class QueryTimePartitionTest {
res =
getTimePartitionSlotList(
TimeFilter.between(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() - 1,
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval()));
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() - 1,
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval()));
expected =
Arrays.asList(
new TTimePartitionSlot(0),
new TTimePartitionSlot(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval()));
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval()));
assertEquals(expected.size(), res.left.size());
for (int i = 0; i < expected.size(); i++) {
assertEquals(expected.get(i), res.left.get(i));
@@ -460,13 +461,13 @@ public class QueryTimePartitionTest {
getTimePartitionSlotList(
new AndFilter(
TimeFilter.gtEq(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval()),
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval()),
TimeFilter.ltEq(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() + 1)));
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() + 1)));
expected =
Collections.singletonList(
new TTimePartitionSlot(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval()));
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval()));
assertEquals(expected.size(), res.left.size());
for (int i = 0; i < expected.size(); i++) {
assertEquals(expected.get(i), res.left.get(i));
@@ -479,12 +480,12 @@ public class QueryTimePartitionTest {
res =
getTimePartitionSlotList(
TimeFilter.between(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval(),
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() + 1));
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval(),
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() + 1));
expected =
Collections.singletonList(
new TTimePartitionSlot(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval()));
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval()));
assertEquals(expected.size(), res.left.size());
for (int i = 0; i < expected.size(); i++) {
assertEquals(expected.get(i), res.left.get(i));
@@ -507,38 +508,39 @@ public class QueryTimePartitionTest {
new AndFilter(
TimeFilter.gtEq(10),
TimeFilter.lt(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval())),
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval())),
new AndFilter(
-
TimeFilter.gt(IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval()),
+ TimeFilter.gt(
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval()),
TimeFilter.lt(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() * 2
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() * 2
- 100)));
OrFilter orFilter2 =
new OrFilter(
orFilter1,
new AndFilter(
TimeFilter.gt(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() * 2 - 50),
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() * 2 - 50),
TimeFilter.ltEq(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() * 2
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() * 2
- 40)));
OrFilter orFilter3 =
new OrFilter(
orFilter2,
new AndFilter(
TimeFilter.gt(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() * 2 - 20),
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() * 2 - 20),
TimeFilter.ltEq(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() * 3
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() * 3
+ 10)));
OrFilter orFilter4 =
new OrFilter(
orFilter3,
new AndFilter(
TimeFilter.gt(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() * 5 + 1),
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() * 5 + 1),
TimeFilter.lt(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() * 5
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() * 5
+ 10)));
res = getTimePartitionSlotList(orFilter4);
@@ -546,13 +548,13 @@ public class QueryTimePartitionTest {
Arrays.asList(
new TTimePartitionSlot(0),
new TTimePartitionSlot(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval()),
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval()),
new TTimePartitionSlot(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() * 2),
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() * 2),
new TTimePartitionSlot(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() * 3),
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() * 3),
new TTimePartitionSlot(
-
IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval() * 5));
+
CommonDescriptor.getInstance().getConfig().getTimePartitionInterval() * 5));
assertEquals(expected.size(), res.left.size());
for (int i = 0; i < expected.size(); i++) {
assertEquals(expected.get(i), res.left.get(i));
diff --git
a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/write/WritePlanNodeSplitTest.java
b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/write/WritePlanNodeSplitTest.java
index 76b5ced8eb8..b00c7544566 100644
---
a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/write/WritePlanNodeSplitTest.java
+++
b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/node/write/WritePlanNodeSplitTest.java
@@ -26,6 +26,7 @@ import org.apache.iotdb.common.rpc.thrift.TEndPoint;
import org.apache.iotdb.common.rpc.thrift.TRegionReplicaSet;
import org.apache.iotdb.common.rpc.thrift.TSeriesPartitionSlot;
import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.partition.DataPartition;
import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
@@ -71,8 +72,8 @@ public class WritePlanNodeSplitTest {
@Before
public void setUp() {
prevTimePartitionInterval =
- IoTDBDescriptor.getInstance().getConfig().getTimePartitionInterval();
- IoTDBDescriptor.getInstance().getConfig().setTimePartitionInterval(100);
+ CommonDescriptor.getInstance().getConfig().getTimePartitionInterval();
+ CommonDescriptor.getInstance().getConfig().setTimePartitionInterval(100);
TimePartitionUtils.setTimePartitionInterval(100);
executorClassName =
IoTDBDescriptor.getInstance().getConfig().getSeriesPartitionExecutorClass();
@@ -316,6 +317,6 @@ public class WritePlanNodeSplitTest {
@After
public void tearDown() {
TimePartitionUtils.setTimePartitionInterval(prevTimePartitionInterval);
-
IoTDBDescriptor.getInstance().getConfig().setTimePartitionInterval(prevTimePartitionInterval);
+
CommonDescriptor.getInstance().getConfig().setTimePartitionInterval(prevTimePartitionInterval);
}
}
diff --git
a/server/src/test/java/org/apache/iotdb/db/qp/sql/ASTVisitorTest.java
b/server/src/test/java/org/apache/iotdb/db/qp/sql/ASTVisitorTest.java
index 10770eb2044..539a99c48b0 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/sql/ASTVisitorTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/sql/ASTVisitorTest.java
@@ -19,7 +19,7 @@
package org.apache.iotdb.db.qp.sql;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.db.constant.SqlConstant;
import org.apache.iotdb.db.exception.sql.SemanticException;
import org.apache.iotdb.db.mpp.plan.parser.ASTVisitor;
@@ -71,22 +71,22 @@ public class ASTVisitorTest {
@Test
public void testParseTimeFormatNowPrecision() {
- String timePrecision =
IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision();
- IoTDBDescriptor.getInstance().getConfig().setTimestampPrecision("ms");
+ String timePrecision =
CommonDescriptor.getInstance().getConfig().getTimestampPrecision();
+ CommonDescriptor.getInstance().getConfig().setTimestampPrecision("ms");
long now_ms = visitor.parseDateFormat(SqlConstant.NOW_FUNC);
String ms_str = String.valueOf(now_ms);
- IoTDBDescriptor.getInstance().getConfig().setTimestampPrecision("us");
+ CommonDescriptor.getInstance().getConfig().setTimestampPrecision("us");
long now_us = visitor.parseDateFormat(SqlConstant.NOW_FUNC);
String us_str = String.valueOf(now_us);
- IoTDBDescriptor.getInstance().getConfig().setTimestampPrecision("ns");
+ CommonDescriptor.getInstance().getConfig().setTimestampPrecision("ns");
long now_ns = visitor.parseDateFormat(SqlConstant.NOW_FUNC);
String ns_str = String.valueOf(now_ns);
assertEquals(ms_str.length() + 3, (us_str).length());
assertEquals(us_str.length() + 3, (ns_str).length());
-
IoTDBDescriptor.getInstance().getConfig().setTimestampPrecision(timePrecision);
+
CommonDescriptor.getInstance().getConfig().setTimestampPrecision(timePrecision);
}
@Test(expected = SemanticException.class)
diff --git
a/server/src/test/java/org/apache/iotdb/db/tools/PBTreeFileSketchTest.java
b/server/src/test/java/org/apache/iotdb/db/tools/PBTreeFileSketchTest.java
index 1a6140208bf..e5fd2cd1e56 100644
--- a/server/src/test/java/org/apache/iotdb/db/tools/PBTreeFileSketchTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/tools/PBTreeFileSketchTest.java
@@ -19,9 +19,9 @@
package org.apache.iotdb.db.tools;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.exception.MetadataException;
import org.apache.iotdb.commons.schema.node.utils.IMNodeFactory;
-import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.metadata.MetadataConstant;
import org.apache.iotdb.db.metadata.mnode.schemafile.ICachedMNode;
import org.apache.iotdb.db.metadata.mnode.schemafile.factory.CacheMNodeFactory;
@@ -55,7 +55,7 @@ public class PBTreeFileSketchTest {
@Before
public void setUp() {
- IoTDBDescriptor.getInstance()
+ CommonDescriptor.getInstance()
.getConfig()
.setSchemaEngineMode(SchemaEngineMode.PB_Tree.toString());
EnvironmentUtils.envSetUp();
@@ -66,7 +66,7 @@ public class PBTreeFileSketchTest {
File sketch = new File("sketch_schemafile.txt");
sketch.deleteOnExit();
EnvironmentUtils.cleanEnv();
- IoTDBDescriptor.getInstance()
+ CommonDescriptor.getInstance()
.getConfig()
.setSchemaEngineMode(SchemaEngineMode.Memory.toString());
}
diff --git
a/server/src/test/java/org/apache/iotdb/db/tools/TsFileAndModSettleToolTest.java
b/server/src/test/java/org/apache/iotdb/db/tools/TsFileAndModSettleToolTest.java
index e7b3ddb5e11..9dc8066b786 100644
---
a/server/src/test/java/org/apache/iotdb/db/tools/TsFileAndModSettleToolTest.java
+++
b/server/src/test/java/org/apache/iotdb/db/tools/TsFileAndModSettleToolTest.java
@@ -18,6 +18,8 @@
*/
package org.apache.iotdb.db.tools;
+import org.apache.iotdb.commons.conf.CommonConfig;
+import org.apache.iotdb.commons.conf.CommonDescriptor;
import org.apache.iotdb.commons.exception.IllegalPathException;
import org.apache.iotdb.commons.path.PartialPath;
import org.apache.iotdb.commons.utils.FileUtils;
@@ -65,14 +67,14 @@ public class TsFileAndModSettleToolTest {
protected final String SENSOR2 = "sensor_2";
private final long VALUE_OFFSET = 1;
private String path = null;
- private IoTDBConfig config;
+ private final IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+ private static final CommonConfig COMMON_CONFIG =
CommonDescriptor.getInstance().getConfig();
private long originPartitionInterval;
@Before
public void setUp() {
- config = IoTDBDescriptor.getInstance().getConfig();
- originPartitionInterval = config.getTimePartitionInterval();
- config.setTimePartitionInterval(newPartitionInterval);
+ originPartitionInterval = COMMON_CONFIG.getTimePartitionInterval();
+ COMMON_CONFIG.setTimePartitionInterval(newPartitionInterval);
EnvironmentUtils.envSetUp();
File f = new File(folder);
@@ -100,7 +102,7 @@ public class TsFileAndModSettleToolTest {
} catch (Exception e) {
Assert.fail(e.getMessage());
} finally {
- config.setTimePartitionInterval(originPartitionInterval);
+ COMMON_CONFIG.setTimePartitionInterval(originPartitionInterval);
}
}