diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index 31f446df27031..18653648c2ca2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -615,6 +615,32 @@ public class IoTDBConfig { /** the max time to live of a session in ms. Unit: millisecond */ private int sessionTimeoutThreshold = 0; + // ================== Smart Plan Cache Configuration ================== + + /** Maximum number of cached plan entries. */ + private int smartPlanCacheCapacity = 1000; + + /** Maximum memory in bytes for plan cache. */ + private long smartPlanCacheMaxMemoryBytes = 64L * 1024 * 1024; + + /** Minimum sample count before state transition from MONITOR. */ + private int smartPlanCacheMinSamples = 5; + + /** Minimum reusable planning cost (in nanoseconds) to be considered for caching. */ + private long smartPlanCacheMinReusablePlanningCostNanos = 1_000_000L; + + /** Benefit ratio threshold for MONITOR → ACTIVE transition. */ + private double smartPlanCacheAdmitRatio = 0.30d; + + /** Benefit ratio threshold for MONITOR → BYPASS transition. */ + private double smartPlanCacheBypassRatio = 0.20d; + + /** Cooldown time in minutes for BYPASS state before returning to MONITOR. */ + private long smartPlanCacheBypassCooldownMinutes = 10L; + + /** Default plan cache mode: OFF, MONITOR, AUTO, FORCE. */ + private String smartPlanCacheMode = "AUTO"; + /** Replace implementation class of JDBC service */ private String rpcImplClassName = ClientRPCServiceImpl.class.getName(); @@ -1914,6 +1940,73 @@ public void setQueryTimeoutThreshold(long queryTimeoutThreshold) { this.queryTimeoutThreshold = queryTimeoutThreshold; } + // ================== Smart Plan Cache Getters/Setters ================== + + public int getSmartPlanCacheCapacity() { + return smartPlanCacheCapacity; + } + + public void setSmartPlanCacheCapacity(int smartPlanCacheCapacity) { + this.smartPlanCacheCapacity = smartPlanCacheCapacity; + } + + public long getSmartPlanCacheMaxMemoryBytes() { + return smartPlanCacheMaxMemoryBytes; + } + + public void setSmartPlanCacheMaxMemoryBytes(long smartPlanCacheMaxMemoryBytes) { + this.smartPlanCacheMaxMemoryBytes = smartPlanCacheMaxMemoryBytes; + } + + public int getSmartPlanCacheMinSamples() { + return smartPlanCacheMinSamples; + } + + public void setSmartPlanCacheMinSamples(int smartPlanCacheMinSamples) { + this.smartPlanCacheMinSamples = smartPlanCacheMinSamples; + } + + public long getSmartPlanCacheMinReusablePlanningCostNanos() { + return smartPlanCacheMinReusablePlanningCostNanos; + } + + public void setSmartPlanCacheMinReusablePlanningCostNanos( + long smartPlanCacheMinReusablePlanningCostNanos) { + this.smartPlanCacheMinReusablePlanningCostNanos = smartPlanCacheMinReusablePlanningCostNanos; + } + + public double getSmartPlanCacheAdmitRatio() { + return smartPlanCacheAdmitRatio; + } + + public void setSmartPlanCacheAdmitRatio(double smartPlanCacheAdmitRatio) { + this.smartPlanCacheAdmitRatio = smartPlanCacheAdmitRatio; + } + + public double getSmartPlanCacheBypassRatio() { + return smartPlanCacheBypassRatio; + } + + public void setSmartPlanCacheBypassRatio(double smartPlanCacheBypassRatio) { + this.smartPlanCacheBypassRatio = smartPlanCacheBypassRatio; + } + + public long getSmartPlanCacheBypassCooldownMinutes() { + return smartPlanCacheBypassCooldownMinutes; + } + + public void setSmartPlanCacheBypassCooldownMinutes(long smartPlanCacheBypassCooldownMinutes) { + this.smartPlanCacheBypassCooldownMinutes = smartPlanCacheBypassCooldownMinutes; + } + + public String getSmartPlanCacheMode() { + return smartPlanCacheMode; + } + + public void setSmartPlanCacheMode(String smartPlanCacheMode) { + this.smartPlanCacheMode = smartPlanCacheMode; + } + public int getSessionTimeoutThreshold() { return sessionTimeoutThreshold; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index 8fb45106d1a68..899f81865c138 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -484,6 +484,52 @@ public void loadProperties(TrimProperties properties) throws BadNodeUrlException properties.getProperty( "query_timeout_threshold", Long.toString(conf.getQueryTimeoutThreshold())))); + // Smart Plan Cache configuration + conf.setSmartPlanCacheCapacity( + Integer.parseInt( + properties.getProperty( + "smart_plan_cache_capacity", + Integer.toString(conf.getSmartPlanCacheCapacity())))); + + conf.setSmartPlanCacheMaxMemoryBytes( + Long.parseLong( + properties.getProperty( + "smart_plan_cache_max_memory_bytes", + Long.toString(conf.getSmartPlanCacheMaxMemoryBytes())))); + + conf.setSmartPlanCacheMinSamples( + Integer.parseInt( + properties.getProperty( + "smart_plan_cache_min_samples", + Integer.toString(conf.getSmartPlanCacheMinSamples())))); + + conf.setSmartPlanCacheMinReusablePlanningCostNanos( + Long.parseLong( + properties.getProperty( + "smart_plan_cache_min_reusable_planning_cost_nanos", + Long.toString(conf.getSmartPlanCacheMinReusablePlanningCostNanos())))); + + conf.setSmartPlanCacheAdmitRatio( + Double.parseDouble( + properties.getProperty( + "smart_plan_cache_admit_ratio", + Double.toString(conf.getSmartPlanCacheAdmitRatio())))); + + conf.setSmartPlanCacheBypassRatio( + Double.parseDouble( + properties.getProperty( + "smart_plan_cache_bypass_ratio", + Double.toString(conf.getSmartPlanCacheBypassRatio())))); + + conf.setSmartPlanCacheBypassCooldownMinutes( + Long.parseLong( + properties.getProperty( + "smart_plan_cache_bypass_cooldown_minutes", + Long.toString(conf.getSmartPlanCacheBypassCooldownMinutes())))); + + conf.setSmartPlanCacheMode( + properties.getProperty("smart_plan_cache_mode", conf.getSmartPlanCacheMode())); + conf.setSessionTimeoutThreshold( Integer.parseInt( properties.getProperty( diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java index 88bd1998f683c..249395af3f141 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/common/MPPQueryContext.java @@ -34,6 +34,9 @@ import org.apache.iotdb.db.queryengine.plan.planner.memory.MemoryReservationManager; import org.apache.iotdb.db.queryengine.plan.planner.memory.NotThreadSafeMemoryReservationManager; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.NodeRef; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.db.queryengine.plan.relational.planner.Symbol; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Identifier; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; @@ -45,6 +48,7 @@ import org.apache.tsfile.utils.Pair; import java.time.ZoneId; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -445,6 +449,172 @@ public void setLogicalOptimizationCost(long logicalOptimizeCost) { queryPlanStatistics.setLogicalOptimizationCost(logicalOptimizeCost); } + public void setPlanCacheStatus(String planCacheStatus) { + if (queryPlanStatistics == null) { + queryPlanStatistics = new QueryPlanStatistics(); + } + queryPlanStatistics.setPlanCacheStatus(planCacheStatus); + } + + public void setPlanCacheStatus(String planCacheStatus, String planCacheReason) { + if (queryPlanStatistics == null) { + queryPlanStatistics = new QueryPlanStatistics(); + } + queryPlanStatistics.setPlanCacheStatus(planCacheStatus, planCacheReason); + } + + public String getPlanCacheStatus() { + if (queryPlanStatistics == null) { + return "DISABLED"; + } + return queryPlanStatistics.getPlanCacheStatus(); + } + + public void setPlanCacheState(String planCacheState) { + if (queryPlanStatistics == null) { + queryPlanStatistics = new QueryPlanStatistics(); + } + queryPlanStatistics.setPlanCacheState(planCacheState); + } + + public String getPlanCacheState() { + if (queryPlanStatistics == null) { + return "N/A"; + } + return queryPlanStatistics.getPlanCacheState(); + } + + public String getPlanCacheReason() { + if (queryPlanStatistics == null) { + return ""; + } + return queryPlanStatistics.getPlanCacheReason(); + } + + public void setPlanCacheLookupCost(long planCacheLookupCost) { + if (queryPlanStatistics == null) { + queryPlanStatistics = new QueryPlanStatistics(); + } + queryPlanStatistics.setPlanCacheLookupCost(planCacheLookupCost); + } + + public long getPlanCacheLookupCost() { + if (queryPlanStatistics == null) { + return 0; + } + return queryPlanStatistics.getPlanCacheLookupCost(); + } + + public void setSavedLogicalPlanningCost(long savedLogicalPlanningCost) { + if (queryPlanStatistics == null) { + queryPlanStatistics = new QueryPlanStatistics(); + } + queryPlanStatistics.setSavedLogicalPlanningCost(savedLogicalPlanningCost); + } + + public long getSavedLogicalPlanningCost() { + if (queryPlanStatistics == null) { + return 0; + } + return queryPlanStatistics.getSavedLogicalPlanningCost(); + } + + public void setReusablePlanningCost(long reusablePlanningCost) { + if (queryPlanStatistics == null) { + queryPlanStatistics = new QueryPlanStatistics(); + } + queryPlanStatistics.setReusablePlanningCost(reusablePlanningCost); + } + + public long getReusablePlanningCost() { + if (queryPlanStatistics == null) { + return 0; + } + return queryPlanStatistics.getReusablePlanningCost(); + } + + public void setFirstResponseLatency(long firstResponseLatency) { + if (queryPlanStatistics == null) { + queryPlanStatistics = new QueryPlanStatistics(); + } + queryPlanStatistics.setFirstResponseLatency(firstResponseLatency); + } + + public long getFirstResponseLatency() { + if (queryPlanStatistics == null) { + return 0; + } + return queryPlanStatistics.getFirstResponseLatency(); + } + + public void setPlanCacheDiagnostics( + double ewmaReusablePlanningCost, + double ewmaFirstResponseLatency, + double ewmaBenefitRatio, + long sampleCount, + long hitCount, + long missCount, + long bypassCount, + long minReusablePlanningCostThreshold, + double admitRatioThreshold, + double bypassRatioThreshold) { + if (queryPlanStatistics == null) { + queryPlanStatistics = new QueryPlanStatistics(); + } + queryPlanStatistics.setEwmaReusablePlanningCost(ewmaReusablePlanningCost); + queryPlanStatistics.setEwmaFirstResponseLatency(ewmaFirstResponseLatency); + queryPlanStatistics.setEwmaBenefitRatio(ewmaBenefitRatio); + queryPlanStatistics.setProfileSampleCount(sampleCount); + queryPlanStatistics.setProfileHitCount(hitCount); + queryPlanStatistics.setProfileMissCount(missCount); + queryPlanStatistics.setProfileBypassCount(bypassCount); + queryPlanStatistics.setMinReusablePlanningCostThreshold(minReusablePlanningCostThreshold); + queryPlanStatistics.setAdmitRatioThreshold(admitRatioThreshold); + queryPlanStatistics.setBypassRatioThreshold(bypassRatioThreshold); + } + + public double getEwmaReusablePlanningCost() { + return queryPlanStatistics == null ? 0 : queryPlanStatistics.getEwmaReusablePlanningCost(); + } + + public double getEwmaFirstResponseLatency() { + return queryPlanStatistics == null ? 0 : queryPlanStatistics.getEwmaFirstResponseLatency(); + } + + public double getEwmaBenefitRatio() { + return queryPlanStatistics == null ? 0 : queryPlanStatistics.getEwmaBenefitRatio(); + } + + public long getProfileSampleCount() { + return queryPlanStatistics == null ? 0 : queryPlanStatistics.getProfileSampleCount(); + } + + public long getProfileHitCount() { + return queryPlanStatistics == null ? 0 : queryPlanStatistics.getProfileHitCount(); + } + + public long getProfileMissCount() { + return queryPlanStatistics == null ? 0 : queryPlanStatistics.getProfileMissCount(); + } + + public long getProfileBypassCount() { + return queryPlanStatistics == null ? 0 : queryPlanStatistics.getProfileBypassCount(); + } + + public long getMinReusablePlanningCostThreshold() { + return queryPlanStatistics == null + ? 0 + : queryPlanStatistics.getMinReusablePlanningCostThreshold(); + } + + public double getAdmitRatioThreshold() { + return queryPlanStatistics == null ? 0 : queryPlanStatistics.getAdmitRatioThreshold(); + } + + public double getBypassRatioThreshold() { + return queryPlanStatistics == null ? 0 : queryPlanStatistics.getBypassRatioThreshold(); + } + // region =========== FE memory related, make sure its not called concurrently =========== /** @@ -675,4 +845,58 @@ public IAuditEntity setSqlString(String sqlString) { } // ================= Authentication Interfaces ========================= + + // ================== Plan cache related ================== + + /** The cache key (fingerprint) for this query, set during planning. Empty if not cacheable. */ + private String planCacheCachedKey = ""; + + /** Whether a cache lookup was attempted for this query during planning. */ + private boolean planCacheLookupAttempted = false; + + // the outer list corresponds one-to-one with scanNodes + private final List> metadataExpressionLists = new ArrayList<>(); + private final List> attributeColumnsLists = new ArrayList<>(); + private final List> assignmentsLists = new ArrayList<>(); + + public List> getMetadataExpressionLists() { + return metadataExpressionLists; + } + + public List> getAttributeColumnsLists() { + return attributeColumnsLists; + } + + public List> getAssignmentsLists() { + return assignmentsLists; + } + + public void addMetadataExpressionList(List expressions) { + metadataExpressionLists.add( + expressions != null ? new ArrayList<>(expressions) : Collections.emptyList()); + } + + public void addAttributeColumnsList(List columns) { + attributeColumnsLists.add(columns != null ? new ArrayList<>(columns) : Collections.emptyList()); + } + + public void addAssignmentsList(Map assignments) { + assignmentsLists.add(assignments != null ? assignments : Collections.emptyMap()); + } + + public String getPlanCacheCachedKey() { + return planCacheCachedKey; + } + + public void setPlanCacheCachedKey(String planCacheCachedKey) { + this.planCacheCachedKey = planCacheCachedKey; + } + + public boolean isPlanCacheLookupAttempted() { + return planCacheLookupAttempted; + } + + public void setPlanCacheLookupAttempted(boolean planCacheLookupAttempted) { + this.planCacheLookupAttempted = planCacheLookupAttempted; + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java index b4f1a5c7261bf..5a947fa6dd6db 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/execution/operator/ExplainAnalyzeOperator.java @@ -102,7 +102,7 @@ public ExplainAnalyzeOperator( QueryExecution queryExecution = (QueryExecution) coordinator.getQueryExecution(queryId); this.instances = queryExecution.getDistributedPlan().getInstances(); mppQueryContext = queryExecution.getContext(); - fragmentInstanceStatisticsDrawer.renderPlanStatistics(mppQueryContext); + fragmentInstanceStatisticsDrawer.renderPlanStatistics(mppQueryContext, verbose); // The time interval guarantees the result of EXPLAIN ANALYZE will be printed at least three // times. diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java index 7500fbb9c34db..ab49d38bfdfbe 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/execution/QueryExecution.java @@ -48,6 +48,7 @@ import org.apache.iotdb.db.queryengine.plan.planner.plan.FragmentInstance; import org.apache.iotdb.db.queryengine.plan.planner.plan.LogicalQueryPlan; import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNodeUtil; +import org.apache.iotdb.db.queryengine.plan.relational.planner.PlanCacheManager; import org.apache.iotdb.db.queryengine.plan.scheduler.IScheduler; import org.apache.iotdb.db.utils.SetThreadName; import org.apache.iotdb.mpp.rpc.thrift.TFragmentInstanceId; @@ -696,10 +697,30 @@ public long getStartExecutionTime() { @Override public synchronized void recordExecutionTime(long executionTime) { + boolean isFirstRpc = (totalExecutionTime == 0); totalExecutionTime += executionTime; // recordExecutionTime is called after current rpc finished, so we need to set // startTimeOfCurrentRpc to -1 this.startTimeOfCurrentRpc = -1; + + // On the first RPC completion, use the real server-side RPC time as firstResponseLatency + // and feed it to PlanCacheManager for accurate benefit ratio calculation. + if (isFirstRpc) { + String cachedKey = context.getPlanCacheCachedKey(); + if (cachedKey != null && !cachedKey.isEmpty()) { + long reusablePlanningCost = context.getReusablePlanningCost(); + boolean lookupAttempted = context.isPlanCacheLookupAttempted(); + + // executionTime is the total server-side time for the first executeQueryStatement RPC, + // which is the real firstResponseLatency (from coordinator entry to first response). + context.setFirstResponseLatency(executionTime); + + PlanCacheManager.PlanCacheState newState = + PlanCacheManager.getInstance() + .recordExecution(cachedKey, reusablePlanningCost, executionTime, lookupAttempted); + context.setPlanCacheState(newState.name()); + } + } } @Override diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CachedValue.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CachedValue.java new file mode 100644 index 0000000000000..426526aa98c43 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/CachedValue.java @@ -0,0 +1,671 @@ +/* + * 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.queryengine.plan.relational.planner; + +import org.apache.iotdb.db.queryengine.common.QueryId; +import org.apache.iotdb.db.queryengine.common.header.DatasetHeader; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanVisitor; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.db.queryengine.plan.relational.planner.iterative.GroupReference; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.AggregationTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ApplyNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CorrelatedJoinNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.GroupNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.JoinNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.LinearFillNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.OffsetNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.OutputNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.PatternRecognitionNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.PreviousFillNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ProjectNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.SemiJoinNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.SortNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableFunctionNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TableFunctionProcessorNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TopKNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeDeviceViewScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ValueFillNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.WindowNode; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.AstVisitor; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ComparisonExpression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Literal; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LogicalExpression; + +import com.google.common.collect.ImmutableList; +import org.apache.tsfile.read.common.type.Type; +import org.apache.tsfile.utils.RamUsageEstimator; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +public class CachedValue { + private static final int INSTANCE_SIZE = + (int) RamUsageEstimator.shallowSizeOfInstance(CachedValue.class); + + PlanNode planNode; + List scanNodes; + + DatasetHeader respHeader; + HashMap symbolMap; + int symbolNextId; + + // Used for indexScan to fetch device + List> metadataExpressionLists; + List> attributeColumnsLists; + List> assignmentsLists; + + List literalReference; + + long estimatedMemoryUsage; + + public CachedValue( + PlanNode planNode, + List scanNodes, + List literalReference, + DatasetHeader header, + HashMap symbolMap, + int symbolNextId, + List> metadataExpressionLists, + List> attributeColumnsLists, + List> assignmentsLists) { + this.planNode = planNode; + this.scanNodes = scanNodes; + this.respHeader = header; + this.symbolMap = symbolMap; + this.symbolNextId = symbolNextId; + this.metadataExpressionLists = metadataExpressionLists; + this.attributeColumnsLists = attributeColumnsLists; + this.assignmentsLists = assignmentsLists; + this.literalReference = literalReference; + this.estimatedMemoryUsage = estimateMemoryUsage(); + } + + public DatasetHeader getRespHeader() { + return respHeader; + } + + public PlanNode getPlanNode() { + return planNode; + } + + public List getScanNodes() { + return scanNodes; + } + + public HashMap getSymbolMap() { + return symbolMap; + } + + public int getSymbolNextId() { + return symbolNextId; + } + + public List> getMetadataExpressionLists() { + return metadataExpressionLists; + } + + public List> getAttributeColumnsLists() { + return attributeColumnsLists; + } + + public List> getAssignmentsLists() { + return assignmentsLists; + } + + public List getLiteralReference() { + return literalReference; + } + + /** Clone a new planNode using the new literal list */ + public static PlanNode clonePlanWithNewLiterals(PlanNode node, ClonerContext context) { + return node.accept(new PlanNodeCloner(), context); + } + + /** Clone new metadataExpressions using the new literal list */ + public static List cloneMetadataExpressions( + List metadataExpressionList, List newLiterals) { + if (metadataExpressionList == null) { + return null; + } + List clonedList = new ArrayList<>(metadataExpressionList.size()); + ExpressionCloner exprCloner = new ExpressionCloner(); + for (Expression expr : metadataExpressionList) { + clonedList.add(expr.accept(exprCloner, newLiterals)); + } + return clonedList; + } + + /** + * ExpressionCloner is responsible for deep cloning SQL Expression trees. It replaces Literal + * nodes. + */ + private static class ExpressionCloner extends AstVisitor> { + @Override + protected Expression visitExpression(Expression node, List context) { + // Default case, just return the node itself + return node; + } + + @Override + protected Expression visitLiteral(Literal node, List context) { + int idx = node.getLiteralIndex(); + if (idx >= 0 && idx < context.size()) { + return context.get(idx); + } + return node; + } + + @Override + protected Expression visitComparisonExpression( + ComparisonExpression node, List context) { + return new ComparisonExpression( + node.getOperator(), + node.getLeft().accept(this, context), + node.getRight().accept(this, context)); + } + + @Override + protected Expression visitLogicalExpression(LogicalExpression node, List context) { + List newTerms = new ArrayList<>(); + for (Expression term : node.getTerms()) { + newTerms.add(term.accept(this, context)); + } + return new LogicalExpression(node.getOperator(), newTerms); + } + + // FunctionCall, Between, InPredicate, etc + } + + private static class PlanNodeCloner extends PlanVisitor { + private final ExpressionCloner exprCloner = new ExpressionCloner(); + + @Override + public PlanNode visitPlan(PlanNode node, ClonerContext context) { + throw new UnsupportedOperationException( + "visitNode of Node {" + node.getClass() + "} is not supported in PlanNodeCloner"); + } + + @Override + public PlanNode visitOutput(OutputNode node, ClonerContext context) { + PlanNode newChild = node.getChild().accept(this, context); + + return new OutputNode( + context.getQueryId().genPlanNodeId(), + newChild, + node.getColumnNames(), + node.getOutputSymbols()); + } + + @Override + public PlanNode visitSort(SortNode node, ClonerContext context) { + PlanNode newChild = node.getChild().accept(this, context); + + return new SortNode( + context.getQueryId().genPlanNodeId(), + newChild, + node.getOrderingScheme(), + node.isPartial(), + node.isOrderByAllIdsAndTime()); + } + + @Override + public PlanNode visitFilter(FilterNode node, ClonerContext context) { + Expression newPredicate = node.getPredicate().accept(exprCloner, context.getNewLiterals()); + PlanNode newChild = node.getChild().accept(this, context); + return new FilterNode(context.getQueryId().genPlanNodeId(), newChild, newPredicate); + } + + @Override + public PlanNode visitProject(ProjectNode node, ClonerContext context) { + PlanNode newChild = node.getChild().accept(this, context); + + Map newAssignmentsMap = new HashMap<>(); + for (Map.Entry entry : node.getAssignments().entrySet()) { + Expression clonedExpr = + entry.getValue().accept(new ExpressionCloner(), context.getNewLiterals()); + newAssignmentsMap.put(entry.getKey(), clonedExpr); + } + Assignments newAssignments = new Assignments(newAssignmentsMap); + + return new ProjectNode(context.getQueryId().genPlanNodeId(), newChild, newAssignments); + } + + @Override + public PlanNode visitJoin(JoinNode node, ClonerContext context) { + PlanNode newLeft = node.getLeftChild().accept(this, context); + PlanNode newRight = node.getRightChild().accept(this, context); + + Optional newFilter = + node.getFilter().map(expr -> expr.accept(exprCloner, context.getNewLiterals())); + + return new JoinNode( + context.getQueryId().genPlanNodeId(), + node.getJoinType(), + newLeft, + newRight, + ImmutableList.copyOf(node.getCriteria()), + node.getAsofCriteria(), + ImmutableList.copyOf(node.getLeftOutputSymbols()), + ImmutableList.copyOf(node.getRightOutputSymbols()), + newFilter, + node.isSpillable()); + } + + @Override + public PlanNode visitDeviceTableScan(DeviceTableScanNode node, ClonerContext context) { + // deep copy pushDownPredicate + Expression newPredicate = + node.getPushDownPredicate() == null + ? null + : node.getPushDownPredicate() + .accept(new ExpressionCloner(), context.getNewLiterals()); + + // deep copy timePredicate + Expression newTimePredicate = + node.getTimePredicate() + .map(tp -> tp.accept(new ExpressionCloner(), context.getNewLiterals())) + .orElse(null); + + DeviceTableScanNode newNode = + new DeviceTableScanNode( + context.getQueryId().genPlanNodeId(), + node.getQualifiedObjectName(), + node.getOutputSymbols(), + node.getAssignments(), + node.getDeviceEntries(), + node.getTagAndAttributeIndexMap(), + node.getScanOrder(), + newTimePredicate, + newPredicate, + node.getPushDownLimit(), + node.getPushDownOffset(), + node.isPushLimitToEachDevice(), + node.containsNonAlignedDevice()); + + return newNode; + } + + @Override + public PlanNode visitAggregationTableScan( + AggregationTableScanNode node, ClonerContext context) { + // deep copy pushDownPredicate + Expression newPredicate = + node.getPushDownPredicate() == null + ? null + : node.getPushDownPredicate() + .accept(new ExpressionCloner(), context.getNewLiterals()); + + // deep copy timePredicate + Expression newTimePredicate = + node.getTimePredicate() + .map(tp -> tp.accept(new ExpressionCloner(), context.getNewLiterals())) + .orElse(null); + + // deep copy projection + Assignments newProjection = node.getProjection() == null ? null : node.getProjection(); + + // deep copy aggregations + Map newAggregations = new HashMap<>(); + for (Map.Entry entry : + node.getAggregations().entrySet()) { + AggregationNode.Aggregation agg = entry.getValue(); + AggregationNode.Aggregation newAgg = + new AggregationNode.Aggregation( + agg.getResolvedFunction(), + agg.getArguments().stream() + .map(arg -> arg.accept(new ExpressionCloner(), context.getNewLiterals())) + .collect(Collectors.toList()), + agg.isDistinct(), + agg.getFilter(), + agg.getOrderingScheme(), + agg.getMask()); + newAggregations.put(entry.getKey(), newAgg); + } + + // deep copy groupingSets + AggregationNode.GroupingSetDescriptor newGroupingSets = node.getGroupingSets(); + + // deep copy preGroupedSymbols + List newPreGroupedSymbols = ImmutableList.copyOf(node.getPreGroupedSymbols()); + + return new AggregationTableScanNode( + context.getQueryId().genPlanNodeId(), + node.getQualifiedObjectName(), + node.getOutputSymbols(), + node.getAssignments(), + node.getDeviceEntries(), + node.getTagAndAttributeIndexMap(), + node.getScanOrder(), + newTimePredicate, + newPredicate, + node.getPushDownLimit(), + node.getPushDownOffset(), + node.isPushLimitToEachDevice(), + node.containsNonAlignedDevice(), + newProjection, + newAggregations, + newGroupingSets, + newPreGroupedSymbols, + node.getStep(), + node.getGroupIdSymbol()); + } + + @Override + public PlanNode visitPatternRecognition(PatternRecognitionNode node, ClonerContext context) { + PlanNode newChild = node.getChild().accept(this, context); + PatternRecognitionNode newNode = + new PatternRecognitionNode( + context.getQueryId().genPlanNodeId(), // 🔑 新的 PlanNodeId + newChild, + node.getPartitionBy(), + node.getOrderingScheme(), + node.getHashSymbol(), + node.getMeasures(), + node.getRowsPerMatch(), + node.getSkipToLabels(), + node.getSkipToPosition(), + node.getPattern(), + node.getVariableDefinitions()); + return newNode; + } + + @Override + public PlanNode visitAggregation(AggregationNode node, ClonerContext context) { + PlanNode newChild = node.getChild().accept(this, context); + + return new AggregationNode( + context.getQueryId().genPlanNodeId(), + newChild, + node.getAggregations(), + node.getGroupingSets(), + node.getPreGroupedSymbols(), + node.getStep(), + node.getHashSymbol(), + node.getGroupIdSymbol()); + } + + @Override + public PlanNode visitWindowFunction(WindowNode node, ClonerContext context) { + PlanNode newChild = node.getChild().accept(this, context); + + return new WindowNode( + context.getQueryId().genPlanNodeId(), + newChild, + node.getSpecification(), + node.getWindowFunctions(), + node.getHashSymbol(), + node.getPrePartitionedInputs(), + node.getPreSortedOrderPrefix()); + } + + @Override + public PlanNode visitTableFunction(TableFunctionNode node, ClonerContext context) { + List newChildren = new ArrayList<>(); + for (PlanNode child : node.getChildren()) { + newChildren.add(child.accept(this, context)); + } + + return new TableFunctionNode( + context.getQueryId().genPlanNodeId(), + node.getName(), + node.getTableFunctionHandle(), + node.getProperOutputs(), + newChildren, + node.getTableArgumentProperties()); + } + + @Override + public PlanNode visitTableFunctionProcessor( + TableFunctionProcessorNode node, ClonerContext context) { + PlanNode newChild = node.getChild().accept(this, context); + + return new TableFunctionProcessorNode( + context.getQueryId().genPlanNodeId(), + node.getName(), + node.getProperOutputs(), + Optional.ofNullable(newChild), + node.getPassThroughSpecification(), + node.getRequiredSymbols(), + node.getDataOrganizationSpecification(), + node.isRowSemantic(), + node.getTableFunctionHandle(), + node.isRequireRecordSnapshot()); + } + + @Override + public PlanNode visitOffset(OffsetNode node, ClonerContext context) { + PlanNode newChild = node.getChild().accept(this, context); + return new OffsetNode(context.getQueryId().genPlanNodeId(), newChild, node.getCount()); + } + + @Override + public PlanNode visitExplainAnalyze(ExplainAnalyzeNode node, ClonerContext context) { + PlanNode newChild = node.getChild().accept(this, context); + return new ExplainAnalyzeNode( + context.getQueryId().genPlanNodeId(), + newChild, + node.isVerbose(), + context.getLocalQueryId(), + node.getTimeout(), + node.getOutputSymbols().get(0), + node.getChildPermittedOutputs()); + } + + @Override + public PlanNode visitApply(ApplyNode node, ClonerContext context) { + PlanNode newLeft = node.getLeftChild().accept(this, context); + PlanNode newRight = node.getRightChild().accept(this, context); + return new ApplyNode( + context.getQueryId().genPlanNodeId(), + newLeft, + newRight, + node.getSubqueryAssignments(), + node.getCorrelation(), + node.getOriginSubquery()); + } + + @Override + public PlanNode visitCorrelatedJoin(CorrelatedJoinNode node, ClonerContext context) { + PlanNode newLeft = node.getLeftChild().accept(this, context); + PlanNode newRight = node.getRightChild().accept(this, context); + return new CorrelatedJoinNode( + context.getQueryId().genPlanNodeId(), + newLeft, + newRight, + node.getCorrelation(), + node.getJoinType(), + node.getFilter(), + node.getOriginSubquery()); + } + + @Override + public PlanNode visitLinearFill(LinearFillNode node, ClonerContext context) { + PlanNode newChild = node.getChild().accept(this, context); + LinearFillNode newNode = (LinearFillNode) node.clone(); + newNode.setChild(newChild); + return newNode; + } + + @Override + public PlanNode visitPreviousFill(PreviousFillNode node, ClonerContext context) { + PlanNode newChild = node.getChild().accept(this, context); + PreviousFillNode newNode = (PreviousFillNode) node.clone(); + newNode.setChild(newChild); + return newNode; + } + + @Override + public PlanNode visitValueFill(ValueFillNode node, ClonerContext context) { + PlanNode newChild = node.getChild().accept(this, context); + return new ValueFillNode( + context.getQueryId().genPlanNodeId(), newChild, node.getFilledValue()); + } + + @Override + public PlanNode visitGroup(GroupNode node, ClonerContext context) { + PlanNode newChild = node.getChild().accept(this, context); + return new GroupNode( + context.getQueryId().genPlanNodeId(), + newChild, + node.getOrderingScheme(), + node.getPartitionKeyCount()); + } + + @Override + public PlanNode visitTopK(TopKNode node, ClonerContext context) { + List newChildren = new ArrayList<>(); + for (PlanNode child : node.getChildren()) { + newChildren.add(child.accept(this, context)); + } + return new TopKNode( + context.getQueryId().genPlanNodeId(), + newChildren, + node.getOrderingScheme(), + node.getCount(), + node.getOutputSymbols(), + node.isChildrenDataInOrder()); + } + + @Override + public PlanNode visitSemiJoin(SemiJoinNode node, ClonerContext context) { + PlanNode newLeft = node.getLeftChild().accept(this, context); + PlanNode newRight = node.getRightChild().accept(this, context); + return new SemiJoinNode( + context.getQueryId().genPlanNodeId(), + newLeft, + newRight, + node.getSourceJoinSymbol(), + node.getFilteringSourceJoinSymbol(), + node.getSemiJoinOutput()); + } + + @Override + public PlanNode visitGroupReference(GroupReference node, ClonerContext context) { + return new GroupReference( + context.getQueryId().genPlanNodeId(), node.getGroupId(), node.getOutputSymbols()); + } + + @Override + public PlanNode visitTreeDeviceViewScan(TreeDeviceViewScanNode node, ClonerContext context) { + return new TreeDeviceViewScanNode( + context.getQueryId().genPlanNodeId(), + node.getQualifiedObjectName(), + node.getOutputSymbols(), + node.getAssignments(), + node.getTagAndAttributeIndexMap(), + node.getTreeDBName(), + node.getMeasurementColumnNameMap()); + } + } + + public static List collectDeviceTableScanNodes(PlanNode root) { + List list = new ArrayList<>(); + traverse(root, list); + return list; + } + + private static void traverse(PlanNode node, List list) { + if (node instanceof DeviceTableScanNode) { + list.add((DeviceTableScanNode) node); + } + for (PlanNode child : node.getChildren()) { + traverse(child, list); + } + } + + public long estimateMemoryUsage() { + long size = INSTANCE_SIZE; + + if (planNode != null) { + size += RamUsageEstimator.sizeOfObject(planNode); + } + + if (respHeader != null) { + size += RamUsageEstimator.sizeOfObject(respHeader); + } + + if (symbolMap != null) { + size += RamUsageEstimator.sizeOfMap(symbolMap); + } + + if (metadataExpressionLists != null) { + for (List list : metadataExpressionLists) { + if (list != null) { + for (Expression e : list) { + if (e != null) { + size += RamUsageEstimator.sizeOfObject(e); + } + } + } + } + } + + if (attributeColumnsLists != null) { + for (List list : attributeColumnsLists) { + if (list != null) { + for (String s : list) { + if (s != null) { + size += RamUsageEstimator.sizeOf(s); + } + } + } + } + } + + // scanNodes, assignmentsLists, and literalReferences all store references to a small part of + // the planNode, so there is no need to repeat the calculation + + return size; + } + + public static class ClonerContext { + private final QueryId queryId; + private final long localQueryId; + private final List newLiterals; + + public ClonerContext(QueryId queryId, long localQueryId, List newLiterals) { + this.queryId = queryId; + this.localQueryId = localQueryId; + this.newLiterals = newLiterals; + } + + public QueryId getQueryId() { + return queryId; + } + + public long getLocalQueryId() { + return localQueryId; + } + + public List getNewLiterals() { + return newLiterals; + } + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanCacheManager.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanCacheManager.java new file mode 100644 index 0000000000000..bc67290ea50a6 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanCacheManager.java @@ -0,0 +1,422 @@ +/* + * 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.queryengine.plan.relational.planner; + +import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.queryengine.common.header.DatasetHeader; +import org.apache.iotdb.db.queryengine.plan.planner.plan.node.PlanNode; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Literal; + +import org.apache.tsfile.read.common.type.Type; +import org.apache.tsfile.utils.RamUsageEstimator; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +/** LRU cache with lightweight state-machine based admission control. */ +public class PlanCacheManager { + private static final int INSTANCE_SIZE = + (int) RamUsageEstimator.shallowSizeOfInstance(PlanCacheManager.class); + + private static class SingletonHolder { + private static final PlanCacheManager INSTANCE = new PlanCacheManager(); + } + + public enum PlanCacheState { + MONITOR, + ACTIVE, + BYPASS + } + + public static class LookupDecision { + private final PlanCacheState state; + private final boolean shouldLookup; + private final String reason; + + private LookupDecision(PlanCacheState state, boolean shouldLookup, String reason) { + this.state = state; + this.shouldLookup = shouldLookup; + this.reason = reason; + } + + public PlanCacheState getState() { + return state; + } + + public boolean shouldLookup() { + return shouldLookup; + } + + public String getReason() { + return reason; + } + } + + public static class ProfileDiagnostics { + private final double ewmaReusablePlanningCost; + private final double ewmaFirstResponseLatency; + private final double ewmaBenefitRatio; + private final long sampleCount; + private final long hitCount; + private final long missCount; + private final long bypassCount; + + private ProfileDiagnostics( + double ewmaReusablePlanningCost, + double ewmaFirstResponseLatency, + double ewmaBenefitRatio, + long sampleCount, + long hitCount, + long missCount, + long bypassCount) { + this.ewmaReusablePlanningCost = ewmaReusablePlanningCost; + this.ewmaFirstResponseLatency = ewmaFirstResponseLatency; + this.ewmaBenefitRatio = ewmaBenefitRatio; + this.sampleCount = sampleCount; + this.hitCount = hitCount; + this.missCount = missCount; + this.bypassCount = bypassCount; + } + + public double getEwmaReusablePlanningCost() { + return ewmaReusablePlanningCost; + } + + public double getEwmaFirstResponseLatency() { + return ewmaFirstResponseLatency; + } + + public double getEwmaBenefitRatio() { + return ewmaBenefitRatio; + } + + public long getSampleCount() { + return sampleCount; + } + + public long getHitCount() { + return hitCount; + } + + public long getMissCount() { + return missCount; + } + + public long getBypassCount() { + return bypassCount; + } + } + + private static class TemplateProfile { + private static final double EWMA_ALPHA = 0.5; + + private PlanCacheState state = PlanCacheState.MONITOR; + private boolean warmedUp; + private long sampleCount; + private long hitCount; + private long missCount; + private long bypassCount; + private long lastAccessTime; + private long lastStateChangeTime; + private long cooldownDeadline; + private double ewmaReusablePlanningCost; + private double ewmaFirstResponseLatency; + private double ewmaBenefitRatio; + + synchronized LookupDecision beforeLookup(long now, long bypassCooldownNanos) { + if (state == PlanCacheState.BYPASS && now >= cooldownDeadline) { + state = PlanCacheState.MONITOR; + lastStateChangeTime = now; + } + lastAccessTime = now; + if (state == PlanCacheState.ACTIVE) { + return new LookupDecision(state, true, ""); + } + if (state == PlanCacheState.BYPASS) { + bypassCount++; + return new LookupDecision(state, false, "Low_Benefit"); + } + return new LookupDecision(state, false, "Collecting_Samples"); + } + + synchronized PlanCacheState recordExecution( + long reusablePlanningCost, + long firstResponseLatency, + boolean cacheLookupMiss, + int minSamples, + long minReusablePlanningCost, + double admitRatio, + double bypassRatio, + long bypassCooldownNanos, + long now) { + lastAccessTime = now; + if (cacheLookupMiss) { + missCount++; + } + + // Skip the first execution to avoid JVM warmup noise (class loading, JIT, + // cold caches) that produces unrepresentative planning cost. + if (!warmedUp) { + warmedUp = true; + return state; + } + + sampleCount++; + + ewmaReusablePlanningCost = ewma(ewmaReusablePlanningCost, reusablePlanningCost); + ewmaFirstResponseLatency = ewma(ewmaFirstResponseLatency, firstResponseLatency); + double benefitRatio = + firstResponseLatency <= 0 ? 0 : ((double) reusablePlanningCost) / firstResponseLatency; + ewmaBenefitRatio = ewma(ewmaBenefitRatio, benefitRatio); + + if (state == PlanCacheState.MONITOR && sampleCount >= minSamples) { + if (ewmaReusablePlanningCost >= minReusablePlanningCost && ewmaBenefitRatio >= admitRatio) { + state = PlanCacheState.ACTIVE; + lastStateChangeTime = now; + } else if (ewmaReusablePlanningCost < minReusablePlanningCost + || ewmaBenefitRatio < bypassRatio) { + state = PlanCacheState.BYPASS; + cooldownDeadline = now + bypassCooldownNanos; + lastStateChangeTime = now; + } + } else if (state == PlanCacheState.ACTIVE) { + // Degrade ACTIVE → MONITOR when benefit drops below admission thresholds. + // This prevents historical hot templates from polluting the cache indefinitely. + if (ewmaReusablePlanningCost < minReusablePlanningCost + || ewmaBenefitRatio < admitRatio) { + state = PlanCacheState.MONITOR; + sampleCount = 0; // Reset samples for re-evaluation + lastStateChangeTime = now; + } + } + return state; + } + + synchronized void recordCacheHit(long now) { + hitCount++; + lastAccessTime = now; + } + + synchronized PlanCacheState getState() { + return state; + } + + synchronized long getEstimatedReusablePlanningCost() { + return (long) ewmaReusablePlanningCost; + } + + synchronized ProfileDiagnostics getDiagnostics() { + return new ProfileDiagnostics( + ewmaReusablePlanningCost, + ewmaFirstResponseLatency, + ewmaBenefitRatio, + sampleCount, + hitCount, + missCount, + bypassCount); + } + + private double ewma(double current, double latest) { + if (current == 0) { + return latest; + } + return current * (1 - EWMA_ALPHA) + latest * EWMA_ALPHA; + } + } + + private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig(); + + private int getMaxCacheSize() { + return CONFIG.getSmartPlanCacheCapacity(); + } + + private long getMaxMemoryBytes() { + return CONFIG.getSmartPlanCacheMaxMemoryBytes(); + } + + private int getMinSamples() { + return CONFIG.getSmartPlanCacheMinSamples(); + } + + private long getMinReusablePlanningCostNanos() { + return CONFIG.getSmartPlanCacheMinReusablePlanningCostNanos(); + } + + private double getAdmitRatio() { + return CONFIG.getSmartPlanCacheAdmitRatio(); + } + + private double getBypassRatio() { + return CONFIG.getSmartPlanCacheBypassRatio(); + } + + private long getBypassCooldownNanos() { + return CONFIG.getSmartPlanCacheBypassCooldownMinutes() * 60L * 1_000_000_000L; + } + + private final AtomicLong currentMemoryBytes = new AtomicLong(INSTANCE_SIZE); + private final Map planCache; + private final Map templateProfiles; + + private PlanCacheManager() { + this.planCache = new LinkedHashMap<>(16, 0.75f, true); + this.templateProfiles = new ConcurrentHashMap<>(); + } + + public static PlanCacheManager getInstance() { + return SingletonHolder.INSTANCE; + } + + public LookupDecision getLookupDecision(String cacheKey) { + TemplateProfile profile = + templateProfiles.computeIfAbsent(cacheKey, ignored -> new TemplateProfile()); + return profile.beforeLookup(System.nanoTime(), getBypassCooldownNanos()); + } + + public void recordCacheHit(String cacheKey) { + TemplateProfile profile = templateProfiles.get(cacheKey); + if (profile != null) { + profile.recordCacheHit(System.nanoTime()); + } + } + + public PlanCacheState recordExecution( + String cacheKey, + long reusablePlanningCost, + long firstResponseLatency, + boolean cacheLookupMiss) { + TemplateProfile profile = + templateProfiles.computeIfAbsent(cacheKey, ignored -> new TemplateProfile()); + return profile.recordExecution( + reusablePlanningCost, + firstResponseLatency, + cacheLookupMiss, + getMinSamples(), + getMinReusablePlanningCostNanos(), + getAdmitRatio(), + getBypassRatio(), + getBypassCooldownNanos(), + System.nanoTime()); + } + + public long getEstimatedReusablePlanningCost(String cacheKey) { + TemplateProfile profile = templateProfiles.get(cacheKey); + return profile == null ? 0 : profile.getEstimatedReusablePlanningCost(); + } + + public ProfileDiagnostics getProfileDiagnostics(String cacheKey) { + TemplateProfile profile = templateProfiles.get(cacheKey); + return profile == null ? null : profile.getDiagnostics(); + } + + public boolean shouldCache(String cacheKey) { + TemplateProfile profile = templateProfiles.get(cacheKey); + return profile != null && profile.getState() == PlanCacheState.ACTIVE; + } + + public void cacheValue( + String cachedKey, + PlanNode planNode, + List scanNodes, + List literalReference, + DatasetHeader header, + HashMap symbolMap, + int symbolNextId, + List> metadataExpressionLists, + List> attributeColumnsLists, + List> assignmentsLists) { + CachedValue newValue = + new CachedValue( + planNode, + scanNodes, + literalReference, + header, + symbolMap, + symbolNextId, + metadataExpressionLists, + attributeColumnsLists, + assignmentsLists); + + long keySize = RamUsageEstimator.sizeOf(cachedKey); + long newValueSize = newValue.estimateMemoryUsage(); + + synchronized (planCache) { + CachedValue oldValue = planCache.put(cachedKey, newValue); + if (oldValue != null) { + currentMemoryBytes.addAndGet(-oldValue.estimateMemoryUsage()); + } else { + currentMemoryBytes.addAndGet(keySize); + } + currentMemoryBytes.addAndGet(newValueSize); + + Iterator> iterator = planCache.entrySet().iterator(); + while ((currentMemoryBytes.get() > getMaxMemoryBytes() + || planCache.size() > getMaxCacheSize()) + && iterator.hasNext()) { + Map.Entry eldest = iterator.next(); + CachedValue evicted = eldest.getValue(); + long evictedKeySize = RamUsageEstimator.sizeOf(eldest.getKey()); + long evictedValueSize = evicted.estimateMemoryUsage(); + iterator.remove(); + currentMemoryBytes.addAndGet(-(evictedKeySize + evictedValueSize)); + } + } + } + + public int size() { + synchronized (planCache) { + return planCache.size(); + } + } + + public CachedValue getCachedValue(String cacheKey) { + synchronized (planCache) { + return planCache.get(cacheKey); + } + } + + public long getCurrentMemoryBytes() { + return currentMemoryBytes.get(); + } + + @TestOnly + public PlanCacheState getTemplateState(String cacheKey) { + TemplateProfile profile = templateProfiles.get(cacheKey); + return profile == null ? null : profile.getState(); + } + + public void clear() { + synchronized (planCache) { + planCache.clear(); + currentMemoryBytes.set(INSTANCE_SIZE); + } + templateProfiles.clear(); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SymbolAllocator.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SymbolAllocator.java index 5840017e5c4fd..d605357942fb2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SymbolAllocator.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/SymbolAllocator.java @@ -109,6 +109,22 @@ public TypeProvider getTypes() { return TypeProvider.viewOf(symbolMap); } + public HashMap cloneSymbolMap() { + return new HashMap<>(symbolMap); + } + + public void fill(Map symbolMap) { + this.symbolMap.putAll(symbolMap); + } + + public int getNextId() { + return nextId; + } + + public void setNextId(int nextId) { + this.nextId = nextId; + } + private int nextId() { return nextId++; } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java index c33cb05250bd1..5967a061ce659 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/TableLogicalPlanner.java @@ -19,6 +19,9 @@ package org.apache.iotdb.db.queryengine.plan.relational.planner; +import org.apache.iotdb.common.rpc.thrift.TTimePartitionSlot; +import org.apache.iotdb.commons.partition.DataPartition; +import org.apache.iotdb.commons.partition.DataPartitionQueryParam; import org.apache.iotdb.commons.partition.SchemaPartition; import org.apache.iotdb.commons.path.PathPatternTree; import org.apache.iotdb.commons.schema.column.ColumnHeader; @@ -26,6 +29,8 @@ import org.apache.iotdb.commons.schema.table.TreeViewSchema; import org.apache.iotdb.commons.schema.table.TsTable; import org.apache.iotdb.commons.utils.TestOnly; +import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.sql.SemanticException; import org.apache.iotdb.db.queryengine.common.MPPQueryContext; import org.apache.iotdb.db.queryengine.common.QueryId; @@ -44,14 +49,17 @@ import org.apache.iotdb.db.queryengine.plan.relational.analyzer.RelationId; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.RelationType; import org.apache.iotdb.db.queryengine.plan.relational.analyzer.Scope; +import org.apache.iotdb.db.queryengine.plan.relational.analyzer.predicate.ConvertPredicateToTimeFilterVisitor; import org.apache.iotdb.db.queryengine.plan.relational.execution.querystats.PlanOptimizersStatsCollector; import org.apache.iotdb.db.queryengine.plan.relational.metadata.ColumnSchema; +import org.apache.iotdb.db.queryengine.plan.relational.metadata.DeviceEntry; import org.apache.iotdb.db.queryengine.plan.relational.metadata.Metadata; import org.apache.iotdb.db.queryengine.plan.relational.metadata.QualifiedObjectName; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableMetadataImpl; import org.apache.iotdb.db.queryengine.plan.relational.metadata.TableSchema; import org.apache.iotdb.db.queryengine.plan.relational.planner.ir.PredicateWithUncorrelatedScalarSubqueryReconstructor; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.CopyToNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.DeviceTableScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ExplainAnalyzeNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.FilterNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.IntoNode; @@ -59,6 +67,7 @@ import org.apache.iotdb.db.queryengine.plan.relational.planner.node.OffsetNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.OutputNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.ProjectNode; +import org.apache.iotdb.db.queryengine.plan.relational.planner.node.TreeDeviceViewScanNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.CreateOrUpdateTableDeviceNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.TableDeviceAttributeUpdateNode; import org.apache.iotdb.db.queryengine.plan.relational.planner.node.schema.TableDeviceFetchNode; @@ -74,36 +83,56 @@ import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Delete; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Explain; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ExplainAnalyze; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Expression; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.FetchDevice; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Insert; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Literal; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LiteralMarkerReplacer; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LoadTsFile; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.PipeEnriched; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Query; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.ShowDevice; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Table; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.TableNameRewriter; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Update; import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.WrappedStatement; +import org.apache.iotdb.db.queryengine.plan.relational.sql.util.SqlFormatter; import org.apache.iotdb.db.queryengine.plan.relational.type.InternalTypeManager; import org.apache.iotdb.db.schemaengine.table.DataNodeTableCache; +import org.apache.iotdb.db.utils.TimestampPrecisionUtils; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.tsfile.enums.TSDataType; +import org.apache.tsfile.external.commons.codec.digest.DigestUtils; import org.apache.tsfile.read.common.type.LongType; import org.apache.tsfile.read.common.type.StringType; import org.apache.tsfile.read.common.type.TypeFactory; +import org.apache.tsfile.read.filter.basic.Filter; +import org.apache.tsfile.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.stream.Collectors; import static com.google.common.collect.ImmutableList.toImmutableList; import static java.util.Objects.requireNonNull; import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.LOGICAL_PLANNER; import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.LOGICAL_PLAN_OPTIMIZE; +import static org.apache.iotdb.db.queryengine.metric.QueryPlanCostMetricSet.SCHEMA_FETCHER; +import static org.apache.iotdb.db.queryengine.plan.analyze.AnalyzeVisitor.getTimePartitionSlotList; import static org.apache.iotdb.db.queryengine.plan.relational.metadata.MetadataUtil.createQualifiedObjectName; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.CachedValue.cloneMetadataExpressions; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.CachedValue.clonePlanWithNewLiterals; +import static org.apache.iotdb.db.queryengine.plan.relational.planner.CachedValue.collectDeviceTableScanNodes; import static org.apache.iotdb.db.queryengine.plan.relational.planner.QueryPlanner.visibleFields; import static org.apache.iotdb.db.queryengine.plan.relational.planner.node.OutputNode.COLUMN_NAME_PREFIX; import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.CountDevice.COUNT_DEVICE_HEADER_STRING; @@ -117,7 +146,6 @@ public class TableLogicalPlanner { private final List planOptimizers; private final Metadata metadata; private final WarningCollector warningCollector; - private PredicateWithUncorrelatedScalarSubqueryReconstructor predicateWithUncorrelatedScalarSubqueryReconstructor; @@ -163,9 +191,237 @@ public void setPredicateWithUncorrelatedScalarSubqueryReconstructor( predicateWithUncorrelatedScalarSubqueryReconstructor; } + private List generalizeStatement(Query query) { + LiteralMarkerReplacer literalMarkerReplacer = new LiteralMarkerReplacer(); + literalMarkerReplacer.process(query); + return literalMarkerReplacer.getLiteralList(); + } + + private String calculateCacheKey(Statement statement, Analysis analysis) { + StringBuilder sb = new StringBuilder(); + Statement normalized = TableNameRewriter.rewrite(statement, analysis.getDatabaseName()); + sb.append(SqlFormatter.formatSql(normalized)); + long version = DataNodeTableCache.getInstance().getInstanceVersion(); + sb.append(version); + String rawKey = sb.toString(); + return DigestUtils.md5Hex(rawKey.getBytes(StandardCharsets.UTF_8)); + } + + private void populatePlanCacheDiagnostics(String cachedKey, MPPQueryContext queryContext) { + PlanCacheManager.ProfileDiagnostics diag = + PlanCacheManager.getInstance().getProfileDiagnostics(cachedKey); + if (diag != null) { + queryContext.setPlanCacheDiagnostics( + diag.getEwmaReusablePlanningCost(), + diag.getEwmaFirstResponseLatency(), + diag.getEwmaBenefitRatio(), + diag.getSampleCount(), + diag.getHitCount(), + diag.getMissCount(), + diag.getBypassCount(), + CONFIG.getSmartPlanCacheMinReusablePlanningCostNanos(), + CONFIG.getSmartPlanCacheAdmitRatio(), + CONFIG.getSmartPlanCacheBypassRatio()); + } + } + + private static final Logger logger = LoggerFactory.getLogger(TableLogicalPlanner.class); + private static final IoTDBConfig CONFIG = IoTDBDescriptor.getInstance().getConfig(); + + private DataPartition fetchDataPartitionByDevices( + final String + database, // for tree view, database should be the real tree db name with `root.` prefix + final List deviceEntries, + final Filter globalTimeFilter) { + final Pair, Pair> res = + getTimePartitionSlotList(globalTimeFilter, queryContext); + + // there is no satisfied time range + if (res.left.isEmpty() && Boolean.FALSE.equals(res.right.left)) { + return new DataPartition( + Collections.emptyMap(), + CONFIG.getSeriesPartitionExecutorClass(), + CONFIG.getSeriesPartitionSlotNum()); + } + + final List dataPartitionQueryParams = + deviceEntries.stream() + .map( + deviceEntry -> + new DataPartitionQueryParam( + deviceEntry.getDeviceID(), res.left, res.right.left, res.right.right)) + .collect(Collectors.toList()); + + if (res.right.left || res.right.right) { + return metadata.getDataPartitionWithUnclosedTimeRange(database, dataPartitionQueryParams); + } else { + return metadata.getDataPartition(database, dataPartitionQueryParams); + } + } + + /** The metadataExpression in cachedValue needs to be used to update the scanNode in planNode */ + private void adjustSchema( + DeviceTableScanNode scanNode, + List metaExprs, + List attributeCols, + Map assignments, + Analysis analysis) { + + long startTime = System.nanoTime(); + + Map> deviceEntriesMap = + metadata.indexScan( + scanNode.getQualifiedObjectName(), metaExprs, attributeCols, queryContext); + + String deviceDatabase = + !deviceEntriesMap.isEmpty() ? deviceEntriesMap.keySet().iterator().next() : null; + List deviceEntries = + deviceDatabase != null ? deviceEntriesMap.get(deviceDatabase) : Collections.emptyList(); + scanNode.setDeviceEntries(deviceEntries); + + Filter timeFilter = + scanNode + .getTimePredicate() + .map( + v -> + v.accept( + new ConvertPredicateToTimeFilterVisitor( + queryContext.getZoneId(), TimestampPrecisionUtils.currPrecision), + null)) + .orElse(null); + scanNode.setTimeFilter(timeFilter); + + DataPartition dataPartition = + fetchDataPartitionByDevices( + scanNode instanceof TreeDeviceViewScanNode + ? ((TreeDeviceViewScanNode) scanNode).getTreeDBName() + : scanNode.getQualifiedObjectName().getDatabaseName(), + deviceEntries, + timeFilter); + analysis.upsertDataPartition(dataPartition); + + long schemaFetchCost = System.nanoTime() - startTime; + QueryPlanCostMetricSet.getInstance().recordTablePlanCost(SCHEMA_FETCHER, schemaFetchCost); + queryContext.setFetchSchemaCost(schemaFetchCost); + + if (deviceEntries.isEmpty() && analysis.noAggregates() && !analysis.hasJoinNode()) { + analysis.setEmptyDataSource(true); + analysis.setFinishQueryAfterAnalyze(); + } + } + + private Query getCacheableQuery(Statement statement) { + if (statement instanceof Query) { + return (Query) statement; + } + if (statement instanceof Explain) { + return getCacheableQuery(((Explain) statement).getStatement()); + } + if (statement instanceof ExplainAnalyze) { + return getCacheableQuery(((ExplainAnalyze) statement).getStatement()); + } + return null; + } + public LogicalQueryPlan plan(final Analysis analysis) { long startTime = System.nanoTime(); - final Statement statement = analysis.getStatement(); + long totalStartTime = startTime; + Statement statement = analysis.getStatement(); + + String cachedKey = ""; + List literalReference = null; + PlanCacheManager.LookupDecision lookupDecision = null; + boolean cacheLookupAttempted = false; + Query cacheableQuery = getCacheableQuery(statement); + + if (cacheableQuery != null) { + List literalList = generalizeStatement(cacheableQuery); + cachedKey = calculateCacheKey(cacheableQuery, analysis); + lookupDecision = PlanCacheManager.getInstance().getLookupDecision(cachedKey); + queryContext.setPlanCacheState(lookupDecision.getState().name()); + populatePlanCacheDiagnostics(cachedKey, queryContext); + if (lookupDecision.shouldLookup()) { + long lookupStartTime = System.nanoTime(); + cacheLookupAttempted = true; + CachedValue cachedValue = PlanCacheManager.getInstance().getCachedValue(cachedKey); + queryContext.setPlanCacheLookupCost(System.nanoTime() - lookupStartTime); + if (cachedValue != null && !queryContext.isExplainAnalyze()) { + queryContext.setPlanCacheStatus("HIT"); + queryContext.setSavedLogicalPlanningCost( + PlanCacheManager.getInstance().getEstimatedReusablePlanningCost(cachedKey)); + PlanCacheManager.getInstance().recordCacheHit(cachedKey); + // deal with the device stuff + long curTime = System.nanoTime(); + symbolAllocator.fill(cachedValue.getSymbolMap()); + symbolAllocator.setNextId(cachedValue.getSymbolNextId()); + analysis.setRespDatasetHeader(cachedValue.getRespHeader()); + + // Clone the PlanNode with new literals + CachedValue.ClonerContext clonerContext = + new CachedValue.ClonerContext( + queryContext.getQueryId(), queryContext.getLocalQueryId(), literalList); + PlanNode newPlan = clonePlanWithNewLiterals(cachedValue.getPlanNode(), clonerContext); + // Clone the metadata expressions with new literals + List> newMetadataExpressionLists = new ArrayList<>(); + if (cachedValue.getMetadataExpressionLists() != null) { + for (List exprList : cachedValue.getMetadataExpressionLists()) { + if (exprList != null) { + newMetadataExpressionLists.add(cloneMetadataExpressions(exprList, literalList)); + } else { + // occupy an empty list and maintain a one-to-one correspondence with scanNodes + newMetadataExpressionLists.add(new ArrayList<>()); + } + } + } + + List scanNodes = collectDeviceTableScanNodes(newPlan); + + for (int i = 0; i < scanNodes.size(); i++) { + DeviceTableScanNode scanNode = scanNodes.get(i); + + List metaExprs = + i < newMetadataExpressionLists.size() + ? newMetadataExpressionLists.get(i) + : Collections.emptyList(); + List attributeCols = + i < cachedValue.getAttributeColumnsLists().size() + ? cachedValue.getAttributeColumnsLists().get(i) + : Collections.emptyList(); + Map assignments = + i < cachedValue.getAssignmentsLists().size() + ? cachedValue.getAssignmentsLists().get(i) + : Collections.emptyMap(); + + adjustSchema(scanNode, metaExprs, attributeCols, assignments, analysis); + } + + logger.info( + "Logical plan is cached, adjustment cost time: {}", System.nanoTime() - curTime); + logger.info("Logical plan is cached, cost time: {}", System.nanoTime() - totalStartTime); + logger.info( + "Logical plan is cached, fetch schema cost time: {}", + queryContext.getFetchPartitionCost() + queryContext.getFetchSchemaCost()); + + return new LogicalQueryPlan(queryContext, newPlan); + } + if (cachedValue != null) { + // EXPLAIN ANALYZE: cache exists but we need full plan with ExplainAnalyzeNode + queryContext.setPlanCacheStatus("HIT"); + queryContext.setSavedLogicalPlanningCost( + PlanCacheManager.getInstance().getEstimatedReusablePlanningCost(cachedKey)); + } else { + queryContext.setPlanCacheStatus("MISS"); + } + } else { + queryContext.setPlanCacheStatus("BYPASS", lookupDecision.getReason()); + } + // Following implementation of plan should be based on the generalizedStatement + literalReference = literalList; + } else { + queryContext.setPlanCacheStatus("BYPASS", "Unsupported_Statement"); + } + + // The logical plan was not hit. The logical plan generation stage needs to be executed PlanNode planNode = planStatement(analysis, statement); if (analysis.isQuery()) { @@ -197,7 +453,53 @@ public LogicalQueryPlan plan(final Analysis analysis) { queryContext.setLogicalOptimizationCost(logicalOptimizationCost); QueryPlanCostMetricSet.getInstance() .recordTablePlanCost(LOGICAL_PLAN_OPTIMIZE, logicalOptimizationCost); + + long reusablePlanningCost = logicalPlanCostTime + logicalOptimizationCost; + queryContext.setReusablePlanningCost(reusablePlanningCost); + + if (cacheableQuery != null) { + // Store cache context for deferred recordExecution in QueryExecution.recordExecutionTime() + // where the real firstResponseLatency (first RPC server-side time) is available. + queryContext.setPlanCacheCachedKey(cachedKey); + queryContext.setPlanCacheLookupAttempted(cacheLookupAttempted); + + PlanCacheManager.PlanCacheState currentState = + lookupDecision != null + ? lookupDecision.getState() + : PlanCacheManager.PlanCacheState.MONITOR; + queryContext.setPlanCacheState(currentState.name()); + queryContext.setSavedLogicalPlanningCost( + currentState == PlanCacheManager.PlanCacheState.ACTIVE ? reusablePlanningCost : 0); + if (PlanCacheManager.getInstance().shouldCache(cachedKey)) { + CachedValue.ClonerContext clonerContext = + new CachedValue.ClonerContext( + queryContext.getQueryId(), queryContext.getLocalQueryId(), literalReference); + PlanNode clonedPlan = clonePlanWithNewLiterals(planNode, clonerContext); + List scanNodes = collectDeviceTableScanNodes(clonedPlan); + + PlanCacheManager.getInstance() + .cacheValue( + cachedKey, + clonedPlan, + scanNodes, + literalReference, + analysis.getRespDatasetHeader(), + symbolAllocator.cloneSymbolMap(), + symbolAllocator.getNextId(), + queryContext.getMetadataExpressionLists(), + queryContext.getAttributeColumnsLists(), + queryContext.getAssignmentsLists()); + if (!cacheLookupAttempted && lookupDecision != null) { + queryContext.setPlanCacheStatus( + currentState == PlanCacheManager.PlanCacheState.ACTIVE ? "MISS" : "MONITOR"); + } + } + } } + logger.info( + "Logical plan is generated, fetch schema cost time: {}", + queryContext.getFetchPartitionCost() + queryContext.getFetchSchemaCost()); + logger.info("Logical plan is generated, cost time: {}", System.nanoTime() - totalStartTime); return new LogicalQueryPlan(queryContext, planNode); } @@ -398,7 +700,6 @@ private RelationPlan createRelationPlan(Analysis analysis, PipeEnriched pipeEnri } private RelationPlan createRelationPlan(Analysis analysis, Query query) { - // materialize cte if needed if (!queryContext.isInnerTriggeredQuery()) { CteMaterializer.getInstance().materializeCTE(analysis, queryContext); } @@ -624,7 +925,6 @@ private RelationPlan planExplainAnalyze(final ExplainAnalyze statement, final An queryContext.getLocalQueryId(), queryContext.getTimeOut(), symbol, - // recording permittedOutputs of ExplainAnalyzeNode's child getChildPermittedOutputs(analysis, statement.getStatement(), originalQueryPlan)); return new RelationPlan( newRoot, @@ -651,7 +951,6 @@ private RelationPlan createRelationPlan(final CopyTo statement, final Analysis a innerQueryRelationPlan.getRoot(), statement.getTargetFileName(), statement.getOptions(), - // recording permittedOutputs of CopyToNode's child getChildPermittedOutputs( analysis, statement.getQueryStatement(), innerQueryRelationPlan), innerQueryRespDatasetHeader, diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java index 7bd4530afd194..eab98bdc65a4c 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/planner/optimizations/PushPredicateIntoTableScan.java @@ -688,16 +688,21 @@ private void getDeviceEntriesWithDataPartitions( } } + List metaDataExpression = + metadataExpressions.stream() + .map( + expression -> + ReplaceSymbolInExpression.transform( + expression, tableScanNode.getAssignments())) + .collect(Collectors.toList()); + queryContext.addMetadataExpressionList(metaDataExpression); + queryContext.addAttributeColumnsList(attributeColumns); + queryContext.addAssignmentsList(tableScanNode.getAssignments()); long startTime = System.nanoTime(); final Map> deviceEntriesMap = metadata.indexScan( tableScanNode.getQualifiedObjectName(), - metadataExpressions.stream() - .map( - expression -> - ReplaceSymbolInExpression.transform( - expression, tableScanNode.getAssignments())) - .collect(Collectors.toList()), + metaDataExpression, attributeColumns, queryContext); if (deviceEntriesMap.size() > 1) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/BinaryLiteral.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/BinaryLiteral.java index d5baa741c7bef..277577950dfee 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/BinaryLiteral.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/BinaryLiteral.java @@ -44,7 +44,7 @@ public class BinaryLiteral extends Literal { private static final CharMatcher HEX_DIGIT_MATCHER = CharMatcher.inRange('A', 'F').or(CharMatcher.inRange('0', '9')).precomputed(); - private final byte[] value; + private byte[] value; public BinaryLiteral(String value) { super(null); @@ -153,4 +153,9 @@ public long ramBytesUsed() { + AstMemoryEstimationHelper.getEstimatedSizeOfNodeLocation(getLocationInternal()) + RamUsageEstimator.sizeOf(value); } + + @Override + public void replace(Literal literal) { + this.value = ((BinaryLiteral) literal).getValue(); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Literal.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Literal.java index b60e5a454a78d..914f2c5ae2717 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Literal.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/Literal.java @@ -44,4 +44,23 @@ public List getChildren() { * @return a value that can be processed by IoTDB storage engine */ public abstract Object getTsValue(); + + // for literalMarker + private int literalIndex = -1; + + public void setLiteralIndex(int literalIndex) { + this.literalIndex = literalIndex; + } + + public int getLiteralIndex() { + return literalIndex; + } + + public boolean isLiteralMarker() { + return literalIndex != -1; + } + + public void replace(Literal literal) { + throw new UnsupportedOperationException("Literal Replacement is not supported in current type"); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/LiteralMarkerReplacer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/LiteralMarkerReplacer.java new file mode 100644 index 0000000000000..c4ad36709352b --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/LiteralMarkerReplacer.java @@ -0,0 +1,190 @@ +/* + * 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.queryengine.plan.relational.sql.ast; + +import java.util.LinkedList; +import java.util.List; + +public class LiteralMarkerReplacer extends AstVisitor { + + private static final String NOT_SUPPORTED = + "visit() not supported for %s in LiteralMarkerReplacer."; + + List literalList = new LinkedList(); + int literalGlobalIndex = -1; + + public List getLiteralList() { + return literalList; + } + + @Override + protected Void visitNode(Node node, Void context) { + throw new UnsupportedOperationException( + "visitNode of Node {" + node.getClass() + "} is not supported in LiteralMarkerReplacer"); + } + + @Override + protected Void visitQuery(Query node, Void context) { + node.getQueryBody().accept(this, context); + return context; + } + + @Override + protected Void visitQuerySpecification(QuerySpecification node, Void context) { + node.getSelect().accept(this, context); + node.getWhere().ifPresent(where -> where.accept(this, context)); + node.getGroupBy().ifPresent(groupBy -> groupBy.accept(this, context)); + node.getHaving().ifPresent(having -> having.accept(this, context)); + node.getOrderBy().ifPresent(orderBy -> orderBy.accept(this, context)); + node.getLimit().ifPresent(limit -> limit.accept(this, context)); + node.getOffset().ifPresent(offset -> offset.accept(this, context)); + return context; + } + + @Override + protected Void visitSelect(Select node, Void context) { + node.getSelectItems().forEach(item -> item.accept(this, context)); + return context; + } + + @Override + protected Void visitAllColumns(AllColumns node, Void context) { + node.getTarget().ifPresent(target -> target.accept(this, context)); + return context; + } + + @Override + protected Void visitSingleColumn(SingleColumn node, Void context) { + node.getExpression().accept(this, context); + return context; + } + + @Override + protected Void visitOrderBy(OrderBy node, Void context) { + node.getSortItems().forEach(item -> item.getSortKey().accept(this, context)); + return context; + } + + @Override + protected Void visitGroupBy(GroupBy node, Void context) { + for (GroupingElement element : node.getGroupingElements()) { + element.getExpressions().forEach(expression -> expression.accept(this, context)); + } + return context; + } + + @Override + protected Void visitLimit(Limit node, Void context) { + node.getRowCount().accept(this, context); + return context; + } + + @Override + protected Void visitOffset(Offset node, Void context) { + node.getRowCount().accept(this, context); + return context; + } + + // ============================ Expression ============================ + + @Override + protected Void visitExpression(Expression node, Void context) { + // do nothing + return context; + } + + @Override + protected Void visitLogicalExpression(LogicalExpression node, Void context) { + node.getTerms().forEach(term -> term.accept(this, context)); + return context; + } + + @Override + protected Void visitComparisonExpression(ComparisonExpression node, Void context) { + node.getLeft().accept(this, context); + node.getRight().accept(this, context); + return context; + } + + @Override + protected Void visitIsNullPredicate(IsNullPredicate node, Void context) { + node.getValue().accept(this, context); + return context; + } + + @Override + protected Void visitIsNotNullPredicate(IsNotNullPredicate node, Void context) { + node.getValue().accept(this, context); + return context; + } + + @Override + protected Void visitFunctionCall(FunctionCall node, Void context) { + node.getArguments().forEach(arg -> arg.accept(this, context)); + return context; + } + + @Override + protected Void visitLikePredicate(LikePredicate node, Void context) { + node.getValue().accept(this, context); + return context; + } + + @Override + protected Void visitBetweenPredicate(BetweenPredicate node, Void context) { + node.getValue().accept(this, context); + node.getMin().accept(this, context); + node.getMax().accept(this, context); + return context; + } + + // TODO it can be optimized by an array of expressions + @Override + protected Void visitInPredicate(InPredicate node, Void context) { + node.getValue().accept(this, context); + return context; + } + + @Override + protected Void visitNotExpression(NotExpression node, Void context) { + node.getValue().accept(this, context); + return context; + } + + @Override + protected Void visitArithmeticBinary(ArithmeticBinaryExpression node, Void context) { + node.getLeft().accept(this, context); + node.getRight().accept(this, context); + return context; + } + + @Override + protected Void visitArithmeticUnary(ArithmeticUnaryExpression node, Void context) { + node.getValue().accept(this, context); + return context; + } + + @Override + protected Void visitLiteral(Literal node, Void context) { + literalList.add(node); + node.setLiteralIndex(++literalGlobalIndex); + return context; + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/LongLiteral.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/LongLiteral.java index 930714f1181e3..43f47516a7ab2 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/LongLiteral.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/LongLiteral.java @@ -35,8 +35,8 @@ public class LongLiteral extends Literal { private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(LongLiteral.class); - private final String value; - private final long parsedValue; + private String value; + private long parsedValue; public LongLiteral(String value) { super(null); @@ -145,4 +145,11 @@ public long ramBytesUsed() { + AstMemoryEstimationHelper.getEstimatedSizeOfNodeLocation(getLocationInternal()) + RamUsageEstimator.sizeOf(value); } + + @Override + public void replace(Literal literal) { + LongLiteral longLiteral = (LongLiteral) literal; + this.value = longLiteral.getValue(); + this.parsedValue = longLiteral.getParsedValue(); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/StringLiteral.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/StringLiteral.java index 981ccd6c1e834..c5dacb1e6fdfb 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/StringLiteral.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/StringLiteral.java @@ -36,8 +36,8 @@ public class StringLiteral extends Literal { private static final long INSTANCE_SIZE = RamUsageEstimator.shallowSizeOfInstance(StringLiteral.class); - private final String value; - private final int length; + private String value; + private int length; public StringLiteral(String value) { super(null); @@ -120,4 +120,11 @@ public long ramBytesUsed() { + AstMemoryEstimationHelper.getEstimatedSizeOfNodeLocation(getLocationInternal()) + RamUsageEstimator.sizeOf(value); } + + @Override + public void replace(Literal literal) { + StringLiteral stringLiteral = (StringLiteral) literal; + this.value = stringLiteral.getValue(); + this.length = stringLiteral.length(); + } } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/TableNameRewriter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/TableNameRewriter.java new file mode 100644 index 0000000000000..0595af618a920 --- /dev/null +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/ast/TableNameRewriter.java @@ -0,0 +1,113 @@ +/* + * 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.queryengine.plan.relational.sql.ast; + +import java.util.Optional; + +public class TableNameRewriter extends AstVisitor { + + /** + * Normalize all table names in the Statement to dbName.tableName + * + * @param statement original AST + * @param defaultDb The dbName currently used in the session (use dbName;) + * @return normalized Statement + */ + public static Statement rewrite(Statement statement, String defaultDb) { + return (Statement) statement.accept(new TableNameRewriter(), defaultDb); + } + + @Override + protected Node visitNode(Node node, String defaultDb) { + return node; + } + + @Override + protected Node visitQuery(Query node, String defaultDb) { + // rewrite queryBody + QueryBody newBody = (QueryBody) node.getQueryBody().accept(this, defaultDb); + + return new Query( + node.getLocation().orElse(null), + node.getWith(), + newBody, + node.getFill(), + node.getOrderBy(), + node.getOffset(), + node.getLimit()); + } + + @Override + protected Node visitQuerySpecification(QuerySpecification node, String defaultDb) { + // rewrite FROM + Optional newFrom = + node.getFrom().map(relation -> (Relation) relation.accept(this, defaultDb)); + + // rewrite WHERE + Optional newWhere = + node.getWhere().map(expr -> (Expression) expr.accept(this, defaultDb)); + + return new QuerySpecification( + node.getLocation().orElse(null), + node.getSelect(), + newFrom, + newWhere, + node.getGroupBy(), + node.getHaving(), + node.getFill(), + node.getWindows(), + node.getOrderBy(), + node.getOffset(), + node.getLimit()); + } + + @Override + protected Node visitTable(Table node, String defaultDb) { + QualifiedName name = node.getName(); + if (name.getParts().size() == 1 && defaultDb != null) { + // tableName -> dbName.tableName + return new Table(QualifiedName.of(defaultDb, name.getSuffix())); + } + return node; + } + + @Override + protected Node visitJoin(Join node, String defaultDb) { + Relation newLeft = (Relation) node.getLeft().accept(this, defaultDb); + Relation newRight = (Relation) node.getRight().accept(this, defaultDb); + + if (node.getCriteria().isPresent()) { + return new Join( + node.getLocation().orElse(null), + node.getType(), + newLeft, + newRight, + node.getCriteria().get()); + } else { + return new Join(node.getLocation().orElse(null), node.getType(), newLeft, newRight); + } + } + + @Override + protected Node visitAliasedRelation(AliasedRelation node, String defaultDb) { + Relation rewritten = (Relation) node.getRelation().accept(this, defaultDb); + return new AliasedRelation(rewritten, node.getAlias(), node.getColumnNames()); + } +} diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/util/ExpressionFormatter.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/util/ExpressionFormatter.java index ac2f640087354..31db432168ed8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/util/ExpressionFormatter.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/plan/relational/sql/util/ExpressionFormatter.java @@ -99,7 +99,6 @@ import static java.util.stream.Collectors.joining; import static java.util.stream.Collectors.toList; import static org.apache.iotdb.commons.udf.builtin.relational.TableBuiltinScalarFunction.DATE_BIN; -import static org.apache.iotdb.db.queryengine.plan.relational.sql.ast.FrameBound.Type.UNBOUNDED_PRECEDING; import static org.apache.iotdb.db.queryengine.plan.relational.sql.util.ReservedIdentifiers.reserved; import static org.apache.iotdb.db.queryengine.plan.relational.sql.util.SqlFormatter.formatName; import static org.apache.iotdb.db.queryengine.plan.relational.sql.util.SqlFormatter.formatSql; @@ -111,6 +110,8 @@ public final class ExpressionFormatter { () -> new DecimalFormat("0.###################E0###", new DecimalFormatSymbols(Locale.US))); + private static final String LITERAL_MARKER_FORMAT = "LiteralMarker(#%s)"; + private ExpressionFormatter() {} public static String formatExpression(Expression expression) { @@ -193,6 +194,10 @@ protected String visitExtract(Extract node, Void context) { @Override protected String visitBooleanLiteral(BooleanLiteral node, Void context) { + if (node.isLiteralMarker()) { + return String.format(LITERAL_MARKER_FORMAT, node.getLiteralIndex()); + } + return literalFormatter .map(formatter -> formatter.apply(node)) .orElseGet(() -> String.valueOf(node.getValue())); @@ -200,6 +205,10 @@ protected String visitBooleanLiteral(BooleanLiteral node, Void context) { @Override protected String visitStringLiteral(StringLiteral node, Void context) { + if (node.isLiteralMarker()) { + return String.format(LITERAL_MARKER_FORMAT, node.getLiteralIndex()); + } + return literalFormatter .map(formatter -> formatter.apply(node)) .orElseGet(() -> formatStringLiteral(node.getValue())); @@ -207,6 +216,10 @@ protected String visitStringLiteral(StringLiteral node, Void context) { @Override protected String visitBinaryLiteral(BinaryLiteral node, Void context) { + if (node.isLiteralMarker()) { + return String.format(LITERAL_MARKER_FORMAT, node.getLiteralIndex()); + } + return literalFormatter .map(formatter -> formatter.apply(node)) .orElseGet(() -> "X'" + node.toHexString() + "'"); @@ -224,11 +237,17 @@ protected String visitAllRows(AllRows node, Void context) { @Override protected String visitLongLiteral(LongLiteral node, Void context) { + if (node.isLiteralMarker()) { + return String.format(LITERAL_MARKER_FORMAT, node.getLiteralIndex()); + } return literalFormatter.map(formatter -> formatter.apply(node)).orElseGet(node::getValue); } @Override protected String visitDoubleLiteral(DoubleLiteral node, Void context) { + if (node.isLiteralMarker()) { + return String.format(LITERAL_MARKER_FORMAT, node.getLiteralIndex()); + } return literalFormatter .map(formatter -> formatter.apply(node)) .orElseGet(() -> doubleFormatter.get().format(node.getValue())); @@ -237,6 +256,9 @@ protected String visitDoubleLiteral(DoubleLiteral node, Void context) { // do not use doubleFormatter, to prevent from introducing the precision noise @Override protected String visitFloatLiteral(FloatLiteral node, Void context) { + if (node.isLiteralMarker()) { + return String.format(LITERAL_MARKER_FORMAT, node.getLiteralIndex()); + } return literalFormatter .map(formatter -> formatter.apply(node)) .orElseGet(() -> String.valueOf(node.getValue())); @@ -244,6 +266,9 @@ protected String visitFloatLiteral(FloatLiteral node, Void context) { @Override protected String visitDecimalLiteral(DecimalLiteral node, Void context) { + if (node.isLiteralMarker()) { + return String.format(LITERAL_MARKER_FORMAT, node.getLiteralIndex()); + } return literalFormatter .map(formatter -> formatter.apply(node)) // TODO return node value without "DECIMAL '..'" when @@ -253,6 +278,9 @@ protected String visitDecimalLiteral(DecimalLiteral node, Void context) { @Override protected String visitGenericLiteral(GenericLiteral node, Void context) { + if (node.isLiteralMarker()) { + return String.format(LITERAL_MARKER_FORMAT, node.getLiteralIndex()); + } return literalFormatter .map(formatter -> formatter.apply(node)) .orElseGet(() -> node.getType() + " " + formatStringLiteral(node.getValue())); @@ -260,6 +288,9 @@ protected String visitGenericLiteral(GenericLiteral node, Void context) { @Override protected String visitNullLiteral(NullLiteral node, Void context) { + if (node.isLiteralMarker()) { + return String.format(LITERAL_MARKER_FORMAT, node.getLiteralIndex()); + } return literalFormatter.map(formatter -> formatter.apply(node)).orElse("null"); } diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/FragmentInstanceStatisticsDrawer.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/FragmentInstanceStatisticsDrawer.java index 5e45c080ae3c9..c8a241d24562d 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/FragmentInstanceStatisticsDrawer.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/FragmentInstanceStatisticsDrawer.java @@ -38,7 +38,7 @@ public class FragmentInstanceStatisticsDrawer { private final List planHeader = new ArrayList<>(); private static final double NS_TO_MS_FACTOR = 1.0 / 1000000; - public void renderPlanStatistics(MPPQueryContext context) { + public void renderPlanStatistics(MPPQueryContext context, boolean verbose) { addLine( planHeader, 0, @@ -70,6 +70,71 @@ public void renderPlanStatistics(MPPQueryContext context) { String.format( "Distribution Plan Cost: %.3f ms", context.getDistributionPlanCost() * NS_TO_MS_FACTOR)); + addLine( + planHeader, + 0, + String.format( + "Plan Cache Status: %s%s", + context.getPlanCacheStatus(), + context.getPlanCacheReason().isEmpty() + ? "" + : String.format(" (%s)", context.getPlanCacheReason()))); + addLine(planHeader, 0, String.format("Plan Cache State: %s", context.getPlanCacheState())); + addLine( + planHeader, + 0, + String.format( + "Plan Cache Lookup Cost: %.3f ms", context.getPlanCacheLookupCost() * NS_TO_MS_FACTOR)); + addLine( + planHeader, + 0, + String.format( + "Saved Logical Planning Cost: %.3f ms", + context.getSavedLogicalPlanningCost() * NS_TO_MS_FACTOR)); + if (verbose) { + renderPlanCacheDiagnostics(context); + } + } + + public void renderPlanStatistics(MPPQueryContext context) { + renderPlanStatistics(context, false); + } + + private void renderPlanCacheDiagnostics(MPPQueryContext context) { + String state = context.getPlanCacheState(); + if ("N/A".equals(state) || "DISABLED".equals(context.getPlanCacheStatus())) { + return; + } + addLine( + planHeader, + 0, + String.format( + "EWMA Reusable Planning Cost: %.3f ms (threshold: %.3f ms)", + context.getEwmaReusablePlanningCost() * NS_TO_MS_FACTOR, + context.getMinReusablePlanningCostThreshold() * NS_TO_MS_FACTOR)); + addLine( + planHeader, + 0, + String.format( + "EWMA First Response Latency: %.3f ms", + context.getEwmaFirstResponseLatency() * NS_TO_MS_FACTOR)); + addLine( + planHeader, + 0, + String.format( + "EWMA Benefit Ratio: %.4f (admit: %.4f, bypass: %.4f)", + context.getEwmaBenefitRatio(), + context.getAdmitRatioThreshold(), + context.getBypassRatioThreshold())); + addLine( + planHeader, + 0, + String.format( + "Profile Counters: samples=%d, hits=%d, misses=%d, bypasses=%d", + context.getProfileSampleCount(), + context.getProfileHitCount(), + context.getProfileMissCount(), + context.getProfileBypassCount())); } public void renderDispatchCost(MPPQueryContext context) { diff --git a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/QueryPlanStatistics.java b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/QueryPlanStatistics.java index edb13217db22c..2332e785022f8 100644 --- a/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/QueryPlanStatistics.java +++ b/iotdb-core/datanode/src/main/java/org/apache/iotdb/db/queryengine/statistics/QueryPlanStatistics.java @@ -20,6 +20,9 @@ package org.apache.iotdb.db.queryengine.statistics; public class QueryPlanStatistics { + private static final String DEFAULT_PLAN_CACHE_STATUS = "DISABLED"; + private static final String DEFAULT_PLAN_CACHE_STATE = "N/A"; + private long analyzeCost; private long fetchPartitionCost; private long fetchSchemaCost; @@ -27,6 +30,25 @@ public class QueryPlanStatistics { private long logicalOptimizationCost; private long distributionPlanCost; private long dispatchCost = 0; + private String planCacheStatus = DEFAULT_PLAN_CACHE_STATUS; + private String planCacheState = DEFAULT_PLAN_CACHE_STATE; + private String planCacheReason = ""; + private long planCacheLookupCost; + private long savedLogicalPlanningCost; + private long reusablePlanningCost; + private long firstResponseLatency; + + // Plan cache profile diagnostics (populated when verbose) + private double ewmaReusablePlanningCost; + private double ewmaFirstResponseLatency; + private double ewmaBenefitRatio; + private long profileSampleCount; + private long profileHitCount; + private long profileMissCount; + private long profileBypassCount; + private long minReusablePlanningCostThreshold; + private double admitRatioThreshold; + private double bypassRatioThreshold; public void setAnalyzeCost(long analyzeCost) { this.analyzeCost = analyzeCost; @@ -83,4 +105,142 @@ public void recordDispatchCost(long dispatchCost) { public long getDispatchCost() { return dispatchCost; } + + public void setPlanCacheStatus(String planCacheStatus) { + this.planCacheStatus = planCacheStatus; + this.planCacheReason = ""; + } + + public void setPlanCacheStatus(String planCacheStatus, String planCacheReason) { + this.planCacheStatus = planCacheStatus; + this.planCacheReason = planCacheReason; + } + + public String getPlanCacheStatus() { + return planCacheStatus; + } + + public String getPlanCacheState() { + return planCacheState; + } + + public void setPlanCacheState(String planCacheState) { + this.planCacheState = planCacheState; + } + + public String getPlanCacheReason() { + return planCacheReason; + } + + public long getPlanCacheLookupCost() { + return planCacheLookupCost; + } + + public void setPlanCacheLookupCost(long planCacheLookupCost) { + this.planCacheLookupCost = planCacheLookupCost; + } + + public long getSavedLogicalPlanningCost() { + return savedLogicalPlanningCost; + } + + public void setSavedLogicalPlanningCost(long savedLogicalPlanningCost) { + this.savedLogicalPlanningCost = savedLogicalPlanningCost; + } + + public long getReusablePlanningCost() { + return reusablePlanningCost; + } + + public void setReusablePlanningCost(long reusablePlanningCost) { + this.reusablePlanningCost = reusablePlanningCost; + } + + public long getFirstResponseLatency() { + return firstResponseLatency; + } + + public void setFirstResponseLatency(long firstResponseLatency) { + this.firstResponseLatency = firstResponseLatency; + } + + public double getEwmaReusablePlanningCost() { + return ewmaReusablePlanningCost; + } + + public void setEwmaReusablePlanningCost(double ewmaReusablePlanningCost) { + this.ewmaReusablePlanningCost = ewmaReusablePlanningCost; + } + + public double getEwmaFirstResponseLatency() { + return ewmaFirstResponseLatency; + } + + public void setEwmaFirstResponseLatency(double ewmaFirstResponseLatency) { + this.ewmaFirstResponseLatency = ewmaFirstResponseLatency; + } + + public double getEwmaBenefitRatio() { + return ewmaBenefitRatio; + } + + public void setEwmaBenefitRatio(double ewmaBenefitRatio) { + this.ewmaBenefitRatio = ewmaBenefitRatio; + } + + public long getProfileSampleCount() { + return profileSampleCount; + } + + public void setProfileSampleCount(long profileSampleCount) { + this.profileSampleCount = profileSampleCount; + } + + public long getProfileHitCount() { + return profileHitCount; + } + + public void setProfileHitCount(long profileHitCount) { + this.profileHitCount = profileHitCount; + } + + public long getProfileMissCount() { + return profileMissCount; + } + + public void setProfileMissCount(long profileMissCount) { + this.profileMissCount = profileMissCount; + } + + public long getProfileBypassCount() { + return profileBypassCount; + } + + public void setProfileBypassCount(long profileBypassCount) { + this.profileBypassCount = profileBypassCount; + } + + public long getMinReusablePlanningCostThreshold() { + return minReusablePlanningCostThreshold; + } + + public void setMinReusablePlanningCostThreshold(long minReusablePlanningCostThreshold) { + this.minReusablePlanningCostThreshold = minReusablePlanningCostThreshold; + } + + public double getAdmitRatioThreshold() { + return admitRatioThreshold; + } + + public void setAdmitRatioThreshold(double admitRatioThreshold) { + this.admitRatioThreshold = admitRatioThreshold; + } + + public double getBypassRatioThreshold() { + return bypassRatioThreshold; + } + + public void setBypassRatioThreshold(double bypassRatioThreshold) { + this.bypassRatioThreshold = bypassRatioThreshold; + } } diff --git a/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanCacheTest.java b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanCacheTest.java new file mode 100644 index 0000000000000..2ed7df0038d29 --- /dev/null +++ b/iotdb-core/datanode/src/test/java/org/apache/iotdb/db/queryengine/plan/relational/planner/PlanCacheTest.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.queryengine.plan.relational.planner; + +import org.apache.iotdb.db.protocol.session.IClientSession; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.LiteralMarkerReplacer; +import org.apache.iotdb.db.queryengine.plan.relational.sql.ast.Statement; +import org.apache.iotdb.db.queryengine.plan.relational.sql.parser.SqlParser; +import org.apache.iotdb.db.queryengine.plan.relational.sql.util.SqlFormatter; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +import java.time.ZoneId; + +public class PlanCacheTest { + + @After + public void tearDown() { + PlanCacheManager.getInstance().clear(); + } + + @Test + public void statementGeneralizationTest() { + String databaseName = "testdb"; + IClientSession clientSession = Mockito.mock(IClientSession.class); + Mockito.when(clientSession.getDatabaseName()).thenReturn(databaseName); + + SqlParser sqlParser = new SqlParser(); + String sql = + "select id + 1 from table1 where id > 10 and time=13289078 and deviceId = 'test' order by rank+1"; + Statement originalStatement = + sqlParser.createStatement(sql, ZoneId.systemDefault(), clientSession); + + LiteralMarkerReplacer literalMarkerReplacer = new LiteralMarkerReplacer(); + literalMarkerReplacer.process(originalStatement); + + String generalizedSql = SqlFormatter.formatSql(originalStatement); + String expectedSql = + String.join( + "\n", + "SELECT (id + LiteralMarker(#0))", + "FROM", + " table1", + "WHERE ((id > LiteralMarker(#1)) AND (time = LiteralMarker(#2)) AND (deviceId = LiteralMarker(#3)))", + "ORDER BY (rank + LiteralMarker(#4)) ASC\n"); + Assert.assertEquals(expectedSql, generalizedSql); + Assert.assertEquals(5, literalMarkerReplacer.getLiteralList().size()); + } + + @Test + public void planCacheStatePromotionTest() { + String cacheKey = "promotion-key"; + PlanCacheManager planCacheManager = PlanCacheManager.getInstance(); + + for (int i = 0; i < 5; i++) { + PlanCacheManager.LookupDecision lookupDecision = planCacheManager.getLookupDecision(cacheKey); + Assert.assertEquals(PlanCacheManager.PlanCacheState.MONITOR, lookupDecision.getState()); + Assert.assertFalse(lookupDecision.shouldLookup()); + planCacheManager.recordExecution(cacheKey, 2_000_000L, 5_000_000L, false); + } + + Assert.assertEquals( + PlanCacheManager.PlanCacheState.ACTIVE, planCacheManager.getTemplateState(cacheKey)); + Assert.assertTrue(planCacheManager.shouldCache(cacheKey)); + Assert.assertTrue(planCacheManager.getEstimatedReusablePlanningCost(cacheKey) >= 2_000_000L); + } + + @Test + public void planCacheStateBypassTest() { + String cacheKey = "bypass-key"; + PlanCacheManager planCacheManager = PlanCacheManager.getInstance(); + + for (int i = 0; i < 5; i++) { + planCacheManager.recordExecution(cacheKey, 100_000L, 5_000_000L, false); + } + + Assert.assertEquals( + PlanCacheManager.PlanCacheState.BYPASS, planCacheManager.getTemplateState(cacheKey)); + + PlanCacheManager.LookupDecision lookupDecision = planCacheManager.getLookupDecision(cacheKey); + Assert.assertEquals(PlanCacheManager.PlanCacheState.BYPASS, lookupDecision.getState()); + Assert.assertFalse(lookupDecision.shouldLookup()); + Assert.assertEquals("Low_Benefit", lookupDecision.getReason()); + } +} diff --git a/scripts/tools/gen_insert_t_d1_d1000.py b/scripts/tools/gen_insert_t_d1_d1000.py new file mode 100644 index 0000000000000..bd00015d7d2c4 --- /dev/null +++ b/scripts/tools/gen_insert_t_d1_d1000.py @@ -0,0 +1,19 @@ +from pathlib import Path + + +def build_sql_lines(start: int = 1, end: int = 1000) -> list[str]: + lines = ["use test;"] + for i in range(start, end + 1): + lines.append(f"insert into t values(now(), 'd{i}', {i});") + return lines + + +def main() -> None: + out_path = Path("insert_t_d1_d1000.sql") + sql = "\n".join(build_sql_lines()) + "\n" + out_path.write_text(sql, encoding="utf-8") + print(f"Generated SQL file: {out_path.resolve()}") + + +if __name__ == "__main__": + main() diff --git a/scripts/tools/insert_t_d1_d1000.sql b/scripts/tools/insert_t_d1_d1000.sql new file mode 100644 index 0000000000000..73627d9947e1e --- /dev/null +++ b/scripts/tools/insert_t_d1_d1000.sql @@ -0,0 +1,1001 @@ +use test; +insert into t values(now(), 'd1', 1); +insert into t values(now(), 'd2', 2); +insert into t values(now(), 'd3', 3); +insert into t values(now(), 'd4', 4); +insert into t values(now(), 'd5', 5); +insert into t values(now(), 'd6', 6); +insert into t values(now(), 'd7', 7); +insert into t values(now(), 'd8', 8); +insert into t values(now(), 'd9', 9); +insert into t values(now(), 'd10', 10); +insert into t values(now(), 'd11', 11); +insert into t values(now(), 'd12', 12); +insert into t values(now(), 'd13', 13); +insert into t values(now(), 'd14', 14); +insert into t values(now(), 'd15', 15); +insert into t values(now(), 'd16', 16); +insert into t values(now(), 'd17', 17); +insert into t values(now(), 'd18', 18); +insert into t values(now(), 'd19', 19); +insert into t values(now(), 'd20', 20); +insert into t values(now(), 'd21', 21); +insert into t values(now(), 'd22', 22); +insert into t values(now(), 'd23', 23); +insert into t values(now(), 'd24', 24); +insert into t values(now(), 'd25', 25); +insert into t values(now(), 'd26', 26); +insert into t values(now(), 'd27', 27); +insert into t values(now(), 'd28', 28); +insert into t values(now(), 'd29', 29); +insert into t values(now(), 'd30', 30); +insert into t values(now(), 'd31', 31); +insert into t values(now(), 'd32', 32); +insert into t values(now(), 'd33', 33); +insert into t values(now(), 'd34', 34); +insert into t values(now(), 'd35', 35); +insert into t values(now(), 'd36', 36); +insert into t values(now(), 'd37', 37); +insert into t values(now(), 'd38', 38); +insert into t values(now(), 'd39', 39); +insert into t values(now(), 'd40', 40); +insert into t values(now(), 'd41', 41); +insert into t values(now(), 'd42', 42); +insert into t values(now(), 'd43', 43); +insert into t values(now(), 'd44', 44); +insert into t values(now(), 'd45', 45); +insert into t values(now(), 'd46', 46); +insert into t values(now(), 'd47', 47); +insert into t values(now(), 'd48', 48); +insert into t values(now(), 'd49', 49); +insert into t values(now(), 'd50', 50); +insert into t values(now(), 'd51', 51); +insert into t values(now(), 'd52', 52); +insert into t values(now(), 'd53', 53); +insert into t values(now(), 'd54', 54); +insert into t values(now(), 'd55', 55); +insert into t values(now(), 'd56', 56); +insert into t values(now(), 'd57', 57); +insert into t values(now(), 'd58', 58); +insert into t values(now(), 'd59', 59); +insert into t values(now(), 'd60', 60); +insert into t values(now(), 'd61', 61); +insert into t values(now(), 'd62', 62); +insert into t values(now(), 'd63', 63); +insert into t values(now(), 'd64', 64); +insert into t values(now(), 'd65', 65); +insert into t values(now(), 'd66', 66); +insert into t values(now(), 'd67', 67); +insert into t values(now(), 'd68', 68); +insert into t values(now(), 'd69', 69); +insert into t values(now(), 'd70', 70); +insert into t values(now(), 'd71', 71); +insert into t values(now(), 'd72', 72); +insert into t values(now(), 'd73', 73); +insert into t values(now(), 'd74', 74); +insert into t values(now(), 'd75', 75); +insert into t values(now(), 'd76', 76); +insert into t values(now(), 'd77', 77); +insert into t values(now(), 'd78', 78); +insert into t values(now(), 'd79', 79); +insert into t values(now(), 'd80', 80); +insert into t values(now(), 'd81', 81); +insert into t values(now(), 'd82', 82); +insert into t values(now(), 'd83', 83); +insert into t values(now(), 'd84', 84); +insert into t values(now(), 'd85', 85); +insert into t values(now(), 'd86', 86); +insert into t values(now(), 'd87', 87); +insert into t values(now(), 'd88', 88); +insert into t values(now(), 'd89', 89); +insert into t values(now(), 'd90', 90); +insert into t values(now(), 'd91', 91); +insert into t values(now(), 'd92', 92); +insert into t values(now(), 'd93', 93); +insert into t values(now(), 'd94', 94); +insert into t values(now(), 'd95', 95); +insert into t values(now(), 'd96', 96); +insert into t values(now(), 'd97', 97); +insert into t values(now(), 'd98', 98); +insert into t values(now(), 'd99', 99); +insert into t values(now(), 'd100', 100); +insert into t values(now(), 'd101', 101); +insert into t values(now(), 'd102', 102); +insert into t values(now(), 'd103', 103); +insert into t values(now(), 'd104', 104); +insert into t values(now(), 'd105', 105); +insert into t values(now(), 'd106', 106); +insert into t values(now(), 'd107', 107); +insert into t values(now(), 'd108', 108); +insert into t values(now(), 'd109', 109); +insert into t values(now(), 'd110', 110); +insert into t values(now(), 'd111', 111); +insert into t values(now(), 'd112', 112); +insert into t values(now(), 'd113', 113); +insert into t values(now(), 'd114', 114); +insert into t values(now(), 'd115', 115); +insert into t values(now(), 'd116', 116); +insert into t values(now(), 'd117', 117); +insert into t values(now(), 'd118', 118); +insert into t values(now(), 'd119', 119); +insert into t values(now(), 'd120', 120); +insert into t values(now(), 'd121', 121); +insert into t values(now(), 'd122', 122); +insert into t values(now(), 'd123', 123); +insert into t values(now(), 'd124', 124); +insert into t values(now(), 'd125', 125); +insert into t values(now(), 'd126', 126); +insert into t values(now(), 'd127', 127); +insert into t values(now(), 'd128', 128); +insert into t values(now(), 'd129', 129); +insert into t values(now(), 'd130', 130); +insert into t values(now(), 'd131', 131); +insert into t values(now(), 'd132', 132); +insert into t values(now(), 'd133', 133); +insert into t values(now(), 'd134', 134); +insert into t values(now(), 'd135', 135); +insert into t values(now(), 'd136', 136); +insert into t values(now(), 'd137', 137); +insert into t values(now(), 'd138', 138); +insert into t values(now(), 'd139', 139); +insert into t values(now(), 'd140', 140); +insert into t values(now(), 'd141', 141); +insert into t values(now(), 'd142', 142); +insert into t values(now(), 'd143', 143); +insert into t values(now(), 'd144', 144); +insert into t values(now(), 'd145', 145); +insert into t values(now(), 'd146', 146); +insert into t values(now(), 'd147', 147); +insert into t values(now(), 'd148', 148); +insert into t values(now(), 'd149', 149); +insert into t values(now(), 'd150', 150); +insert into t values(now(), 'd151', 151); +insert into t values(now(), 'd152', 152); +insert into t values(now(), 'd153', 153); +insert into t values(now(), 'd154', 154); +insert into t values(now(), 'd155', 155); +insert into t values(now(), 'd156', 156); +insert into t values(now(), 'd157', 157); +insert into t values(now(), 'd158', 158); +insert into t values(now(), 'd159', 159); +insert into t values(now(), 'd160', 160); +insert into t values(now(), 'd161', 161); +insert into t values(now(), 'd162', 162); +insert into t values(now(), 'd163', 163); +insert into t values(now(), 'd164', 164); +insert into t values(now(), 'd165', 165); +insert into t values(now(), 'd166', 166); +insert into t values(now(), 'd167', 167); +insert into t values(now(), 'd168', 168); +insert into t values(now(), 'd169', 169); +insert into t values(now(), 'd170', 170); +insert into t values(now(), 'd171', 171); +insert into t values(now(), 'd172', 172); +insert into t values(now(), 'd173', 173); +insert into t values(now(), 'd174', 174); +insert into t values(now(), 'd175', 175); +insert into t values(now(), 'd176', 176); +insert into t values(now(), 'd177', 177); +insert into t values(now(), 'd178', 178); +insert into t values(now(), 'd179', 179); +insert into t values(now(), 'd180', 180); +insert into t values(now(), 'd181', 181); +insert into t values(now(), 'd182', 182); +insert into t values(now(), 'd183', 183); +insert into t values(now(), 'd184', 184); +insert into t values(now(), 'd185', 185); +insert into t values(now(), 'd186', 186); +insert into t values(now(), 'd187', 187); +insert into t values(now(), 'd188', 188); +insert into t values(now(), 'd189', 189); +insert into t values(now(), 'd190', 190); +insert into t values(now(), 'd191', 191); +insert into t values(now(), 'd192', 192); +insert into t values(now(), 'd193', 193); +insert into t values(now(), 'd194', 194); +insert into t values(now(), 'd195', 195); +insert into t values(now(), 'd196', 196); +insert into t values(now(), 'd197', 197); +insert into t values(now(), 'd198', 198); +insert into t values(now(), 'd199', 199); +insert into t values(now(), 'd200', 200); +insert into t values(now(), 'd201', 201); +insert into t values(now(), 'd202', 202); +insert into t values(now(), 'd203', 203); +insert into t values(now(), 'd204', 204); +insert into t values(now(), 'd205', 205); +insert into t values(now(), 'd206', 206); +insert into t values(now(), 'd207', 207); +insert into t values(now(), 'd208', 208); +insert into t values(now(), 'd209', 209); +insert into t values(now(), 'd210', 210); +insert into t values(now(), 'd211', 211); +insert into t values(now(), 'd212', 212); +insert into t values(now(), 'd213', 213); +insert into t values(now(), 'd214', 214); +insert into t values(now(), 'd215', 215); +insert into t values(now(), 'd216', 216); +insert into t values(now(), 'd217', 217); +insert into t values(now(), 'd218', 218); +insert into t values(now(), 'd219', 219); +insert into t values(now(), 'd220', 220); +insert into t values(now(), 'd221', 221); +insert into t values(now(), 'd222', 222); +insert into t values(now(), 'd223', 223); +insert into t values(now(), 'd224', 224); +insert into t values(now(), 'd225', 225); +insert into t values(now(), 'd226', 226); +insert into t values(now(), 'd227', 227); +insert into t values(now(), 'd228', 228); +insert into t values(now(), 'd229', 229); +insert into t values(now(), 'd230', 230); +insert into t values(now(), 'd231', 231); +insert into t values(now(), 'd232', 232); +insert into t values(now(), 'd233', 233); +insert into t values(now(), 'd234', 234); +insert into t values(now(), 'd235', 235); +insert into t values(now(), 'd236', 236); +insert into t values(now(), 'd237', 237); +insert into t values(now(), 'd238', 238); +insert into t values(now(), 'd239', 239); +insert into t values(now(), 'd240', 240); +insert into t values(now(), 'd241', 241); +insert into t values(now(), 'd242', 242); +insert into t values(now(), 'd243', 243); +insert into t values(now(), 'd244', 244); +insert into t values(now(), 'd245', 245); +insert into t values(now(), 'd246', 246); +insert into t values(now(), 'd247', 247); +insert into t values(now(), 'd248', 248); +insert into t values(now(), 'd249', 249); +insert into t values(now(), 'd250', 250); +insert into t values(now(), 'd251', 251); +insert into t values(now(), 'd252', 252); +insert into t values(now(), 'd253', 253); +insert into t values(now(), 'd254', 254); +insert into t values(now(), 'd255', 255); +insert into t values(now(), 'd256', 256); +insert into t values(now(), 'd257', 257); +insert into t values(now(), 'd258', 258); +insert into t values(now(), 'd259', 259); +insert into t values(now(), 'd260', 260); +insert into t values(now(), 'd261', 261); +insert into t values(now(), 'd262', 262); +insert into t values(now(), 'd263', 263); +insert into t values(now(), 'd264', 264); +insert into t values(now(), 'd265', 265); +insert into t values(now(), 'd266', 266); +insert into t values(now(), 'd267', 267); +insert into t values(now(), 'd268', 268); +insert into t values(now(), 'd269', 269); +insert into t values(now(), 'd270', 270); +insert into t values(now(), 'd271', 271); +insert into t values(now(), 'd272', 272); +insert into t values(now(), 'd273', 273); +insert into t values(now(), 'd274', 274); +insert into t values(now(), 'd275', 275); +insert into t values(now(), 'd276', 276); +insert into t values(now(), 'd277', 277); +insert into t values(now(), 'd278', 278); +insert into t values(now(), 'd279', 279); +insert into t values(now(), 'd280', 280); +insert into t values(now(), 'd281', 281); +insert into t values(now(), 'd282', 282); +insert into t values(now(), 'd283', 283); +insert into t values(now(), 'd284', 284); +insert into t values(now(), 'd285', 285); +insert into t values(now(), 'd286', 286); +insert into t values(now(), 'd287', 287); +insert into t values(now(), 'd288', 288); +insert into t values(now(), 'd289', 289); +insert into t values(now(), 'd290', 290); +insert into t values(now(), 'd291', 291); +insert into t values(now(), 'd292', 292); +insert into t values(now(), 'd293', 293); +insert into t values(now(), 'd294', 294); +insert into t values(now(), 'd295', 295); +insert into t values(now(), 'd296', 296); +insert into t values(now(), 'd297', 297); +insert into t values(now(), 'd298', 298); +insert into t values(now(), 'd299', 299); +insert into t values(now(), 'd300', 300); +insert into t values(now(), 'd301', 301); +insert into t values(now(), 'd302', 302); +insert into t values(now(), 'd303', 303); +insert into t values(now(), 'd304', 304); +insert into t values(now(), 'd305', 305); +insert into t values(now(), 'd306', 306); +insert into t values(now(), 'd307', 307); +insert into t values(now(), 'd308', 308); +insert into t values(now(), 'd309', 309); +insert into t values(now(), 'd310', 310); +insert into t values(now(), 'd311', 311); +insert into t values(now(), 'd312', 312); +insert into t values(now(), 'd313', 313); +insert into t values(now(), 'd314', 314); +insert into t values(now(), 'd315', 315); +insert into t values(now(), 'd316', 316); +insert into t values(now(), 'd317', 317); +insert into t values(now(), 'd318', 318); +insert into t values(now(), 'd319', 319); +insert into t values(now(), 'd320', 320); +insert into t values(now(), 'd321', 321); +insert into t values(now(), 'd322', 322); +insert into t values(now(), 'd323', 323); +insert into t values(now(), 'd324', 324); +insert into t values(now(), 'd325', 325); +insert into t values(now(), 'd326', 326); +insert into t values(now(), 'd327', 327); +insert into t values(now(), 'd328', 328); +insert into t values(now(), 'd329', 329); +insert into t values(now(), 'd330', 330); +insert into t values(now(), 'd331', 331); +insert into t values(now(), 'd332', 332); +insert into t values(now(), 'd333', 333); +insert into t values(now(), 'd334', 334); +insert into t values(now(), 'd335', 335); +insert into t values(now(), 'd336', 336); +insert into t values(now(), 'd337', 337); +insert into t values(now(), 'd338', 338); +insert into t values(now(), 'd339', 339); +insert into t values(now(), 'd340', 340); +insert into t values(now(), 'd341', 341); +insert into t values(now(), 'd342', 342); +insert into t values(now(), 'd343', 343); +insert into t values(now(), 'd344', 344); +insert into t values(now(), 'd345', 345); +insert into t values(now(), 'd346', 346); +insert into t values(now(), 'd347', 347); +insert into t values(now(), 'd348', 348); +insert into t values(now(), 'd349', 349); +insert into t values(now(), 'd350', 350); +insert into t values(now(), 'd351', 351); +insert into t values(now(), 'd352', 352); +insert into t values(now(), 'd353', 353); +insert into t values(now(), 'd354', 354); +insert into t values(now(), 'd355', 355); +insert into t values(now(), 'd356', 356); +insert into t values(now(), 'd357', 357); +insert into t values(now(), 'd358', 358); +insert into t values(now(), 'd359', 359); +insert into t values(now(), 'd360', 360); +insert into t values(now(), 'd361', 361); +insert into t values(now(), 'd362', 362); +insert into t values(now(), 'd363', 363); +insert into t values(now(), 'd364', 364); +insert into t values(now(), 'd365', 365); +insert into t values(now(), 'd366', 366); +insert into t values(now(), 'd367', 367); +insert into t values(now(), 'd368', 368); +insert into t values(now(), 'd369', 369); +insert into t values(now(), 'd370', 370); +insert into t values(now(), 'd371', 371); +insert into t values(now(), 'd372', 372); +insert into t values(now(), 'd373', 373); +insert into t values(now(), 'd374', 374); +insert into t values(now(), 'd375', 375); +insert into t values(now(), 'd376', 376); +insert into t values(now(), 'd377', 377); +insert into t values(now(), 'd378', 378); +insert into t values(now(), 'd379', 379); +insert into t values(now(), 'd380', 380); +insert into t values(now(), 'd381', 381); +insert into t values(now(), 'd382', 382); +insert into t values(now(), 'd383', 383); +insert into t values(now(), 'd384', 384); +insert into t values(now(), 'd385', 385); +insert into t values(now(), 'd386', 386); +insert into t values(now(), 'd387', 387); +insert into t values(now(), 'd388', 388); +insert into t values(now(), 'd389', 389); +insert into t values(now(), 'd390', 390); +insert into t values(now(), 'd391', 391); +insert into t values(now(), 'd392', 392); +insert into t values(now(), 'd393', 393); +insert into t values(now(), 'd394', 394); +insert into t values(now(), 'd395', 395); +insert into t values(now(), 'd396', 396); +insert into t values(now(), 'd397', 397); +insert into t values(now(), 'd398', 398); +insert into t values(now(), 'd399', 399); +insert into t values(now(), 'd400', 400); +insert into t values(now(), 'd401', 401); +insert into t values(now(), 'd402', 402); +insert into t values(now(), 'd403', 403); +insert into t values(now(), 'd404', 404); +insert into t values(now(), 'd405', 405); +insert into t values(now(), 'd406', 406); +insert into t values(now(), 'd407', 407); +insert into t values(now(), 'd408', 408); +insert into t values(now(), 'd409', 409); +insert into t values(now(), 'd410', 410); +insert into t values(now(), 'd411', 411); +insert into t values(now(), 'd412', 412); +insert into t values(now(), 'd413', 413); +insert into t values(now(), 'd414', 414); +insert into t values(now(), 'd415', 415); +insert into t values(now(), 'd416', 416); +insert into t values(now(), 'd417', 417); +insert into t values(now(), 'd418', 418); +insert into t values(now(), 'd419', 419); +insert into t values(now(), 'd420', 420); +insert into t values(now(), 'd421', 421); +insert into t values(now(), 'd422', 422); +insert into t values(now(), 'd423', 423); +insert into t values(now(), 'd424', 424); +insert into t values(now(), 'd425', 425); +insert into t values(now(), 'd426', 426); +insert into t values(now(), 'd427', 427); +insert into t values(now(), 'd428', 428); +insert into t values(now(), 'd429', 429); +insert into t values(now(), 'd430', 430); +insert into t values(now(), 'd431', 431); +insert into t values(now(), 'd432', 432); +insert into t values(now(), 'd433', 433); +insert into t values(now(), 'd434', 434); +insert into t values(now(), 'd435', 435); +insert into t values(now(), 'd436', 436); +insert into t values(now(), 'd437', 437); +insert into t values(now(), 'd438', 438); +insert into t values(now(), 'd439', 439); +insert into t values(now(), 'd440', 440); +insert into t values(now(), 'd441', 441); +insert into t values(now(), 'd442', 442); +insert into t values(now(), 'd443', 443); +insert into t values(now(), 'd444', 444); +insert into t values(now(), 'd445', 445); +insert into t values(now(), 'd446', 446); +insert into t values(now(), 'd447', 447); +insert into t values(now(), 'd448', 448); +insert into t values(now(), 'd449', 449); +insert into t values(now(), 'd450', 450); +insert into t values(now(), 'd451', 451); +insert into t values(now(), 'd452', 452); +insert into t values(now(), 'd453', 453); +insert into t values(now(), 'd454', 454); +insert into t values(now(), 'd455', 455); +insert into t values(now(), 'd456', 456); +insert into t values(now(), 'd457', 457); +insert into t values(now(), 'd458', 458); +insert into t values(now(), 'd459', 459); +insert into t values(now(), 'd460', 460); +insert into t values(now(), 'd461', 461); +insert into t values(now(), 'd462', 462); +insert into t values(now(), 'd463', 463); +insert into t values(now(), 'd464', 464); +insert into t values(now(), 'd465', 465); +insert into t values(now(), 'd466', 466); +insert into t values(now(), 'd467', 467); +insert into t values(now(), 'd468', 468); +insert into t values(now(), 'd469', 469); +insert into t values(now(), 'd470', 470); +insert into t values(now(), 'd471', 471); +insert into t values(now(), 'd472', 472); +insert into t values(now(), 'd473', 473); +insert into t values(now(), 'd474', 474); +insert into t values(now(), 'd475', 475); +insert into t values(now(), 'd476', 476); +insert into t values(now(), 'd477', 477); +insert into t values(now(), 'd478', 478); +insert into t values(now(), 'd479', 479); +insert into t values(now(), 'd480', 480); +insert into t values(now(), 'd481', 481); +insert into t values(now(), 'd482', 482); +insert into t values(now(), 'd483', 483); +insert into t values(now(), 'd484', 484); +insert into t values(now(), 'd485', 485); +insert into t values(now(), 'd486', 486); +insert into t values(now(), 'd487', 487); +insert into t values(now(), 'd488', 488); +insert into t values(now(), 'd489', 489); +insert into t values(now(), 'd490', 490); +insert into t values(now(), 'd491', 491); +insert into t values(now(), 'd492', 492); +insert into t values(now(), 'd493', 493); +insert into t values(now(), 'd494', 494); +insert into t values(now(), 'd495', 495); +insert into t values(now(), 'd496', 496); +insert into t values(now(), 'd497', 497); +insert into t values(now(), 'd498', 498); +insert into t values(now(), 'd499', 499); +insert into t values(now(), 'd500', 500); +insert into t values(now(), 'd501', 501); +insert into t values(now(), 'd502', 502); +insert into t values(now(), 'd503', 503); +insert into t values(now(), 'd504', 504); +insert into t values(now(), 'd505', 505); +insert into t values(now(), 'd506', 506); +insert into t values(now(), 'd507', 507); +insert into t values(now(), 'd508', 508); +insert into t values(now(), 'd509', 509); +insert into t values(now(), 'd510', 510); +insert into t values(now(), 'd511', 511); +insert into t values(now(), 'd512', 512); +insert into t values(now(), 'd513', 513); +insert into t values(now(), 'd514', 514); +insert into t values(now(), 'd515', 515); +insert into t values(now(), 'd516', 516); +insert into t values(now(), 'd517', 517); +insert into t values(now(), 'd518', 518); +insert into t values(now(), 'd519', 519); +insert into t values(now(), 'd520', 520); +insert into t values(now(), 'd521', 521); +insert into t values(now(), 'd522', 522); +insert into t values(now(), 'd523', 523); +insert into t values(now(), 'd524', 524); +insert into t values(now(), 'd525', 525); +insert into t values(now(), 'd526', 526); +insert into t values(now(), 'd527', 527); +insert into t values(now(), 'd528', 528); +insert into t values(now(), 'd529', 529); +insert into t values(now(), 'd530', 530); +insert into t values(now(), 'd531', 531); +insert into t values(now(), 'd532', 532); +insert into t values(now(), 'd533', 533); +insert into t values(now(), 'd534', 534); +insert into t values(now(), 'd535', 535); +insert into t values(now(), 'd536', 536); +insert into t values(now(), 'd537', 537); +insert into t values(now(), 'd538', 538); +insert into t values(now(), 'd539', 539); +insert into t values(now(), 'd540', 540); +insert into t values(now(), 'd541', 541); +insert into t values(now(), 'd542', 542); +insert into t values(now(), 'd543', 543); +insert into t values(now(), 'd544', 544); +insert into t values(now(), 'd545', 545); +insert into t values(now(), 'd546', 546); +insert into t values(now(), 'd547', 547); +insert into t values(now(), 'd548', 548); +insert into t values(now(), 'd549', 549); +insert into t values(now(), 'd550', 550); +insert into t values(now(), 'd551', 551); +insert into t values(now(), 'd552', 552); +insert into t values(now(), 'd553', 553); +insert into t values(now(), 'd554', 554); +insert into t values(now(), 'd555', 555); +insert into t values(now(), 'd556', 556); +insert into t values(now(), 'd557', 557); +insert into t values(now(), 'd558', 558); +insert into t values(now(), 'd559', 559); +insert into t values(now(), 'd560', 560); +insert into t values(now(), 'd561', 561); +insert into t values(now(), 'd562', 562); +insert into t values(now(), 'd563', 563); +insert into t values(now(), 'd564', 564); +insert into t values(now(), 'd565', 565); +insert into t values(now(), 'd566', 566); +insert into t values(now(), 'd567', 567); +insert into t values(now(), 'd568', 568); +insert into t values(now(), 'd569', 569); +insert into t values(now(), 'd570', 570); +insert into t values(now(), 'd571', 571); +insert into t values(now(), 'd572', 572); +insert into t values(now(), 'd573', 573); +insert into t values(now(), 'd574', 574); +insert into t values(now(), 'd575', 575); +insert into t values(now(), 'd576', 576); +insert into t values(now(), 'd577', 577); +insert into t values(now(), 'd578', 578); +insert into t values(now(), 'd579', 579); +insert into t values(now(), 'd580', 580); +insert into t values(now(), 'd581', 581); +insert into t values(now(), 'd582', 582); +insert into t values(now(), 'd583', 583); +insert into t values(now(), 'd584', 584); +insert into t values(now(), 'd585', 585); +insert into t values(now(), 'd586', 586); +insert into t values(now(), 'd587', 587); +insert into t values(now(), 'd588', 588); +insert into t values(now(), 'd589', 589); +insert into t values(now(), 'd590', 590); +insert into t values(now(), 'd591', 591); +insert into t values(now(), 'd592', 592); +insert into t values(now(), 'd593', 593); +insert into t values(now(), 'd594', 594); +insert into t values(now(), 'd595', 595); +insert into t values(now(), 'd596', 596); +insert into t values(now(), 'd597', 597); +insert into t values(now(), 'd598', 598); +insert into t values(now(), 'd599', 599); +insert into t values(now(), 'd600', 600); +insert into t values(now(), 'd601', 601); +insert into t values(now(), 'd602', 602); +insert into t values(now(), 'd603', 603); +insert into t values(now(), 'd604', 604); +insert into t values(now(), 'd605', 605); +insert into t values(now(), 'd606', 606); +insert into t values(now(), 'd607', 607); +insert into t values(now(), 'd608', 608); +insert into t values(now(), 'd609', 609); +insert into t values(now(), 'd610', 610); +insert into t values(now(), 'd611', 611); +insert into t values(now(), 'd612', 612); +insert into t values(now(), 'd613', 613); +insert into t values(now(), 'd614', 614); +insert into t values(now(), 'd615', 615); +insert into t values(now(), 'd616', 616); +insert into t values(now(), 'd617', 617); +insert into t values(now(), 'd618', 618); +insert into t values(now(), 'd619', 619); +insert into t values(now(), 'd620', 620); +insert into t values(now(), 'd621', 621); +insert into t values(now(), 'd622', 622); +insert into t values(now(), 'd623', 623); +insert into t values(now(), 'd624', 624); +insert into t values(now(), 'd625', 625); +insert into t values(now(), 'd626', 626); +insert into t values(now(), 'd627', 627); +insert into t values(now(), 'd628', 628); +insert into t values(now(), 'd629', 629); +insert into t values(now(), 'd630', 630); +insert into t values(now(), 'd631', 631); +insert into t values(now(), 'd632', 632); +insert into t values(now(), 'd633', 633); +insert into t values(now(), 'd634', 634); +insert into t values(now(), 'd635', 635); +insert into t values(now(), 'd636', 636); +insert into t values(now(), 'd637', 637); +insert into t values(now(), 'd638', 638); +insert into t values(now(), 'd639', 639); +insert into t values(now(), 'd640', 640); +insert into t values(now(), 'd641', 641); +insert into t values(now(), 'd642', 642); +insert into t values(now(), 'd643', 643); +insert into t values(now(), 'd644', 644); +insert into t values(now(), 'd645', 645); +insert into t values(now(), 'd646', 646); +insert into t values(now(), 'd647', 647); +insert into t values(now(), 'd648', 648); +insert into t values(now(), 'd649', 649); +insert into t values(now(), 'd650', 650); +insert into t values(now(), 'd651', 651); +insert into t values(now(), 'd652', 652); +insert into t values(now(), 'd653', 653); +insert into t values(now(), 'd654', 654); +insert into t values(now(), 'd655', 655); +insert into t values(now(), 'd656', 656); +insert into t values(now(), 'd657', 657); +insert into t values(now(), 'd658', 658); +insert into t values(now(), 'd659', 659); +insert into t values(now(), 'd660', 660); +insert into t values(now(), 'd661', 661); +insert into t values(now(), 'd662', 662); +insert into t values(now(), 'd663', 663); +insert into t values(now(), 'd664', 664); +insert into t values(now(), 'd665', 665); +insert into t values(now(), 'd666', 666); +insert into t values(now(), 'd667', 667); +insert into t values(now(), 'd668', 668); +insert into t values(now(), 'd669', 669); +insert into t values(now(), 'd670', 670); +insert into t values(now(), 'd671', 671); +insert into t values(now(), 'd672', 672); +insert into t values(now(), 'd673', 673); +insert into t values(now(), 'd674', 674); +insert into t values(now(), 'd675', 675); +insert into t values(now(), 'd676', 676); +insert into t values(now(), 'd677', 677); +insert into t values(now(), 'd678', 678); +insert into t values(now(), 'd679', 679); +insert into t values(now(), 'd680', 680); +insert into t values(now(), 'd681', 681); +insert into t values(now(), 'd682', 682); +insert into t values(now(), 'd683', 683); +insert into t values(now(), 'd684', 684); +insert into t values(now(), 'd685', 685); +insert into t values(now(), 'd686', 686); +insert into t values(now(), 'd687', 687); +insert into t values(now(), 'd688', 688); +insert into t values(now(), 'd689', 689); +insert into t values(now(), 'd690', 690); +insert into t values(now(), 'd691', 691); +insert into t values(now(), 'd692', 692); +insert into t values(now(), 'd693', 693); +insert into t values(now(), 'd694', 694); +insert into t values(now(), 'd695', 695); +insert into t values(now(), 'd696', 696); +insert into t values(now(), 'd697', 697); +insert into t values(now(), 'd698', 698); +insert into t values(now(), 'd699', 699); +insert into t values(now(), 'd700', 700); +insert into t values(now(), 'd701', 701); +insert into t values(now(), 'd702', 702); +insert into t values(now(), 'd703', 703); +insert into t values(now(), 'd704', 704); +insert into t values(now(), 'd705', 705); +insert into t values(now(), 'd706', 706); +insert into t values(now(), 'd707', 707); +insert into t values(now(), 'd708', 708); +insert into t values(now(), 'd709', 709); +insert into t values(now(), 'd710', 710); +insert into t values(now(), 'd711', 711); +insert into t values(now(), 'd712', 712); +insert into t values(now(), 'd713', 713); +insert into t values(now(), 'd714', 714); +insert into t values(now(), 'd715', 715); +insert into t values(now(), 'd716', 716); +insert into t values(now(), 'd717', 717); +insert into t values(now(), 'd718', 718); +insert into t values(now(), 'd719', 719); +insert into t values(now(), 'd720', 720); +insert into t values(now(), 'd721', 721); +insert into t values(now(), 'd722', 722); +insert into t values(now(), 'd723', 723); +insert into t values(now(), 'd724', 724); +insert into t values(now(), 'd725', 725); +insert into t values(now(), 'd726', 726); +insert into t values(now(), 'd727', 727); +insert into t values(now(), 'd728', 728); +insert into t values(now(), 'd729', 729); +insert into t values(now(), 'd730', 730); +insert into t values(now(), 'd731', 731); +insert into t values(now(), 'd732', 732); +insert into t values(now(), 'd733', 733); +insert into t values(now(), 'd734', 734); +insert into t values(now(), 'd735', 735); +insert into t values(now(), 'd736', 736); +insert into t values(now(), 'd737', 737); +insert into t values(now(), 'd738', 738); +insert into t values(now(), 'd739', 739); +insert into t values(now(), 'd740', 740); +insert into t values(now(), 'd741', 741); +insert into t values(now(), 'd742', 742); +insert into t values(now(), 'd743', 743); +insert into t values(now(), 'd744', 744); +insert into t values(now(), 'd745', 745); +insert into t values(now(), 'd746', 746); +insert into t values(now(), 'd747', 747); +insert into t values(now(), 'd748', 748); +insert into t values(now(), 'd749', 749); +insert into t values(now(), 'd750', 750); +insert into t values(now(), 'd751', 751); +insert into t values(now(), 'd752', 752); +insert into t values(now(), 'd753', 753); +insert into t values(now(), 'd754', 754); +insert into t values(now(), 'd755', 755); +insert into t values(now(), 'd756', 756); +insert into t values(now(), 'd757', 757); +insert into t values(now(), 'd758', 758); +insert into t values(now(), 'd759', 759); +insert into t values(now(), 'd760', 760); +insert into t values(now(), 'd761', 761); +insert into t values(now(), 'd762', 762); +insert into t values(now(), 'd763', 763); +insert into t values(now(), 'd764', 764); +insert into t values(now(), 'd765', 765); +insert into t values(now(), 'd766', 766); +insert into t values(now(), 'd767', 767); +insert into t values(now(), 'd768', 768); +insert into t values(now(), 'd769', 769); +insert into t values(now(), 'd770', 770); +insert into t values(now(), 'd771', 771); +insert into t values(now(), 'd772', 772); +insert into t values(now(), 'd773', 773); +insert into t values(now(), 'd774', 774); +insert into t values(now(), 'd775', 775); +insert into t values(now(), 'd776', 776); +insert into t values(now(), 'd777', 777); +insert into t values(now(), 'd778', 778); +insert into t values(now(), 'd779', 779); +insert into t values(now(), 'd780', 780); +insert into t values(now(), 'd781', 781); +insert into t values(now(), 'd782', 782); +insert into t values(now(), 'd783', 783); +insert into t values(now(), 'd784', 784); +insert into t values(now(), 'd785', 785); +insert into t values(now(), 'd786', 786); +insert into t values(now(), 'd787', 787); +insert into t values(now(), 'd788', 788); +insert into t values(now(), 'd789', 789); +insert into t values(now(), 'd790', 790); +insert into t values(now(), 'd791', 791); +insert into t values(now(), 'd792', 792); +insert into t values(now(), 'd793', 793); +insert into t values(now(), 'd794', 794); +insert into t values(now(), 'd795', 795); +insert into t values(now(), 'd796', 796); +insert into t values(now(), 'd797', 797); +insert into t values(now(), 'd798', 798); +insert into t values(now(), 'd799', 799); +insert into t values(now(), 'd800', 800); +insert into t values(now(), 'd801', 801); +insert into t values(now(), 'd802', 802); +insert into t values(now(), 'd803', 803); +insert into t values(now(), 'd804', 804); +insert into t values(now(), 'd805', 805); +insert into t values(now(), 'd806', 806); +insert into t values(now(), 'd807', 807); +insert into t values(now(), 'd808', 808); +insert into t values(now(), 'd809', 809); +insert into t values(now(), 'd810', 810); +insert into t values(now(), 'd811', 811); +insert into t values(now(), 'd812', 812); +insert into t values(now(), 'd813', 813); +insert into t values(now(), 'd814', 814); +insert into t values(now(), 'd815', 815); +insert into t values(now(), 'd816', 816); +insert into t values(now(), 'd817', 817); +insert into t values(now(), 'd818', 818); +insert into t values(now(), 'd819', 819); +insert into t values(now(), 'd820', 820); +insert into t values(now(), 'd821', 821); +insert into t values(now(), 'd822', 822); +insert into t values(now(), 'd823', 823); +insert into t values(now(), 'd824', 824); +insert into t values(now(), 'd825', 825); +insert into t values(now(), 'd826', 826); +insert into t values(now(), 'd827', 827); +insert into t values(now(), 'd828', 828); +insert into t values(now(), 'd829', 829); +insert into t values(now(), 'd830', 830); +insert into t values(now(), 'd831', 831); +insert into t values(now(), 'd832', 832); +insert into t values(now(), 'd833', 833); +insert into t values(now(), 'd834', 834); +insert into t values(now(), 'd835', 835); +insert into t values(now(), 'd836', 836); +insert into t values(now(), 'd837', 837); +insert into t values(now(), 'd838', 838); +insert into t values(now(), 'd839', 839); +insert into t values(now(), 'd840', 840); +insert into t values(now(), 'd841', 841); +insert into t values(now(), 'd842', 842); +insert into t values(now(), 'd843', 843); +insert into t values(now(), 'd844', 844); +insert into t values(now(), 'd845', 845); +insert into t values(now(), 'd846', 846); +insert into t values(now(), 'd847', 847); +insert into t values(now(), 'd848', 848); +insert into t values(now(), 'd849', 849); +insert into t values(now(), 'd850', 850); +insert into t values(now(), 'd851', 851); +insert into t values(now(), 'd852', 852); +insert into t values(now(), 'd853', 853); +insert into t values(now(), 'd854', 854); +insert into t values(now(), 'd855', 855); +insert into t values(now(), 'd856', 856); +insert into t values(now(), 'd857', 857); +insert into t values(now(), 'd858', 858); +insert into t values(now(), 'd859', 859); +insert into t values(now(), 'd860', 860); +insert into t values(now(), 'd861', 861); +insert into t values(now(), 'd862', 862); +insert into t values(now(), 'd863', 863); +insert into t values(now(), 'd864', 864); +insert into t values(now(), 'd865', 865); +insert into t values(now(), 'd866', 866); +insert into t values(now(), 'd867', 867); +insert into t values(now(), 'd868', 868); +insert into t values(now(), 'd869', 869); +insert into t values(now(), 'd870', 870); +insert into t values(now(), 'd871', 871); +insert into t values(now(), 'd872', 872); +insert into t values(now(), 'd873', 873); +insert into t values(now(), 'd874', 874); +insert into t values(now(), 'd875', 875); +insert into t values(now(), 'd876', 876); +insert into t values(now(), 'd877', 877); +insert into t values(now(), 'd878', 878); +insert into t values(now(), 'd879', 879); +insert into t values(now(), 'd880', 880); +insert into t values(now(), 'd881', 881); +insert into t values(now(), 'd882', 882); +insert into t values(now(), 'd883', 883); +insert into t values(now(), 'd884', 884); +insert into t values(now(), 'd885', 885); +insert into t values(now(), 'd886', 886); +insert into t values(now(), 'd887', 887); +insert into t values(now(), 'd888', 888); +insert into t values(now(), 'd889', 889); +insert into t values(now(), 'd890', 890); +insert into t values(now(), 'd891', 891); +insert into t values(now(), 'd892', 892); +insert into t values(now(), 'd893', 893); +insert into t values(now(), 'd894', 894); +insert into t values(now(), 'd895', 895); +insert into t values(now(), 'd896', 896); +insert into t values(now(), 'd897', 897); +insert into t values(now(), 'd898', 898); +insert into t values(now(), 'd899', 899); +insert into t values(now(), 'd900', 900); +insert into t values(now(), 'd901', 901); +insert into t values(now(), 'd902', 902); +insert into t values(now(), 'd903', 903); +insert into t values(now(), 'd904', 904); +insert into t values(now(), 'd905', 905); +insert into t values(now(), 'd906', 906); +insert into t values(now(), 'd907', 907); +insert into t values(now(), 'd908', 908); +insert into t values(now(), 'd909', 909); +insert into t values(now(), 'd910', 910); +insert into t values(now(), 'd911', 911); +insert into t values(now(), 'd912', 912); +insert into t values(now(), 'd913', 913); +insert into t values(now(), 'd914', 914); +insert into t values(now(), 'd915', 915); +insert into t values(now(), 'd916', 916); +insert into t values(now(), 'd917', 917); +insert into t values(now(), 'd918', 918); +insert into t values(now(), 'd919', 919); +insert into t values(now(), 'd920', 920); +insert into t values(now(), 'd921', 921); +insert into t values(now(), 'd922', 922); +insert into t values(now(), 'd923', 923); +insert into t values(now(), 'd924', 924); +insert into t values(now(), 'd925', 925); +insert into t values(now(), 'd926', 926); +insert into t values(now(), 'd927', 927); +insert into t values(now(), 'd928', 928); +insert into t values(now(), 'd929', 929); +insert into t values(now(), 'd930', 930); +insert into t values(now(), 'd931', 931); +insert into t values(now(), 'd932', 932); +insert into t values(now(), 'd933', 933); +insert into t values(now(), 'd934', 934); +insert into t values(now(), 'd935', 935); +insert into t values(now(), 'd936', 936); +insert into t values(now(), 'd937', 937); +insert into t values(now(), 'd938', 938); +insert into t values(now(), 'd939', 939); +insert into t values(now(), 'd940', 940); +insert into t values(now(), 'd941', 941); +insert into t values(now(), 'd942', 942); +insert into t values(now(), 'd943', 943); +insert into t values(now(), 'd944', 944); +insert into t values(now(), 'd945', 945); +insert into t values(now(), 'd946', 946); +insert into t values(now(), 'd947', 947); +insert into t values(now(), 'd948', 948); +insert into t values(now(), 'd949', 949); +insert into t values(now(), 'd950', 950); +insert into t values(now(), 'd951', 951); +insert into t values(now(), 'd952', 952); +insert into t values(now(), 'd953', 953); +insert into t values(now(), 'd954', 954); +insert into t values(now(), 'd955', 955); +insert into t values(now(), 'd956', 956); +insert into t values(now(), 'd957', 957); +insert into t values(now(), 'd958', 958); +insert into t values(now(), 'd959', 959); +insert into t values(now(), 'd960', 960); +insert into t values(now(), 'd961', 961); +insert into t values(now(), 'd962', 962); +insert into t values(now(), 'd963', 963); +insert into t values(now(), 'd964', 964); +insert into t values(now(), 'd965', 965); +insert into t values(now(), 'd966', 966); +insert into t values(now(), 'd967', 967); +insert into t values(now(), 'd968', 968); +insert into t values(now(), 'd969', 969); +insert into t values(now(), 'd970', 970); +insert into t values(now(), 'd971', 971); +insert into t values(now(), 'd972', 972); +insert into t values(now(), 'd973', 973); +insert into t values(now(), 'd974', 974); +insert into t values(now(), 'd975', 975); +insert into t values(now(), 'd976', 976); +insert into t values(now(), 'd977', 977); +insert into t values(now(), 'd978', 978); +insert into t values(now(), 'd979', 979); +insert into t values(now(), 'd980', 980); +insert into t values(now(), 'd981', 981); +insert into t values(now(), 'd982', 982); +insert into t values(now(), 'd983', 983); +insert into t values(now(), 'd984', 984); +insert into t values(now(), 'd985', 985); +insert into t values(now(), 'd986', 986); +insert into t values(now(), 'd987', 987); +insert into t values(now(), 'd988', 988); +insert into t values(now(), 'd989', 989); +insert into t values(now(), 'd990', 990); +insert into t values(now(), 'd991', 991); +insert into t values(now(), 'd992', 992); +insert into t values(now(), 'd993', 993); +insert into t values(now(), 'd994', 994); +insert into t values(now(), 'd995', 995); +insert into t values(now(), 'd996', 996); +insert into t values(now(), 'd997', 997); +insert into t values(now(), 'd998', 998); +insert into t values(now(), 'd999', 999); +insert into t values(now(), 'd1000', 1000);