You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by lt...@apache.org on 2019/11/08 06:59:08 UTC

[incubator-iotdb] branch add_hot_load_configuration created (now cba5533)

This is an automated email from the ASF dual-hosted git repository.

lta pushed a change to branch add_hot_load_configuration
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git.


      at cba5533  add load configuration in anltr

This branch includes the following new commits:

     new bc27c55  add hot load config
     new a2dc165  Merge branch 'master' into add_hot_load_configuration
     new cba5533  add load configuration in anltr

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[incubator-iotdb] 03/03: add load configuration in anltr

Posted by lt...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lta pushed a commit to branch add_hot_load_configuration
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git

commit cba5533720e0c37242ba3e52a62aaa29ffd63656
Author: lta <li...@163.com>
AuthorDate: Fri Nov 8 14:21:20 2019 +0800

    add load configuration in anltr
---
 .../org/apache/iotdb/db/sql/parse/TqlLexer.g       |  4 ++
 .../org/apache/iotdb/db/sql/parse/TqlParser.g      | 34 ++++++++++-------
 .../org/apache/iotdb/db/qp/QueryProcessor.java     | 34 ++++++-----------
 .../apache/iotdb/db/qp/constant/SQLConstant.java   |  3 ++
 .../iotdb/db/qp/executor/QueryProcessExecutor.java | 31 ++++++++++------
 .../org/apache/iotdb/db/qp/logical/Operator.java   |  2 +-
 .../qp/logical/sys/LoadConfigurationOperator.java  | 31 ++++++++++++++++
 .../db/qp/physical/sys/LoadConfigurationPlan.java  | 43 ++++++++++++++++++++++
 .../iotdb/db/qp/strategy/LogicalGenerator.java     |  8 +++-
 .../iotdb/db/qp/strategy/PhysicalGenerator.java    | 20 ++++++----
 .../apache/iotdb/db/qp/plan/PhysicalPlanTest.java  | 10 +++++
 11 files changed, 161 insertions(+), 59 deletions(-)

diff --git a/server/src/main/antlr3/org/apache/iotdb/db/sql/parse/TqlLexer.g b/server/src/main/antlr3/org/apache/iotdb/db/sql/parse/TqlLexer.g
index c49fb6f..6a00dde 100644
--- a/server/src/main/antlr3/org/apache/iotdb/db/sql/parse/TqlLexer.g
+++ b/server/src/main/antlr3/org/apache/iotdb/db/sql/parse/TqlLexer.g
@@ -273,6 +273,10 @@ K_UNSET
     : U N S E T
     ;
 
+K_CONFIGURATION
+    : C O N F I G U R A T I O N
+    ;
+
 //************** logical operator***********
 OPERATOR_AND
     : A N D
diff --git a/server/src/main/antlr3/org/apache/iotdb/db/sql/parse/TqlParser.g b/server/src/main/antlr3/org/apache/iotdb/db/sql/parse/TqlParser.g
index 734adc2..9f420c4 100644
--- a/server/src/main/antlr3/org/apache/iotdb/db/sql/parse/TqlParser.g
+++ b/server/src/main/antlr3/org/apache/iotdb/db/sql/parse/TqlParser.g
@@ -107,6 +107,7 @@ tokens{
     TOK_SHOW;
     TOK_DATE_EXPR;
     TOK_DURATION;
+    TOK_LOAD_CONFIGURATION;
 }
 
 @header{
@@ -121,13 +122,13 @@ ArrayList<ParseError> errors = new ArrayList<ParseError>();
 Stack messages = new Stack<String>();
 private static HashMap<String, String> tokenNameMap;
 static {
-    tokenNameMap = new HashMap<String, String>();
-    tokenNameMap.put("K_AND", "AND");
-    tokenNameMap.put("K_OR", "OR");
-    tokenNameMap.put("K_NOT", "NOT");
-    tokenNameMap.put("K_LIKE", "LIKE");
-    tokenNameMap.put("K_BY", "BY");
-    tokenNameMap.put("K_GROUP", "GROUP");
+  tokenNameMap = new HashMap<String, String>();
+  tokenNameMap.put("K_AND", "AND");
+  tokenNameMap.put("K_OR", "OR");
+  tokenNameMap.put("K_NOT", "NOT");
+  tokenNameMap.put("K_LIKE", "LIKE");
+  tokenNameMap.put("K_BY", "BY");
+  tokenNameMap.put("K_GROUP", "GROUP");
 	tokenNameMap.put("K_FILL", "FILL");
 	tokenNameMap.put("K_LINEAR", "LINEAR");
 	tokenNameMap.put("K_PREVIOUS", "PREVIOUS");
@@ -274,6 +275,7 @@ sqlStatement
     : ddlStatement
     | dmlStatement
     | administrationStatement
+    | configurationStatement
     ;
 
 dmlStatement
@@ -754,6 +756,15 @@ rootOrId
     | ID
     ;
 
+configurationStatement
+    : loadConfigurationStatement
+    ;
+
+loadConfigurationStatement
+    : K_LOAD K_CONFIGURATION
+    -> ^(TOK_LOAD_CONFIGURATION)
+    ;
+
 /*
 ****
 *************
@@ -763,21 +774,18 @@ TTL
 */
 
 ttlStatement
-    :
-    setTTLStatement
+    : setTTLStatement
     | unsetTTLStatement
     | showTTLStatement
     ;
 
 setTTLStatement
-    :
-    K_SET K_TTL K_TO path=prefixPath time=INT
+    : K_SET K_TTL K_TO path=prefixPath time=INT
     -> ^(TOK_TTL TOK_SET $path $time)
     ;
 
 unsetTTLStatement
-    :
-     K_UNSET K_TTL K_TO path=prefixPath
+    : K_UNSET K_TTL K_TO path=prefixPath
     -> ^(TOK_TTL TOK_UNSET $path)
     ;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/QueryProcessor.java b/server/src/main/java/org/apache/iotdb/db/qp/QueryProcessor.java
index 4feb85d..c39d44a 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/QueryProcessor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/QueryProcessor.java
@@ -18,9 +18,9 @@
  */
 package org.apache.iotdb.db.qp;
 
+import java.time.ZoneId;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.exception.ArgsErrorException;
 import org.apache.iotdb.db.exception.MetadataErrorException;
 import org.apache.iotdb.db.exception.qp.IllegalASTFormatException;
 import org.apache.iotdb.db.exception.qp.LogicalOperatorException;
@@ -43,8 +43,6 @@ import org.apache.iotdb.db.sql.parse.AstNode;
 import org.apache.iotdb.db.sql.parse.ParseException;
 import org.apache.iotdb.db.sql.parse.ParseUtils;
 
-import java.time.ZoneId;
-
 /**
  * provide a integration method for other user.
  */
@@ -61,8 +59,7 @@ public class QueryProcessor {
   }
 
   public PhysicalPlan parseSQLToPhysicalPlan(String sqlStr)
-      throws QueryProcessorException, ArgsErrorException,
-      MetadataErrorException {
+      throws QueryProcessorException, MetadataErrorException {
     IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
     return parseSQLToPhysicalPlan(sqlStr, config.getZoneID());
   }
@@ -79,11 +76,9 @@ public class QueryProcessor {
   /**
    * Convert ast tree to Operator which type maybe {@code SFWOperator} or {@code AuthorOperator}
    *
-   * @param astNode
-   *            - input ast tree
+   * @param astNode - input ast tree
    * @return - RootOperator has four subclass:Query/Insert/Delete/Update/Author
-   * @throws QueryProcessorException
-   *             exception in converting sql to operator
+   * @throws QueryProcessorException exception in converting sql to operator
    */
   private RootOperator parseASTToOperator(AstNode astNode, ZoneId zoneId)
       throws QueryProcessorException, MetadataErrorException {
@@ -94,11 +89,9 @@ public class QueryProcessor {
   /**
    * Given a SQL statement and generate an ast tree
    *
-   * @param sqlStr
-   *            input sql command
+   * @param sqlStr input sql command
    * @return ast tree
-   * @throws IllegalASTFormatException
-   *             exception in sql parsing
+   * @throws IllegalASTFormatException exception in sql parsing
    */
   private AstNode parseSQLToAST(String sqlStr) throws IllegalASTFormatException {
     AstNode astTree;
@@ -115,12 +108,9 @@ public class QueryProcessor {
   /**
    * given an unoptimized logical operator tree and return a optimized result.
    *
-   * @param operator
-   *            unoptimized logical operator
-   * @param executor
+   * @param operator unoptimized logical operator
    * @return optimized logical operator
-   * @throws LogicalOptimizeException
-   *             exception in logical optimizing
+   * @throws LogicalOptimizeException exception in logical optimizing
    */
   private Operator logicalOptimize(Operator operator, IQueryProcessExecutor executor)
       throws LogicalOperatorException {
@@ -139,6 +129,7 @@ public class QueryProcessor {
       case GRANT_WATERMARK_EMBEDDING:
       case REVOKE_WATERMARK_EMBEDDING:
       case TTL:
+      case LOAD_CONFIGURATION:
         return operator;
       case QUERY:
       case UPDATE:
@@ -153,12 +144,9 @@ public class QueryProcessor {
   /**
    * given an unoptimized select-from-where operator and return an optimized result.
    *
-   * @param root
-   *            unoptimized select-from-where operator
-   * @param executor
+   * @param root unoptimized select-from-where operator
    * @return optimized select-from-where operator
-   * @throws LogicalOptimizeException
-   *             exception in SFW optimizing
+   * @throws LogicalOptimizeException exception in SFW optimizing
    */
   private SFWOperator optimizeSFWOperator(SFWOperator root, IQueryProcessExecutor executor)
       throws LogicalOperatorException {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
index 8479e7a..adc63f6 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/constant/SQLConstant.java
@@ -102,6 +102,7 @@ public class SQLConstant {
   public static final int TOK_SET = 63;
   public static final int TOK_UNSET = 64;
   public static final int TOK_SHOW = 65;
+  public static final int TOK_LOAD_CONFIGURATION = 66;
 
   public static final Map<Integer, String> tokenSymbol = new HashMap<>();
   public static final Map<Integer, String> tokenNames = new HashMap<>();
@@ -161,6 +162,8 @@ public class SQLConstant {
     tokenNames.put(TOK_SET, "TOK_SET");
     tokenNames.put(TOK_UNSET, "TOK_UNSET");
     tokenNames.put(TOK_SHOW, "TOK_SHOW");
+
+    tokenNames.put(TOK_LOAD_CONFIGURATION, "TOK_LOAD_CONFIGURATION");
   }
 
   static {
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java
index d3d6b64..779fec3 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/QueryProcessExecutor.java
@@ -54,14 +54,14 @@ import org.apache.iotdb.db.qp.physical.crud.BatchInsertPlan;
 import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
 import org.apache.iotdb.db.qp.physical.crud.UpdatePlan;
-import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.DataAuthPlan;
-import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.DeleteStorageGroupPlan;
+import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.PropertyPlan;
-import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
 import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
 import org.apache.iotdb.db.query.context.QueryContext;
 import org.apache.iotdb.db.query.dataset.ListDataSet;
 import org.apache.iotdb.db.query.fill.IFill;
@@ -130,13 +130,16 @@ public class QueryProcessExecutor extends AbstractQueryProcessExecutor {
       case SET_STORAGE_GROUP:
         return setStorageGroup((SetStorageGroupPlan) plan);
       case DELETE_STORAGE_GROUP:
-        return deleteStorageGroup((DeleteStorageGroupPlan) plan);  
+        return deleteStorageGroup((DeleteStorageGroupPlan) plan);
       case PROPERTY:
         PropertyPlan property = (PropertyPlan) plan;
         return operateProperty(property);
       case TTL:
         operateTTL((SetTTLPlan) plan);
         return true;
+      case LOAD_CONFIGURATION:
+        IoTDBDescriptor.getInstance().loadHotModifiedProps();
+        return true;
       default:
         throw new UnsupportedOperationException(
             String.format("operation %s is not supported", plan.getOperatorType()));
@@ -384,8 +387,9 @@ public class QueryProcessExecutor extends AbstractQueryProcessExecutor {
     }
     return true;
   }
-  
-  private boolean createTimeSeries(CreateTimeSeriesPlan createTimeSeriesPlan) throws ProcessorException {
+
+  private boolean createTimeSeries(CreateTimeSeriesPlan createTimeSeriesPlan)
+      throws ProcessorException {
     Path path = createTimeSeriesPlan.getPath();
     TSDataType dataType = createTimeSeriesPlan.getDataType();
     CompressionType compressor = createTimeSeriesPlan.getCompressor();
@@ -401,8 +405,9 @@ public class QueryProcessExecutor extends AbstractQueryProcessExecutor {
     }
     return true;
   }
-  
-  private boolean deleteTimeSeries(DeleteTimeSeriesPlan deleteTimeSeriesPlan) throws ProcessorException {
+
+  private boolean deleteTimeSeries(DeleteTimeSeriesPlan deleteTimeSeriesPlan)
+      throws ProcessorException {
     List<Path> deletePathList = deleteTimeSeriesPlan.getPaths();
     try {
       deleteDataOfTimeSeries(deletePathList);
@@ -415,8 +420,9 @@ public class QueryProcessExecutor extends AbstractQueryProcessExecutor {
     }
     return true;
   }
-  
-  private boolean setStorageGroup(SetStorageGroupPlan setStorageGroupPlan) throws ProcessorException {
+
+  private boolean setStorageGroup(SetStorageGroupPlan setStorageGroupPlan)
+      throws ProcessorException {
     Path path = setStorageGroupPlan.getPath();
     try {
       mManager.setStorageGroupToMTree(path.getFullPath());
@@ -425,8 +431,9 @@ public class QueryProcessExecutor extends AbstractQueryProcessExecutor {
     }
     return true;
   }
-  
-  private boolean deleteStorageGroup(DeleteStorageGroupPlan deleteStorageGroupPlan) throws ProcessorException {
+
+  private boolean deleteStorageGroup(DeleteStorageGroupPlan deleteStorageGroupPlan)
+      throws ProcessorException {
     List<Path> deletePathList = deleteStorageGroupPlan.getPaths();
     try {
       mManager.deleteStorageGroupsFromMTree(deletePathList);
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
index 8b889a8..06fc4be 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/Operator.java
@@ -74,6 +74,6 @@ public abstract class Operator {
     DELETE_ROLE, GRANT_ROLE_PRIVILEGE, REVOKE_ROLE_PRIVILEGE, LIST_USER, LIST_ROLE,
     LIST_USER_PRIVILEGE, LIST_ROLE_PRIVILEGE, LIST_USER_ROLES, LIST_ROLE_USERS,
     GRANT_WATERMARK_EMBEDDING, REVOKE_WATERMARK_EMBEDDING,
-    TTL, DELETE_STORAGE_GROUP
+    TTL, DELETE_STORAGE_GROUP, LOAD_CONFIGURATION
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/LoadConfigurationOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/LoadConfigurationOperator.java
new file mode 100644
index 0000000..18ad4ae
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/sys/LoadConfigurationOperator.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.iotdb.db.qp.logical.sys;
+
+import org.apache.iotdb.db.qp.constant.SQLConstant;
+import org.apache.iotdb.db.qp.logical.RootOperator;
+
+public class LoadConfigurationOperator extends RootOperator {
+
+  public LoadConfigurationOperator() {
+    super(SQLConstant.TOK_LOAD_CONFIGURATION);
+    this.operatorType = OperatorType.LOAD_CONFIGURATION;
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/LoadConfigurationPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/LoadConfigurationPlan.java
new file mode 100644
index 0000000..ae0f77e
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/LoadConfigurationPlan.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.iotdb.db.qp.physical.sys;
+
+import java.util.List;
+import org.apache.iotdb.db.qp.logical.Operator.OperatorType;
+import org.apache.iotdb.db.qp.physical.PhysicalPlan;
+import org.apache.iotdb.tsfile.read.common.Path;
+
+public class LoadConfigurationPlan extends PhysicalPlan {
+
+  public LoadConfigurationPlan() {
+    super(false);
+    this.setOperatorType(OperatorType.LOAD_CONFIGURATION);
+  }
+
+  @Override
+  public List<Path> getPaths() {
+    return null;
+  }
+
+  @Override
+  public String toString() {
+    return getOperatorType().toString();
+  }
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
index cca6ec0..1d24406 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/LogicalGenerator.java
@@ -50,6 +50,7 @@ import static org.apache.iotdb.db.sql.parse.TqlParser.TOK_LINEAR;
 import static org.apache.iotdb.db.sql.parse.TqlParser.TOK_LINK;
 import static org.apache.iotdb.db.sql.parse.TqlParser.TOK_LIST;
 import static org.apache.iotdb.db.sql.parse.TqlParser.TOK_LOAD;
+import static org.apache.iotdb.db.sql.parse.TqlParser.TOK_LOAD_CONFIGURATION;
 import static org.apache.iotdb.db.sql.parse.TqlParser.TOK_PATH;
 import static org.apache.iotdb.db.sql.parse.TqlParser.TOK_PREVIOUS;
 import static org.apache.iotdb.db.sql.parse.TqlParser.TOK_PRIVILEGES;
@@ -103,6 +104,7 @@ import org.apache.iotdb.db.qp.logical.sys.CreateTimeSeriesOperator;
 import org.apache.iotdb.db.qp.logical.sys.DataAuthOperator;
 import org.apache.iotdb.db.qp.logical.sys.DeleteStorageGroupOperator;
 import org.apache.iotdb.db.qp.logical.sys.DeleteTimeSeriesOperator;
+import org.apache.iotdb.db.qp.logical.sys.LoadConfigurationOperator;
 import org.apache.iotdb.db.qp.logical.sys.LoadDataOperator;
 import org.apache.iotdb.db.qp.logical.sys.PropertyOperator;
 import org.apache.iotdb.db.qp.logical.sys.SetStorageGroupOperator;
@@ -274,6 +276,9 @@ public class LogicalGenerator {
       case TOK_GROUPBY_DEVICE:
         ((QueryOperator) initializedOperator).setGroupByDevice(true);
         return;
+      case TOK_LOAD_CONFIGURATION:
+        initializedOperator = new LoadConfigurationOperator();
+        return;
       default:
         throw new QueryProcessorException("Not supported TqlParser type " + token.getText());
     }
@@ -301,8 +306,7 @@ public class LogicalGenerator {
 
   private void analyzeSetTTL(AstNode astNode) {
     String path = parsePath(astNode.getChild(1)).getFullPath();
-    long dataTTL;
-    dataTTL = Long.parseLong(astNode.getChild(2).getText());
+    long dataTTL = Long.parseLong(astNode.getChild(2).getText());
     SetTTLOperator operator = new SetTTLOperator(SQLConstant.TOK_SET);
     initializedOperator = operator;
     operator.setStorageGroup(path);
diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
index 5658e18..874c937 100644
--- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
+++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java
@@ -16,7 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.iotdb.db.qp.strategy;
 
 import java.util.ArrayList;
@@ -40,16 +39,16 @@ import org.apache.iotdb.db.qp.logical.crud.DeleteDataOperator;
 import org.apache.iotdb.db.qp.logical.crud.FilterOperator;
 import org.apache.iotdb.db.qp.logical.crud.InsertOperator;
 import org.apache.iotdb.db.qp.logical.crud.QueryOperator;
-import org.apache.iotdb.db.qp.logical.sys.CreateTimeSeriesOperator;
 import org.apache.iotdb.db.qp.logical.sys.AuthorOperator;
+import org.apache.iotdb.db.qp.logical.sys.CreateTimeSeriesOperator;
 import org.apache.iotdb.db.qp.logical.sys.DataAuthOperator;
-import org.apache.iotdb.db.qp.logical.sys.DeleteTimeSeriesOperator;
 import org.apache.iotdb.db.qp.logical.sys.DeleteStorageGroupOperator;
+import org.apache.iotdb.db.qp.logical.sys.DeleteTimeSeriesOperator;
 import org.apache.iotdb.db.qp.logical.sys.LoadDataOperator;
 import org.apache.iotdb.db.qp.logical.sys.PropertyOperator;
+import org.apache.iotdb.db.qp.logical.sys.SetStorageGroupOperator;
 import org.apache.iotdb.db.qp.logical.sys.SetTTLOperator;
 import org.apache.iotdb.db.qp.logical.sys.ShowTTLOperator;
-import org.apache.iotdb.db.qp.logical.sys.SetStorageGroupOperator;
 import org.apache.iotdb.db.qp.physical.PhysicalPlan;
 import org.apache.iotdb.db.qp.physical.crud.AggregationPlan;
 import org.apache.iotdb.db.qp.physical.crud.DeletePlan;
@@ -57,16 +56,17 @@ import org.apache.iotdb.db.qp.physical.crud.FillQueryPlan;
 import org.apache.iotdb.db.qp.physical.crud.GroupByPlan;
 import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
 import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
-import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
+import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.DataAuthPlan;
-import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.DeleteStorageGroupPlan;
+import org.apache.iotdb.db.qp.physical.sys.DeleteTimeSeriesPlan;
+import org.apache.iotdb.db.qp.physical.sys.LoadConfigurationPlan;
 import org.apache.iotdb.db.qp.physical.sys.LoadDataPlan;
 import org.apache.iotdb.db.qp.physical.sys.PropertyPlan;
+import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
 import org.apache.iotdb.db.qp.physical.sys.SetTTLPlan;
 import org.apache.iotdb.db.qp.physical.sys.ShowTTLPlan;
-import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan;
 import org.apache.iotdb.db.service.TSServiceImpl;
 import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
 import org.apache.iotdb.tsfile.read.common.Path;
@@ -149,8 +149,12 @@ public class PhysicalGenerator {
           case SQLConstant.TOK_SHOW:
             ShowTTLOperator showTTLOperator = (ShowTTLOperator) operator;
             return new ShowTTLPlan(showTTLOperator.getStorageGroups());
+          default:
+            throw new LogicalOperatorException(String
+                .format("not supported operator type %s in ttl operation.", operator.getType()));
         }
-
+      case LOAD_CONFIGURATION:
+        return new LoadConfigurationPlan();
       default:
         throw new LogicalOperatorException("not supported operator type: " + operator.getType());
     }
diff --git a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
index 09ad27a..3297fb0 100644
--- a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java
@@ -37,6 +37,7 @@ import org.apache.iotdb.db.qp.physical.crud.QueryPlan;
 import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan;
 import org.apache.iotdb.db.qp.physical.sys.AuthorPlan;
 import org.apache.iotdb.db.qp.physical.sys.DataAuthPlan;
+import org.apache.iotdb.db.qp.physical.sys.LoadConfigurationPlan;
 import org.apache.iotdb.db.qp.physical.sys.PropertyPlan;
 import org.apache.iotdb.db.qp.utils.MemIntQpExecutor;
 import org.apache.iotdb.db.query.fill.LinearFill;
@@ -487,4 +488,13 @@ public class PhysicalPlanTest {
     Assert.assertEquals(2, dataAuthPlan.getUsers().size());
     Assert.assertEquals(OperatorType.REVOKE_WATERMARK_EMBEDDING, dataAuthPlan.getOperatorType());
   }
+
+  @Test
+  public void testConfiguration()
+      throws QueryProcessorException, MetadataErrorException {
+    String metadata = "load configuration";
+    QueryProcessor processor = new QueryProcessor(new MemIntQpExecutor());
+    LoadConfigurationPlan plan = (LoadConfigurationPlan) processor.parseSQLToPhysicalPlan(metadata);
+    assertEquals("LOAD_CONFIGURATION", plan.toString());
+  }
 }


[incubator-iotdb] 02/03: Merge branch 'master' into add_hot_load_configuration

Posted by lt...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lta pushed a commit to branch add_hot_load_configuration
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git

commit a2dc1659e9a98cbe6696c5b28246382b0496ba8e
Merge: bc27c55 efac7c4
Author: lta <li...@163.com>
AuthorDate: Fri Nov 8 09:48:32 2019 +0800

    Merge branch 'master' into add_hot_load_configuration

 .../UserGuide/3-Server/4-Config Manual.md          |  27 +++
 .../3-Writing Data on HDFS.md                      |  27 +++
 .../UserGuide/3-Server/4-Config Manual.md          |  28 +++
 ...e Hadoop Connector.md => 2-MapReduce TsFile.md} |   1 -
 ...TsFile Spark Connector.md => 3-Spark TsFile.md} |   0
 ...4-Spark IoTDB Connector.md => 4-Spark IoTDB.md} |  10 +-
 ...5-TsFile Hive Connector.md => 5-Hive TsFile.md} |   0
 .../3-Writing Data on HDFS.md                      |  27 +++
 .../iotdb/hadoop/fileSystem/HDFSConfUtil.java      |  14 ++
 .../apache/iotdb/jdbc/IoTDBPreparedStatement.java  |  30 +--
 .../java/org/apache/iotdb/jdbc/IoTDBStatement.java |  25 +-
 .../test/java/org/apache/iotdb/jdbc/BatchTest.java |  29 ++-
 .../resources/conf/iotdb-engine.properties         |  12 +-
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |  40 ++++
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |  27 ++-
 .../org/apache/iotdb/db/service/TSServiceImpl.java | 149 +++++++-----
 .../apache/iotdb/db/utils/TypeInferenceUtils.java  |  10 +-
 .../apache/iotdb/db/integration/IoTDBCloseIT.java  | 202 ++++++++++++++++
 service-rpc/rpc-changelist.md                      |   1 +
 service-rpc/src/main/thrift/rpc.thrift             |   3 +
 .../java/org/apache/iotdb/session/Session.java     |  44 ++--
 .../iotdb/{sparkdb => spark/db}/SQLConstant.java   |   2 +-
 .../iotdb/{sparkdb => spark/db}/Converter.scala    |   2 +-
 .../{sparkdb => spark/db}/DefaultSource.scala      |   2 +-
 .../iotdb/{sparkdb => spark/db}/IoTDBOptions.scala |   2 +-
 .../iotdb/{sparkdb => spark/db}/IoTDBRDD.scala     |   8 +-
 .../{sparkdb => spark/db}/IoTDBRelation.scala      |   2 +-
 .../iotdb/{sparkdb => spark/db}/Transformer.scala  |  66 +++--
 .../iotdb/{sparkdb => spark/db}/package.scala      |   6 +-
 .../{sparkdb => spark/db}/EnvironmentUtils.java    |   2 +-
 .../iotdb/{sparkdb => spark/db}/IoTDBTest.scala    |   2 +-
 .../apache/iotdb/spark/tsfile/DefaultSource.scala  |  16 +-
 .../apache/iotdb/spark/tsfile/Transformer.scala    |  56 ++---
 .../org/apache/iotdb/spark/tsfile/package.scala    |   6 +-
 .../org/apache/iotdb/spark/tsfile/TSFileSuit.scala |   4 +-
 .../iotdb/tsfile/common/conf/TSFileConfig.java     | 265 ++++++++++++---------
 36 files changed, 798 insertions(+), 349 deletions(-)

diff --cc server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
index 28fcd97,8efc8d6..25091b3
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
@@@ -291,14 -342,57 +301,21 @@@ public class IoTDBDescriptor 
            Boolean.parseBoolean(properties.getProperty("dfs_ha_automatic_failover_enabled")));
        TSFileDescriptor.getInstance().getConfig().setDfsClientFailoverProxyProvider(
            properties.getProperty("dfs_client_failover_proxy_provider"));
+       TSFileDescriptor.getInstance().getConfig().setUseKerberos(Boolean.parseBoolean(
+           properties.getProperty("hdfs_use_kerberos", String.valueOf(conf.isUseKerberos()))));
+       TSFileDescriptor.getInstance().getConfig().setKerberosKeytabFilePath(
+           properties.getProperty("kerberos_keytab_file_path", conf.getKerberosKeytabFilePath()));
+       TSFileDescriptor.getInstance().getConfig().setKerberosPrincipal(
+           properties.getProperty("kerberos_principal", conf.getKerberosPrincipal()));
+ 
  
        // set tsfile-format config
 -      TSFileDescriptor.getInstance().getConfig().setGroupSizeInByte(Integer
 -          .parseInt(properties.getProperty("group_size_in_byte",
 -              Integer.toString(TSFileDescriptor.getInstance().getConfig().getGroupSizeInByte()))));
 -      TSFileDescriptor.getInstance().getConfig().setPageSizeInByte(Integer
 -          .parseInt(properties.getProperty("page_size_in_byte",
 -              Integer.toString(TSFileDescriptor.getInstance().getConfig().getPageSizeInByte()))));
 -      if (TSFileDescriptor.getInstance().getConfig().getPageSizeInByte() > TSFileDescriptor
 -          .getInstance().getConfig().getGroupSizeInByte()) {
 -        logger
 -            .warn("page_size is greater than group size, will set it as the same with group size");
 -        TSFileDescriptor.getInstance().getConfig()
 -            .setPageSizeInByte(TSFileDescriptor.getInstance().getConfig().getGroupSizeInByte());
 -      }
 -      TSFileDescriptor.getInstance().getConfig().setMaxNumberOfPointsInPage(Integer
 -          .parseInt(properties.getProperty("max_number_of_points_in_page",
 -              Integer.toString(
 -                  TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage()))));
 -      TSFileDescriptor.getInstance().getConfig().setTimeSeriesDataType(properties
 -          .getProperty("time_series_data_type",
 -              TSFileDescriptor.getInstance().getConfig().getTimeSeriesDataType()));
 -      TSFileDescriptor.getInstance().getConfig().setMaxStringLength(Integer
 -          .parseInt(properties.getProperty("max_string_length",
 -              Integer.toString(TSFileDescriptor.getInstance().getConfig().getMaxStringLength()))));
 -      TSFileDescriptor.getInstance().getConfig().setBloomFilterErrorRate(Double
 -          .parseDouble(properties.getProperty("bloom_filter_error_rate",
 -              Double.toString(
 -                  TSFileDescriptor.getInstance().getConfig().getBloomFilterErrorRate()))));
 -      TSFileDescriptor.getInstance().getConfig().setFloatPrecision(Integer
 -          .parseInt(properties
 -              .getProperty("float_precision", Integer
 -                  .toString(TSFileDescriptor.getInstance().getConfig().getFloatPrecision()))));
 -      TSFileDescriptor.getInstance().getConfig().setTimeEncoder(properties
 -          .getProperty("time_encoder",
 -              TSFileDescriptor.getInstance().getConfig().getTimeEncoder()));
 -      TSFileDescriptor.getInstance().getConfig().setValueEncoder(properties
 -          .getProperty("value_encoder",
 -              TSFileDescriptor.getInstance().getConfig().getValueEncoder()));
 -      TSFileDescriptor.getInstance().getConfig().setCompressor(properties
 -          .getProperty("compressor", TSFileDescriptor.getInstance().getConfig().getCompressor()));
 +      loadTsFileProps(properties);
  
 +    } catch (FileNotFoundException e) {
 +      logger.warn("Fail to find config file {}", url, e);
      } catch (IOException e) {
 -      logger.warn("Cannot load config file because, use default configuration", e);
 +      logger.warn("Cannot load config file, use default configuration", e);
      } catch (Exception e) {
        logger.warn("Incorrect format in config file, use default configuration", e);
      } finally {


[incubator-iotdb] 01/03: add hot load config

Posted by lt...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lta pushed a commit to branch add_hot_load_configuration
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git

commit bc27c55e6edae04c09c3d7416ee388cc1ab74358
Author: lta <li...@163.com>
AuthorDate: Wed Nov 6 20:27:53 2019 +0800

    add hot load config
---
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |  78 +++---
 .../org/apache/iotdb/db/conf/IoTDBConfigCheck.java |   4 +-
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  | 261 +++++++++++++--------
 .../db/conf/directories/DirectoryManager.java      |  28 ++-
 .../directories/strategy/DirectoryStrategy.java    |   2 +
 5 files changed, 230 insertions(+), 143 deletions(-)

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 88a23b5..7fc9528 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
@@ -20,11 +20,9 @@ package org.apache.iotdb.db.conf;
 
 import java.io.File;
 import java.time.ZoneId;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
 import org.apache.iotdb.db.engine.merge.selector.MergeFileStrategy;
 import org.apache.iotdb.db.metadata.MManager;
 import org.apache.iotdb.db.service.TSServiceImpl;
@@ -446,50 +444,41 @@ public class IoTDBConfig {
    * if the folders are relative paths, add IOTDB_HOME as the path prefix
    */
   private void formulateFolders() {
-    List<String> dirs = new ArrayList<>();
-    dirs.add(baseDir);
-    dirs.add(systemDir);
-    dirs.add(schemaDir);
-    dirs.add(walFolder);
-    dirs.add(indexFileDir);
-    dirs.add(queryDir);
-    dirs.addAll(Arrays.asList(dataDirs));
-
-    for (int i = 0; i < 4; i++) {
-      addHomeDir(dirs, i);
-    }
+    baseDir = addHomeDir(baseDir);
+    systemDir = addHomeDir(systemDir);
+    schemaDir = addHomeDir(schemaDir);
+    walFolder = addHomeDir(walFolder);
 
     if (TSFileDescriptor.getInstance().getConfig().getTSFileStorageFs().equals(FSType.HDFS)) {
-      String[] hdfsIps = TSFileDescriptor.getInstance().getConfig().getHdfsIp();
-      String hdfsDir = "hdfs://";
-      if (hdfsIps.length > 1) {
-        hdfsDir += TSFileDescriptor.getInstance().getConfig().getDfsNameServices();
-      } else {
-        hdfsDir += hdfsIps[0] + ":" + TSFileDescriptor.getInstance().getConfig().getHdfsPort();
-      }
-      for (int i = 5; i < dirs.size(); i++) {
-        String dir = dirs.get(i);
-        dir = hdfsDir + File.separatorChar + dir;
-        dirs.set(i, dir);
+      String hdfsDir = getHdfsDir();
+      queryDir = hdfsDir + File.separatorChar + queryDir;
+      for (int i = 0; i < dataDirs.length; i++) {
+        dataDirs[i] = hdfsDir + File.separatorChar + dataDirs[i];
       }
     } else {
-      for (int i = 5; i < dirs.size(); i++) {
-        addHomeDir(dirs, i);
+      queryDir = addHomeDir(queryDir);
+      for (int i = 0; i < dataDirs.length; i++) {
+        dataDirs[i] = addHomeDir(dataDirs[i]);
       }
     }
-    baseDir = dirs.get(0);
-    systemDir = dirs.get(1);
-    schemaDir = dirs.get(2);
-    walFolder = dirs.get(3);
-    indexFileDir = dirs.get(4);
-    queryDir = dirs.get(5);
-    for (int i = 0; i < dataDirs.length; i++) {
-      dataDirs[i] = dirs.get(i + 6);
+  }
+
+  void reloadDataDirs(String[] dataDirs) {
+    if (TSFileDescriptor.getInstance().getConfig().getTSFileStorageFs().equals(FSType.HDFS)) {
+      String hdfsDir = getHdfsDir();
+      for (int i = 0; i < dataDirs.length; i++) {
+        dataDirs[i] = hdfsDir + File.separatorChar + dataDirs[i];
+      }
+    } else {
+      for (int i = 0; i < dataDirs.length; i++) {
+        dataDirs[i] = addHomeDir(dataDirs[i]);
+      }
     }
+    this.dataDirs = dataDirs;
+    DirectoryManager.getInstance().updateFileFolders();
   }
 
-  private void addHomeDir(List<String> dirs, int i) {
-    String dir = dirs.get(i);
+  private String addHomeDir(String dir) {
     String homeDir = System.getProperty(IoTDBConstant.IOTDB_HOME, null);
     if (!new File(dir).isAbsolute() && homeDir != null && homeDir.length() > 0) {
       if (!homeDir.endsWith(File.separator)) {
@@ -497,8 +486,8 @@ public class IoTDBConfig {
       } else {
         dir = homeDir + dir;
       }
-      dirs.set(i, dir);
     }
+    return dir;
   }
 
   private void confirmMultiDirStrategy() {
@@ -518,6 +507,17 @@ public class IoTDBConfig {
     }
   }
 
+  private String getHdfsDir(){
+    String[] hdfsIps = TSFileDescriptor.getInstance().getConfig().getHdfsIp();
+    String hdfsDir = "hdfs://";
+    if (hdfsIps.length > 1) {
+      hdfsDir += TSFileDescriptor.getInstance().getConfig().getDfsNameServices();
+    } else {
+      hdfsDir += hdfsIps[0] + ":" + TSFileDescriptor.getInstance().getConfig().getHdfsPort();
+    }
+    return hdfsDir;
+  }
+
   public String[] getDataDirs() {
     return dataDirs;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
index 7b6bdef..09c648f 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfigCheck.java
@@ -54,7 +54,7 @@ public class IoTDBConfigCheck {
     logger.info("System configuration is ok.");
   }
 
-  public void createDir(String filepath) {
+  private void createDir(String filepath) {
     File dir = SystemFileFactory.INSTANCE.getFile(filepath);
     if (!dir.exists()) {
       dir.mkdirs();
@@ -62,7 +62,7 @@ public class IoTDBConfigCheck {
     }
   }
 
-  public void checkFile(String filepath) {
+  private void checkFile(String filepath) {
     // create file : read timestamp precision from engine.properties, create system_properties.txt
     // use output stream to write timestamp precision to file.
     File file = SystemFileFactory.INSTANCE.getFile(filepath + File.separator + PROPERTIES_FILE_NAME);
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 24cd142..28fcd97 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
@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.time.ZoneId;
 import java.util.Properties;
+import org.apache.iotdb.db.conf.directories.DirectoryManager;
 import org.apache.iotdb.db.utils.FilePathUtils;
 import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
 import org.slf4j.Logger;
@@ -72,25 +73,15 @@ public class IoTDBDescriptor {
    * load an property file and set TsfileDBConfig variables.
    */
   private void loadProps() {
-    InputStream inputStream;
-
     String url = getPropsUrl();
     if (url == null) {
       return;
     }
 
-    try {
-      inputStream = new FileInputStream(new File(url));
-    } catch (FileNotFoundException e) {
-      logger.warn("Fail to find config file {}", url, e);
-      // update all data seriesPath
-      conf.updatePath();
-      return;
-    }
+    try (InputStream inputStream = new FileInputStream(new File(url))) {
 
-    logger.info("Start to read config file {}", url);
-    Properties properties = new Properties();
-    try {
+      logger.info("Start to read config file {}", url);
+      Properties properties = new Properties();
       properties.load(inputStream);
       conf.setEnableStatMonitor(Boolean
           .parseBoolean(properties.getProperty("enable_stat_monitor",
@@ -144,8 +135,7 @@ public class IoTDBDescriptor {
 
       initMemoryAllocate(properties);
 
-      conf.setEnableWal(Boolean.parseBoolean(properties.getProperty("enable_wal",
-          Boolean.toString(conf.isEnableWal()))));
+      loadWALProps(properties);
 
       conf.setBaseDir(properties.getProperty("base_dir", conf.getBaseDir()));
 
@@ -160,14 +150,6 @@ public class IoTDBDescriptor {
 
       conf.setWalFolder(properties.getProperty("wal_dir", conf.getWalFolder()));
 
-      conf.setFlushWalThreshold(Integer
-          .parseInt(properties.getProperty("flush_wal_threshold",
-              Integer.toString(conf.getFlushWalThreshold()))));
-
-      conf.setForceWalPeriodInMs(Long
-          .parseLong(properties.getProperty("force_wal_period_in_ms",
-              Long.toString(conf.getForceWalPeriodInMs()))));
-
       int walBufferSize = Integer.parseInt(properties.getProperty("wal_buffer_size",
           Integer.toString(conf.getWalBufferSize())));
       if (walBufferSize > 0) {
@@ -223,8 +205,8 @@ public class IoTDBDescriptor {
         conf.setChunkBufferPoolEnable(Boolean
             .parseBoolean(properties.getProperty("chunk_buffer_pool_enable")));
       }
-      String tmpTimeZone = properties.getProperty("time_zone", conf.getZoneID().toString());
-      conf.setZoneID(ZoneId.of(tmpTimeZone.trim()));
+      conf.setZoneID(
+          ZoneId.of(properties.getProperty("time_zone", conf.getZoneID().toString().trim())));
       logger.info("Time zone has been set to {}", conf.getZoneID());
 
       conf.setEnableExternalSort(Boolean.parseBoolean(properties
@@ -278,30 +260,7 @@ public class IoTDBDescriptor {
       conf.setWatermarkMethod(
           properties.getProperty("watermark_method", conf.getWatermarkMethod()));
 
-      conf.setAutoCreateSchemaEnabled(
-          Boolean.parseBoolean(properties.getProperty("enable_auto_create_schema",
-              Boolean.toString(conf.isAutoCreateSchemaEnabled()).trim())));
-      conf.setDefaultStorageGroupLevel(
-          Integer.parseInt(properties.getProperty("default_storage_group_level",
-              Integer.toString(conf.getDefaultStorageGroupLevel()))));
-      conf.setDefaultBooleanEncoding(
-          properties.getProperty("default_boolean_encoding",
-              conf.getDefaultBooleanEncoding().toString()));
-      conf.setDefaultInt32Encoding(
-          properties.getProperty("default_int32_encoding",
-              conf.getDefaultInt32Encoding().toString()));
-      conf.setDefaultInt64Encoding(
-          properties.getProperty("default_int64_encoding",
-              conf.getDefaultInt64Encoding().toString()));
-      conf.setDefaultFloatEncoding(
-          properties.getProperty("default_float_encoding",
-              conf.getDefaultFloatEncoding().toString()));
-      conf.setDefaultDoubleEncoding(
-          properties.getProperty("default_double_encoding",
-              conf.getDefaultDoubleEncoding().toString()));
-      conf.setDefaultTextEncoding(
-          properties.getProperty("default_text_encoding",
-              conf.getDefaultTextEncoding().toString()));
+      loadAutoCreateSchemaProps(properties);
 
       conf.setRpcMaxConcurrentClientNum(maxConcurrentClientNum);
 
@@ -319,69 +278,177 @@ public class IoTDBDescriptor {
           String.valueOf(conf.getDefaultTTL()))));
 
       // At the same time, set TSFileConfig
-      TSFileDescriptor.getInstance().getConfig().setTSFileStorageFs(properties.getProperty("tsfile_storage_fs"));
-      TSFileDescriptor.getInstance().getConfig().setHdfsIp(properties.getProperty("hdfs_ip").split(","));
+      TSFileDescriptor.getInstance().getConfig()
+          .setTSFileStorageFs(properties.getProperty("tsfile_storage_fs"));
+      TSFileDescriptor.getInstance().getConfig()
+          .setHdfsIp(properties.getProperty("hdfs_ip").split(","));
       TSFileDescriptor.getInstance().getConfig().setHdfsPort(properties.getProperty("hdfs_port"));
-      TSFileDescriptor.getInstance().getConfig().setDfsNameServices(properties.getProperty("dfs_nameservices"));
-      TSFileDescriptor.getInstance().getConfig().setDfsHaNamenodes(properties.getProperty("dfs_ha_namenodes").split(","));
+      TSFileDescriptor.getInstance().getConfig()
+          .setDfsNameServices(properties.getProperty("dfs_nameservices"));
+      TSFileDescriptor.getInstance().getConfig()
+          .setDfsHaNamenodes(properties.getProperty("dfs_ha_namenodes").split(","));
       TSFileDescriptor.getInstance().getConfig().setDfsHaAutomaticFailoverEnabled(
           Boolean.parseBoolean(properties.getProperty("dfs_ha_automatic_failover_enabled")));
       TSFileDescriptor.getInstance().getConfig().setDfsClientFailoverProxyProvider(
           properties.getProperty("dfs_client_failover_proxy_provider"));
 
       // set tsfile-format config
-      TSFileDescriptor.getInstance().getConfig().setGroupSizeInByte(Integer
-          .parseInt(properties.getProperty("group_size_in_byte",
-              Integer.toString(TSFileDescriptor.getInstance().getConfig().getGroupSizeInByte()))));
-      TSFileDescriptor.getInstance().getConfig().setPageSizeInByte(Integer
-          .parseInt(properties.getProperty("page_size_in_byte",
-              Integer.toString(TSFileDescriptor.getInstance().getConfig().getPageSizeInByte()))));
-      if (TSFileDescriptor.getInstance().getConfig().getPageSizeInByte() > TSFileDescriptor
-          .getInstance().getConfig().getGroupSizeInByte()) {
-        logger
-            .warn("page_size is greater than group size, will set it as the same with group size");
-        TSFileDescriptor.getInstance().getConfig()
-            .setPageSizeInByte(TSFileDescriptor.getInstance().getConfig().getGroupSizeInByte());
-      }
-      TSFileDescriptor.getInstance().getConfig().setMaxNumberOfPointsInPage(Integer
-          .parseInt(properties.getProperty("max_number_of_points_in_page",
-              Integer.toString(
-                  TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage()))));
-      TSFileDescriptor.getInstance().getConfig().setTimeSeriesDataType(properties
-          .getProperty("time_series_data_type",
-              TSFileDescriptor.getInstance().getConfig().getTimeSeriesDataType()));
-      TSFileDescriptor.getInstance().getConfig().setMaxStringLength(Integer
-          .parseInt(properties.getProperty("max_string_length",
-              Integer.toString(TSFileDescriptor.getInstance().getConfig().getMaxStringLength()))));
-      TSFileDescriptor.getInstance().getConfig().setBloomFilterErrorRate(Double
-          .parseDouble(properties.getProperty("bloom_filter_error_rate",
-              Double.toString(
-                  TSFileDescriptor.getInstance().getConfig().getBloomFilterErrorRate()))));
-      TSFileDescriptor.getInstance().getConfig().setFloatPrecision(Integer
-          .parseInt(properties
-              .getProperty("float_precision", Integer
-                  .toString(TSFileDescriptor.getInstance().getConfig().getFloatPrecision()))));
-      TSFileDescriptor.getInstance().getConfig().setTimeEncoder(properties
-          .getProperty("time_encoder",
-              TSFileDescriptor.getInstance().getConfig().getTimeEncoder()));
-      TSFileDescriptor.getInstance().getConfig().setValueEncoder(properties
-          .getProperty("value_encoder",
-              TSFileDescriptor.getInstance().getConfig().getValueEncoder()));
-      TSFileDescriptor.getInstance().getConfig().setCompressor(properties
-          .getProperty("compressor", TSFileDescriptor.getInstance().getConfig().getCompressor()));
+      loadTsFileProps(properties);
 
+    } catch (FileNotFoundException e) {
+      logger.warn("Fail to find config file {}", url, e);
     } catch (IOException e) {
-      logger.warn("Cannot load config file because, use default configuration", e);
+      logger.warn("Cannot load config file, use default configuration", e);
     } catch (Exception e) {
       logger.warn("Incorrect format in config file, use default configuration", e);
     } finally {
       // update all data seriesPath
       conf.updatePath();
-      try {
-        inputStream.close();
-      } catch (IOException e) {
-        logger.error("Fail to close config file input stream because ", e);
+    }
+  }
+
+  private void loadWALProps(Properties properties){
+    conf.setEnableWal(Boolean.parseBoolean(properties.getProperty("enable_wal",
+        Boolean.toString(conf.isEnableWal()))));
+
+    conf.setFlushWalThreshold(Integer
+        .parseInt(properties.getProperty("flush_wal_threshold",
+            Integer.toString(conf.getFlushWalThreshold()))));
+
+    conf.setForceWalPeriodInMs(Long
+        .parseLong(properties.getProperty("force_wal_period_in_ms",
+            Long.toString(conf.getForceWalPeriodInMs()))));
+
+  }
+
+  private void loadAutoCreateSchemaProps(Properties properties){
+    conf.setAutoCreateSchemaEnabled(
+        Boolean.parseBoolean(properties.getProperty("enable_auto_create_schema",
+            Boolean.toString(conf.isAutoCreateSchemaEnabled()).trim())));
+    conf.setDefaultStorageGroupLevel(
+        Integer.parseInt(properties.getProperty("default_storage_group_level",
+            Integer.toString(conf.getDefaultStorageGroupLevel()))));
+    conf.setDefaultBooleanEncoding(
+        properties.getProperty("default_boolean_encoding",
+            conf.getDefaultBooleanEncoding().toString()));
+    conf.setDefaultInt32Encoding(
+        properties.getProperty("default_int32_encoding",
+            conf.getDefaultInt32Encoding().toString()));
+    conf.setDefaultInt64Encoding(
+        properties.getProperty("default_int64_encoding",
+            conf.getDefaultInt64Encoding().toString()));
+    conf.setDefaultFloatEncoding(
+        properties.getProperty("default_float_encoding",
+            conf.getDefaultFloatEncoding().toString()));
+    conf.setDefaultDoubleEncoding(
+        properties.getProperty("default_double_encoding",
+            conf.getDefaultDoubleEncoding().toString()));
+    conf.setDefaultTextEncoding(
+        properties.getProperty("default_text_encoding",
+            conf.getDefaultTextEncoding().toString()));
+  }
+
+  private void loadTsFileProps(Properties properties){
+    TSFileDescriptor.getInstance().getConfig().setGroupSizeInByte(Integer
+        .parseInt(properties.getProperty("group_size_in_byte",
+            Integer.toString(TSFileDescriptor.getInstance().getConfig().getGroupSizeInByte()))));
+    TSFileDescriptor.getInstance().getConfig().setPageSizeInByte(Integer
+        .parseInt(properties.getProperty("page_size_in_byte",
+            Integer.toString(TSFileDescriptor.getInstance().getConfig().getPageSizeInByte()))));
+    if (TSFileDescriptor.getInstance().getConfig().getPageSizeInByte() > TSFileDescriptor
+        .getInstance().getConfig().getGroupSizeInByte()) {
+      logger
+          .warn("page_size is greater than group size, will set it as the same with group size");
+      TSFileDescriptor.getInstance().getConfig()
+          .setPageSizeInByte(TSFileDescriptor.getInstance().getConfig().getGroupSizeInByte());
+    }
+    TSFileDescriptor.getInstance().getConfig().setMaxNumberOfPointsInPage(Integer
+        .parseInt(properties.getProperty("max_number_of_points_in_page",
+            Integer.toString(
+                TSFileDescriptor.getInstance().getConfig().getMaxNumberOfPointsInPage()))));
+    TSFileDescriptor.getInstance().getConfig().setTimeSeriesDataType(properties
+        .getProperty("time_series_data_type",
+            TSFileDescriptor.getInstance().getConfig().getTimeSeriesDataType()));
+    TSFileDescriptor.getInstance().getConfig().setMaxStringLength(Integer
+        .parseInt(properties.getProperty("max_string_length",
+            Integer.toString(TSFileDescriptor.getInstance().getConfig().getMaxStringLength()))));
+    TSFileDescriptor.getInstance().getConfig().setBloomFilterErrorRate(Double
+        .parseDouble(properties.getProperty("bloom_filter_error_rate",
+            Double.toString(
+                TSFileDescriptor.getInstance().getConfig().getBloomFilterErrorRate()))));
+    TSFileDescriptor.getInstance().getConfig().setFloatPrecision(Integer
+        .parseInt(properties
+            .getProperty("float_precision", Integer
+                .toString(TSFileDescriptor.getInstance().getConfig().getFloatPrecision()))));
+    TSFileDescriptor.getInstance().getConfig().setTimeEncoder(properties
+        .getProperty("time_encoder",
+            TSFileDescriptor.getInstance().getConfig().getTimeEncoder()));
+    TSFileDescriptor.getInstance().getConfig().setValueEncoder(properties
+        .getProperty("value_encoder",
+            TSFileDescriptor.getInstance().getConfig().getValueEncoder()));
+    TSFileDescriptor.getInstance().getConfig().setCompressor(properties
+        .getProperty("compressor", TSFileDescriptor.getInstance().getConfig().getCompressor()));
+  }
+
+  public void loadHotModifiedProps() {
+    String url = getPropsUrl();
+    if (url == null) {
+      return;
+    }
+
+    try (InputStream inputStream = new FileInputStream(new File(url))) {
+      logger.info("Start to reload config file {}", url);
+      Properties properties = new Properties();
+      properties.load(inputStream);
+
+      // update data dirs
+      String dataDirs = properties.getProperty("data_dirs", null);
+      if(dataDirs != null){
+        conf.reloadDataDirs(dataDirs.split(","));
+      }
+
+      // update dir strategy
+      String multiDirStrategyClassName = properties.getProperty("multi_dir_strategy", null);
+      if (multiDirStrategyClassName != null && !multiDirStrategyClassName
+          .equals(conf.getMultiDirStrategyClassName())) {
+        conf.setMultiDirStrategyClassName(multiDirStrategyClassName);
+        DirectoryManager.getInstance().updateFileFolders();
+      }
+
+      // update WAL conf
+      loadWALProps(properties);
+
+      // time zone
+      conf.setZoneID(
+          ZoneId.of(properties.getProperty("time_zone", conf.getZoneID().toString().trim())));
+
+      // dynamic parameters
+      long tsfileSizeThreshold = Long.parseLong(properties
+          .getProperty("tsfile_size_threshold",
+              Long.toString(conf.getTsFileSizeThreshold())).trim());
+      if (tsfileSizeThreshold > 0 && !conf.isEnableParameterAdapter()) {
+        conf.setTsFileSizeThreshold(tsfileSizeThreshold);
+      }
+
+      long memTableSizeThreshold = Long.parseLong(properties
+          .getProperty("memtable_size_threshold",
+              Long.toString(conf.getMemtableSizeThreshold())).trim());
+      if (memTableSizeThreshold > 0 && !conf.isEnableParameterAdapter()) {
+        conf.setMemtableSizeThreshold(memTableSizeThreshold);
       }
+
+      // update params of creating schema automatically
+      loadAutoCreateSchemaProps(properties);
+
+      // update tsfile-format config
+      loadTsFileProps(properties);
+
+    } catch (FileNotFoundException e) {
+      logger.warn("Fail to reload config file {}", url, e);
+    } catch (IOException e) {
+      logger.warn("Cannot reload config file, use default configuration", e);
+    } catch (Exception e) {
+      logger.warn("Incorrect format in config file, use default configuration", e);
     }
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java b/server/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java
index 8c14adb..83e5016 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/directories/DirectoryManager.java
@@ -67,11 +67,33 @@ public class DirectoryManager {
       sequenceStrategy.init(sequenceFileFolders);
       unsequenceStrategy = (DirectoryStrategy) clazz.newInstance();
       unsequenceStrategy.init(unsequenceFileFolders);
+    } catch (DiskSpaceInsufficientException e) {
+      logger.error("All disks of folders are full.", e);
     } catch (Exception e) {
-      logger.error("can't find sequenceStrategy {} for mult-directories.", strategyName, e);
+      logger.error("Can't find sequenceStrategy {} for mult-directories.", strategyName, e);
     }
   }
 
+  public void updateFileFolders() {
+    try {
+      sequenceFileFolders =
+          new ArrayList<>(Arrays.asList(IoTDBDescriptor.getInstance().getConfig().getDataDirs()));
+      mkDataDirs(sequenceFileFolders);
+
+      unsequenceFileFolders =
+          new ArrayList<>(Arrays.asList(IoTDBDescriptor.getInstance().getConfig().getDataDirs()));
+      mkDataDirs(unsequenceFileFolders);
+      sequenceStrategy.init(sequenceFileFolders);
+      unsequenceStrategy.init(unsequenceFileFolders);
+    } catch (DiskSpaceInsufficientException e) {
+      logger.error("All disks of folders are full.", e);
+    }
+  }
+
+  private void updateDirectoryStrategy(){
+
+  }
+
   public static DirectoryManager getInstance() {
     return DirectoriesHolder.INSTANCE;
   }
@@ -147,8 +169,4 @@ public class DirectoryManager {
     return new ArrayList<>(unsequenceFileFolders);
   }
 
-  // only used by test
-  public String getUnSequenceFolderForTest() {
-    return unsequenceFileFolders.get(0);
-  }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/directories/strategy/DirectoryStrategy.java b/server/src/main/java/org/apache/iotdb/db/conf/directories/strategy/DirectoryStrategy.java
index 05f38db..741d379 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/directories/strategy/DirectoryStrategy.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/directories/strategy/DirectoryStrategy.java
@@ -19,6 +19,7 @@
 package org.apache.iotdb.db.conf.directories.strategy;
 
 import java.util.List;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.DiskSpaceInsufficientException;
 import org.apache.iotdb.db.utils.CommonUtils;
 import org.slf4j.Logger;
@@ -54,6 +55,7 @@ public abstract class DirectoryStrategy {
       }
     }
     if (!hasSpace) {
+      IoTDBDescriptor.getInstance().getConfig().setReadOnly(true);
       throw new DiskSpaceInsufficientException(
           String.format("All disks of folders %s are full, can't init.", folders));
     }