You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by hu...@apache.org on 2023/05/14 15:09:26 UTC

[iotdb] branch lmh/forecast created (now 445537b5197)

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

hui pushed a change to branch lmh/forecast
in repository https://gitbox.apache.org/repos/asf/iotdb.git


      at 445537b5197 implement analyzer (tmp save)

This branch includes the following new commits:

     new 4ca7d6db46d add `ModelInferenceFunction`
     new 4d58cbbb681 add semantic check
     new 869ee7059c9 add interface `getModelInformation` in IPartitionFetcher
     new 445537b5197 implement analyzer (tmp save)

The 4 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.



[iotdb] 03/04: add interface `getModelInformation` in IPartitionFetcher

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

hui pushed a commit to branch lmh/forecast
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 869ee7059c9494e112947427c5ac9b3d2da94fa5
Author: liuminghui233 <54...@qq.com>
AuthorDate: Sun May 14 22:36:07 2023 +0800

    add interface `getModelInformation` in IPartitionFetcher
---
 .../mpp/plan/analyze/ClusterPartitionFetcher.java  | 27 ++++++++++++++++++++++
 .../db/mpp/plan/analyze/IPartitionFetcher.java     |  3 +++
 .../mpp/plan/analyze/FakePartitionFetcherImpl.java |  6 +++++
 .../iotdb/db/mpp/plan/plan/distribution/Util.java  |  6 +++++
 4 files changed, 42 insertions(+)

diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ClusterPartitionFetcher.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ClusterPartitionFetcher.java
index 7bf6cfecd31..dbc2e3068b9 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ClusterPartitionFetcher.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ClusterPartitionFetcher.java
@@ -26,6 +26,7 @@ import org.apache.iotdb.commons.client.IClientManager;
 import org.apache.iotdb.commons.client.exception.ClientManagerException;
 import org.apache.iotdb.commons.consensus.ConfigRegionId;
 import org.apache.iotdb.commons.exception.IoTDBException;
+import org.apache.iotdb.commons.model.ModelInformation;
 import org.apache.iotdb.commons.partition.DataPartition;
 import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
 import org.apache.iotdb.commons.partition.SchemaNodeManagementPartition;
@@ -38,6 +39,8 @@ import org.apache.iotdb.confignode.rpc.thrift.TSchemaNodeManagementReq;
 import org.apache.iotdb.confignode.rpc.thrift.TSchemaNodeManagementResp;
 import org.apache.iotdb.confignode.rpc.thrift.TSchemaPartitionReq;
 import org.apache.iotdb.confignode.rpc.thrift.TSchemaPartitionTableResp;
+import org.apache.iotdb.confignode.rpc.thrift.TShowModelReq;
+import org.apache.iotdb.confignode.rpc.thrift.TShowModelResp;
 import org.apache.iotdb.confignode.rpc.thrift.TTimeSlotList;
 import org.apache.iotdb.db.client.ConfigNodeClient;
 import org.apache.iotdb.db.client.ConfigNodeClientManager;
@@ -293,6 +296,30 @@ public class ClusterPartitionFetcher implements IPartitionFetcher {
     partitionCache.invalidAllCache();
   }
 
+  @Override
+  public ModelInformation getModelInformation(String modelId) {
+    try (ConfigNodeClient client =
+        configNodeClientManager.borrowClient(ConfigNodeInfo.CONFIG_REGION_ID)) {
+      TShowModelReq showModelReq = new TShowModelReq();
+      showModelReq.setModelId(modelId);
+      TShowModelResp showModelResp = client.showModel(showModelReq);
+      if (showModelResp.getStatus().getCode() == TSStatusCode.SUCCESS_STATUS.getStatusCode()) {
+        if (showModelResp.modelInfoList.size() > 0) {
+          return ModelInformation.deserialize(showModelResp.modelInfoList.get(0));
+        } else {
+          return null;
+        }
+      } else {
+        throw new StatementAnalyzeException(
+            "An error occurred when executing getModelInformation():"
+                + showModelResp.getStatus().getMessage());
+      }
+    } catch (ClientManagerException | TException e) {
+      throw new StatementAnalyzeException(
+          "An error occurred when executing getModelInformation():" + e.getMessage());
+    }
+  }
+
   /** split data partition query param by database */
   private Map<String, List<DataPartitionQueryParam>> splitDataPartitionQueryParam(
       List<DataPartitionQueryParam> dataPartitionQueryParams, boolean isAutoCreate) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/IPartitionFetcher.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/IPartitionFetcher.java
index e2c4b4a7c88..d50bf23fdb5 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/IPartitionFetcher.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/IPartitionFetcher.java
@@ -18,6 +18,7 @@
  */
 package org.apache.iotdb.db.mpp.plan.analyze;
 
+import org.apache.iotdb.commons.model.ModelInformation;
 import org.apache.iotdb.commons.partition.DataPartition;
 import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
 import org.apache.iotdb.commons.partition.SchemaNodeManagementPartition;
@@ -87,4 +88,6 @@ public interface IPartitionFetcher {
 
   /** Invalid all partition cache */
   void invalidAllCache();
+
+  ModelInformation getModelInformation(String modelId);
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/FakePartitionFetcherImpl.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/FakePartitionFetcherImpl.java
index b15229ca779..01fb0795ac8 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/FakePartitionFetcherImpl.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/analyze/FakePartitionFetcherImpl.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.model.ModelInformation;
 import org.apache.iotdb.commons.partition.DataPartition;
 import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
 import org.apache.iotdb.commons.partition.SchemaNodeManagementPartition;
@@ -298,4 +299,9 @@ public class FakePartitionFetcherImpl implements IPartitionFetcher {
 
   @Override
   public void invalidAllCache() {}
+
+  @Override
+  public ModelInformation getModelInformation(String modelId) {
+    return null;
+  }
 }
diff --git a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/Util.java b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/Util.java
index a8aa156554c..a167129114f 100644
--- a/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/Util.java
+++ b/server/src/test/java/org/apache/iotdb/db/mpp/plan/plan/distribution/Util.java
@@ -27,6 +27,7 @@ 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.exception.IllegalPathException;
+import org.apache.iotdb.commons.model.ModelInformation;
 import org.apache.iotdb.commons.partition.DataPartition;
 import org.apache.iotdb.commons.partition.DataPartitionQueryParam;
 import org.apache.iotdb.commons.partition.SchemaNodeManagementPartition;
@@ -395,6 +396,11 @@ public class Util {
 
       @Override
       public void invalidAllCache() {}
+
+      @Override
+      public ModelInformation getModelInformation(String modelId) {
+        return null;
+      }
     };
   }
 


[iotdb] 02/04: add semantic check

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

hui pushed a commit to branch lmh/forecast
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 4d58cbbb6810d380308d60c779b3f1425a1569ab
Author: liuminghui233 <54...@qq.com>
AuthorDate: Sun May 14 22:18:29 2023 +0800

    add semantic check
---
 .../db/mpp/plan/analyze/ExpressionAnalyzer.java    |  2 ++
 .../mpp/plan/statement/component/ResultColumn.java |  3 +-
 .../plan/statement/component/SelectComponent.java  | 10 +++++-
 .../db/mpp/plan/statement/crud/QueryStatement.java | 37 ++++++++++++++++++++--
 4 files changed, 47 insertions(+), 5 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionAnalyzer.java
index 7b31aad6b9d..f4bd391535c 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionAnalyzer.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/ExpressionAnalyzer.java
@@ -187,6 +187,8 @@ public class ExpressionAnalyzer {
           }
         }
         return ResultColumn.ColumnType.AGGREGATION;
+      } else if (((FunctionExpression) expression).isModelInferenceFunction()) {
+        return ResultColumn.ColumnType.MODEL_INFERENCE;
       } else {
         ResultColumn.ColumnType checkedType = null;
         int lastCheckedIndex = 0;
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/ResultColumn.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/ResultColumn.java
index ae23d5e3d5b..ab535116926 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/ResultColumn.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/ResultColumn.java
@@ -121,6 +121,7 @@ public class ResultColumn extends StatementNode {
   public enum ColumnType {
     RAW,
     AGGREGATION,
-    CONSTANT
+    CONSTANT,
+    MODEL_INFERENCE
   }
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SelectComponent.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SelectComponent.java
index 8961affe57f..decdea7f0c8 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SelectComponent.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/component/SelectComponent.java
@@ -36,6 +36,8 @@ public class SelectComponent extends StatementNode {
 
   private boolean hasBuiltInAggregationFunction = false;
 
+  private boolean hasModelInferenceFunction = false;
+
   protected List<ResultColumn> resultColumns = new ArrayList<>();
 
   private Map<String, Expression> aliasToColumnMap;
@@ -48,15 +50,21 @@ public class SelectComponent extends StatementNode {
     return zoneId;
   }
 
-  public boolean isHasBuiltInAggregationFunction() {
+  public boolean hasBuiltInAggregationFunction() {
     return hasBuiltInAggregationFunction;
   }
 
+  public boolean hasModelInferenceFunction() {
+    return hasModelInferenceFunction;
+  }
+
   public void addResultColumn(ResultColumn resultColumn) {
     resultColumns.add(resultColumn);
     ResultColumn.ColumnType columnType = resultColumn.getColumnType();
     if (columnType == ResultColumn.ColumnType.AGGREGATION) {
       hasBuiltInAggregationFunction = true;
+    } else if (columnType == ResultColumn.ColumnType.MODEL_INFERENCE) {
+      hasModelInferenceFunction = true;
     }
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/QueryStatement.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/QueryStatement.java
index c2cfae97f0d..4f02cc41a77 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/QueryStatement.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/statement/crud/QueryStatement.java
@@ -274,7 +274,7 @@ public class QueryStatement extends Statement {
   }
 
   public boolean isAggregationQuery() {
-    return selectComponent.isHasBuiltInAggregationFunction();
+    return selectComponent.hasBuiltInAggregationFunction();
   }
 
   public boolean isGroupByLevel() {
@@ -407,8 +407,7 @@ public class QueryStatement extends Statement {
     List<SortItem> sortItems = getSortItemList();
     List<SortItem> newSortItems = new ArrayList<>();
     int expressionIndex = 0;
-    for (int i = 0; i < sortItems.size(); i++) {
-      SortItem sortItem = sortItems.get(i);
+    for (SortItem sortItem : sortItems) {
       SortItem newSortItem =
           new SortItem(sortItem.getSortKey(), sortItem.getOrdering(), sortItem.getNullOrdering());
       if (sortItem.isExpression()) {
@@ -448,7 +447,39 @@ public class QueryStatement extends Statement {
     return rowOffset > 0;
   }
 
+  public boolean isModelInferenceQuery() {
+    return selectComponent.hasModelInferenceFunction();
+  }
+
   public void semanticCheck() {
+    if (isModelInferenceQuery()) {
+      if (selectComponent.getResultColumns().size() > 1) {
+        throw new SemanticException("");
+      }
+
+      Expression modelInferenceExpression =
+          selectComponent.getResultColumns().get(0).getExpression();
+      if (!(modelInferenceExpression instanceof FunctionExpression
+          && ((FunctionExpression) modelInferenceExpression).isModelInferenceFunction())) {
+        throw new SemanticException("");
+      }
+      if (ExpressionAnalyzer.searchAggregationExpressions(modelInferenceExpression).size() > 0) {
+        throw new SemanticException("");
+      }
+
+      if (hasHaving()
+          || isGroupBy()
+          || isGroupByLevel()
+          || isGroupByTag()
+          || isAlignByDevice()
+          || isLastQuery()
+          || seriesLimit > 0
+          || seriesOffset > 0
+          || isSelectInto()) {
+        throw new SemanticException("");
+      }
+    }
+
     if (isAggregationQuery()) {
       if (disableAlign()) {
         throw new SemanticException("AGGREGATION doesn't support disable align clause.");


[iotdb] 01/04: add `ModelInferenceFunction`

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

hui pushed a commit to branch lmh/forecast
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 4ca7d6db46d625c0b62b59c04dea6e8cdcd71711
Author: liuminghui233 <54...@qq.com>
AuthorDate: Sun May 14 21:32:39 2023 +0800

    add `ModelInferenceFunction`
---
 .../udf/builtin/ModelInferenceFunction.java        | 34 ++++++++++++++++++----
 .../commons/udf/service/UDFManagementService.java  | 19 ++++++++++++
 .../plan/expression/multi/FunctionExpression.java  | 10 +++++++
 .../db/mpp/plan/expression/multi/FunctionType.java |  3 +-
 4 files changed, 59 insertions(+), 7 deletions(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/multi/FunctionType.java b/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/ModelInferenceFunction.java
similarity index 52%
copy from server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/multi/FunctionType.java
copy to node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/ModelInferenceFunction.java
index 734ebb4bef4..f52a0348af6 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/multi/FunctionType.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/udf/builtin/ModelInferenceFunction.java
@@ -17,11 +17,33 @@
  * under the License.
  */
 
-package org.apache.iotdb.db.mpp.plan.expression.multi;
+package org.apache.iotdb.commons.udf.builtin;
 
-/** */
-public enum FunctionType {
-  AGGREGATION_FUNCTION,
-  BUILT_IN_SCALAR_FUNCTION,
-  UDF
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+public enum ModelInferenceFunction {
+  FORECAST("forecast");
+
+  private final String functionName;
+
+  ModelInferenceFunction(String functionName) {
+    this.functionName = functionName;
+  }
+
+  public String getFunctionName() {
+    return functionName;
+  }
+
+  private static final Set<String> NATIVE_FUNCTION_NAMES =
+      new HashSet<>(
+          Arrays.stream(ModelInferenceFunction.values())
+              .map(ModelInferenceFunction::getFunctionName)
+              .collect(Collectors.toList()));
+
+  public static Set<String> getNativeFunctionNames() {
+    return NATIVE_FUNCTION_NAMES;
+  }
 }
diff --git a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFManagementService.java b/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFManagementService.java
index fbdf8684e91..e7ecb32f008 100644
--- a/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFManagementService.java
+++ b/node-commons/src/main/java/org/apache/iotdb/commons/udf/service/UDFManagementService.java
@@ -22,6 +22,7 @@ package org.apache.iotdb.commons.udf.service;
 import org.apache.iotdb.commons.udf.UDFInformation;
 import org.apache.iotdb.commons.udf.UDFTable;
 import org.apache.iotdb.commons.udf.builtin.BuiltinAggregationFunction;
+import org.apache.iotdb.commons.udf.builtin.ModelInferenceFunction;
 import org.apache.iotdb.commons.utils.TestOnly;
 import org.apache.iotdb.udf.api.UDF;
 import org.apache.iotdb.udf.api.UDTF;
@@ -111,8 +112,26 @@ public class UDFManagementService {
     throw new UDFManagementException(errorMessage);
   }
 
+  private void checkIsModelInferenceFunctionName(UDFInformation udfInformation)
+      throws UDFManagementException {
+    String functionName = udfInformation.getFunctionName();
+    String className = udfInformation.getClassName();
+    if (!ModelInferenceFunction.getNativeFunctionNames().contains(functionName.toLowerCase())) {
+      return;
+    }
+
+    String errorMessage =
+        String.format(
+            "Failed to register UDF %s(%s), because the given function name conflicts with the ML model inference function name",
+            functionName, className);
+
+    LOGGER.warn(errorMessage);
+    throw new UDFManagementException(errorMessage);
+  }
+
   private void checkIfRegistered(UDFInformation udfInformation) throws UDFManagementException {
     checkIsBuiltInAggregationFunctionName(udfInformation);
+    checkIsModelInferenceFunctionName(udfInformation);
     String functionName = udfInformation.getFunctionName();
     String className = udfInformation.getClassName();
     UDFInformation information = udfTable.getUDFInformation(functionName);
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/multi/FunctionExpression.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/multi/FunctionExpression.java
index 6a36dc93c9a..b9b8960cb48 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/multi/FunctionExpression.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/multi/FunctionExpression.java
@@ -23,6 +23,7 @@ import org.apache.iotdb.commons.conf.IoTDBConstant;
 import org.apache.iotdb.commons.path.PartialPath;
 import org.apache.iotdb.commons.udf.builtin.BuiltinAggregationFunction;
 import org.apache.iotdb.commons.udf.builtin.BuiltinScalarFunction;
+import org.apache.iotdb.commons.udf.builtin.ModelInferenceFunction;
 import org.apache.iotdb.db.mpp.common.NodeRef;
 import org.apache.iotdb.db.mpp.plan.expression.Expression;
 import org.apache.iotdb.db.mpp.plan.expression.ExpressionType;
@@ -105,6 +106,8 @@ public class FunctionExpression extends Expression {
       functionType = FunctionType.AGGREGATION_FUNCTION;
     } else if (BuiltinScalarFunction.getNativeFunctionNames().contains(functionName)) {
       functionType = FunctionType.BUILT_IN_SCALAR_FUNCTION;
+    } else if (ModelInferenceFunction.getNativeFunctionNames().contains(functionName)) {
+      functionType = FunctionType.MODEL_INFERENCE_FUNCTION;
     } else {
       functionType = FunctionType.UDF;
     }
@@ -125,6 +128,13 @@ public class FunctionExpression extends Expression {
     return functionType == FunctionType.BUILT_IN_SCALAR_FUNCTION;
   }
 
+  public boolean isModelInferenceFunction() {
+    if (functionType == null) {
+      initializeFunctionType();
+    }
+    return functionType == FunctionType.MODEL_INFERENCE_FUNCTION;
+  }
+
   @Override
   public boolean isConstantOperandInternal() {
     if (isConstantOperandCache == null) {
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/multi/FunctionType.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/multi/FunctionType.java
index 734ebb4bef4..c7e9d4ade2c 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/multi/FunctionType.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/expression/multi/FunctionType.java
@@ -23,5 +23,6 @@ package org.apache.iotdb.db.mpp.plan.expression.multi;
 public enum FunctionType {
   AGGREGATION_FUNCTION,
   BUILT_IN_SCALAR_FUNCTION,
-  UDF
+  UDF,
+  MODEL_INFERENCE_FUNCTION
 }


[iotdb] 04/04: implement analyzer (tmp save)

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

hui pushed a commit to branch lmh/forecast
in repository https://gitbox.apache.org/repos/asf/iotdb.git

commit 445537b5197319931be98d390b2ae4d7dc624b71
Author: liuminghui233 <54...@qq.com>
AuthorDate: Sun May 14 23:05:51 2023 +0800

    implement analyzer (tmp save)
---
 .../iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java  |  8 +++++++-
 .../plan/parameter/ModelInferenceDescriptor.java   | 22 ++++++++++++++++++++++
 2 files changed, 29 insertions(+), 1 deletion(-)

diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
index 4b4a89ff48b..973c02a9602 100644
--- a/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/analyze/AnalyzeVisitor.java
@@ -231,12 +231,16 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
     try {
       // check for semantic errors
       queryStatement.semanticCheck();
+      analysis.setStatement(queryStatement);
+
+      if (queryStatement.isModelInferenceQuery()) {
+        analyzeModelInference(analysis, queryStatement);
+      }
 
       // concat path and construct path pattern tree
       PathPatternTree patternTree = new PathPatternTree();
       queryStatement =
           (QueryStatement) new ConcatPathRewriter().rewrite(queryStatement, patternTree);
-      analysis.setStatement(queryStatement);
 
       // request schema fetch API
       logger.debug("[StartFetchSchema]");
@@ -357,6 +361,8 @@ public class AnalyzeVisitor extends StatementVisitor<Analysis, MPPQueryContext>
     return analysis;
   }
 
+  private void analyzeModelInference(Analysis analysis, QueryStatement queryStatement) {}
+
   private Analysis finishQuery(QueryStatement queryStatement, Analysis analysis) {
     if (queryStatement.isSelectInto()) {
       analysis.setRespDatasetHeader(
diff --git a/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/ModelInferenceDescriptor.java b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/ModelInferenceDescriptor.java
new file mode 100644
index 00000000000..9e2c7dc1817
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/mpp/plan/planner/plan/parameter/ModelInferenceDescriptor.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.db.mpp.plan.planner.plan.parameter;
+
+public class ModelInferenceDescriptor {}