You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@iotdb.apache.org by GitBox <gi...@apache.org> on 2021/09/02 12:09:40 UTC

[GitHub] [iotdb] qiaojialin commented on a change in pull request #3796: [IOTDB-1543] LastCache for Template and Vector

qiaojialin commented on a change in pull request #3796:
URL: https://github.com/apache/iotdb/pull/3796#discussion_r700826338



##########
File path: server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java
##########
@@ -1090,32 +1088,40 @@ private void tryToUpdateBatchInsertLastCache(InsertTabletPlan plan, Long latestF
       return;
     }
     IMeasurementMNode[] mNodes = plan.getMeasurementMNodes();
-    int columnIndex = 0;
-    for (int i = 0; i < mNodes.length; i++) {
-      // Don't update cached last value for vector type
-      if (mNodes[i] != null && plan.isAligned()) {
-        columnIndex += mNodes[i].getSchema().getValueMeasurementIdList().size();
-      } else {
+    if (plan.isAligned()) {
+      for (int i = 0; i < mNodes.length; i++) {
+        if (plan.getColumns()[i] == null) {
+          continue;
+        }
+        // Update cached last value with high priority
+        // vector lastCache update need subSensor path
+        IoTDB.metaManager.updateLastCache(
+            plan.getPrefixPath().concatNode(plan.getMeasurements()[i]),

Review comment:
       ```suggestion
               plan.getPrefixPath().concatNode(plan.getEntityPath()[i]),
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/lastCache/LastCacheManager.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.metadata.lastCache;
+
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.lastCache.entry.ILastCacheEntry;
+import org.apache.iotdb.db.metadata.mnode.IEntityMNode;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.executor.fill.LastPointReader;
+import org.apache.iotdb.tsfile.read.TimeValuePair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class LastCacheManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(LastCacheManager.class);
+
+  public static TimeValuePair getLastCache(PartialPath seriesPath, IMeasurementMNode node) {
+    if (node == null) {
+      return null;
+    }
+
+    checkIsEntityTemplateLastCache(node);
+
+    ILastCacheEntry lastCacheEntry = node.getLastCacheEntry();
+    if (seriesPath == null) {
+      return lastCacheEntry.getCachedLast();
+    } else {
+      String measurementId = seriesPath.getMeasurement();
+      if (measurementId.equals(node.getName()) || measurementId.equals(node.getAlias())) {
+        return lastCacheEntry.getCachedLast();
+      } else {
+        IMeasurementSchema schema = node.getSchema();
+        if (schema instanceof VectorMeasurementSchema) {
+          return lastCacheEntry.getCachedLast(
+              schema.getMeasurementIdColumnIndex(seriesPath.getMeasurement()));
+        }
+        return null;
+      }
+    }
+  }
+
+  public static void updateLastCache(
+      PartialPath seriesPath,
+      TimeValuePair timeValuePair,
+      boolean highPriorityUpdate,
+      Long latestFlushedTime,
+      IMeasurementMNode node) {
+    if (node == null) {
+      return;
+    }
+
+    checkIsEntityTemplateLastCache(node);
+
+    ILastCacheEntry lastCacheEntry = node.getLastCacheEntry();
+    if (seriesPath == null) {
+      lastCacheEntry.updateCachedLast(timeValuePair, highPriorityUpdate, latestFlushedTime);
+    } else {
+      String measurementId = seriesPath.getMeasurement();
+      if (measurementId.equals(node.getName()) || measurementId.equals(node.getAlias())) {
+        lastCacheEntry.updateCachedLast(timeValuePair, highPriorityUpdate, latestFlushedTime);
+      } else {
+        IMeasurementSchema schema = node.getSchema();
+        if (schema instanceof VectorMeasurementSchema) {
+          if (lastCacheEntry.isEmpty()) {
+            lastCacheEntry.init(schema.getMeasurementCount());
+          }
+          lastCacheEntry.updateCachedLast(
+              schema.getMeasurementIdColumnIndex(seriesPath.getMeasurement()),
+              timeValuePair,
+              highPriorityUpdate,
+              latestFlushedTime);
+        }
+      }
+    }
+  }
+
+  public static void resetLastCache(PartialPath seriesPath, IMeasurementMNode node) {
+    if (node == null) {
+      return;
+    }
+
+    checkIsEntityTemplateLastCache(node);
+
+    ILastCacheEntry lastCacheEntry = node.getLastCacheEntry();
+    if (seriesPath == null) {
+      lastCacheEntry.resetLastCache();
+    } else {
+      String measurementId = seriesPath.getMeasurement();
+      if (measurementId.equals(node.getName()) || measurementId.equals(node.getAlias())) {
+        lastCacheEntry.resetLastCache();
+      } else {
+        IMeasurementSchema schema = node.getSchema();
+        if (schema instanceof VectorMeasurementSchema) {
+          if (lastCacheEntry.isEmpty()) {
+            lastCacheEntry.init(schema.getMeasurementCount());
+          }
+          lastCacheEntry.resetLastCache(
+              schema.getMeasurementIdColumnIndex(seriesPath.getMeasurement()));
+        }
+      }
+    }
+  }
+
+  private static void checkIsEntityTemplateLastCache(IMeasurementMNode node) {

Review comment:
       This name should contains set or so

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/lastCache/entry/value/MonadLastCacheValue.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.metadata.lastCache.entry.value;
+
+import org.apache.iotdb.tsfile.read.TimeValuePair;
+import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
+
+public class MonadLastCacheValue implements ILastCacheValue {

Review comment:
       ```suggestion
   public class UnaryLastCacheValue implements ILastCacheValue {
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/lastCache/LastCacheManager.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.metadata.lastCache;
+
+import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
+import org.apache.iotdb.db.metadata.PartialPath;
+import org.apache.iotdb.db.metadata.lastCache.entry.ILastCacheEntry;
+import org.apache.iotdb.db.metadata.mnode.IEntityMNode;
+import org.apache.iotdb.db.metadata.mnode.IMNode;
+import org.apache.iotdb.db.metadata.mnode.IMeasurementMNode;
+import org.apache.iotdb.db.metadata.template.Template;
+import org.apache.iotdb.db.query.context.QueryContext;
+import org.apache.iotdb.db.query.control.QueryResourceManager;
+import org.apache.iotdb.db.query.executor.fill.LastPointReader;
+import org.apache.iotdb.tsfile.read.TimeValuePair;
+import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
+import org.apache.iotdb.tsfile.write.schema.VectorMeasurementSchema;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class LastCacheManager {

Review comment:
       add javadoc for each public method

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -1667,26 +1668,60 @@ public void updateLastCache(
       boolean highPriorityUpdate,
       Long latestFlushedTime,
       IMeasurementMNode node) {
-    if (node != null) {
-      node.updateCachedLast(timeValuePair, highPriorityUpdate, latestFlushedTime);
-    } else {
+    if (node == null) {
       try {
-        IMeasurementMNode node1 = (IMeasurementMNode) mtree.getNodeByPath(seriesPath);
-        node1.updateCachedLast(timeValuePair, highPriorityUpdate, latestFlushedTime);
+        node = (IMeasurementMNode) mtree.getNodeByPath(seriesPath);
       } catch (MetadataException e) {
         logger.warn("failed to update last cache for the {}, err:{}", seriesPath, e.getMessage());
+        return;
       }
     }
+
+    LastCacheManager.updateLastCache(
+        seriesPath, timeValuePair, highPriorityUpdate, latestFlushedTime, node);
   }
 
-  public TimeValuePair getLastCache(PartialPath seriesPath) {
+  public TimeValuePair getLastCache(PartialPath seriesPath, IMeasurementMNode node) {

Review comment:
       add javadoc

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/lastCache/entry/LastCacheEntry.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.metadata.lastCache.entry;
+
+import org.apache.iotdb.db.metadata.lastCache.entry.value.ILastCacheValue;
+import org.apache.iotdb.db.metadata.lastCache.entry.value.MonadLastCacheValue;
+import org.apache.iotdb.db.metadata.lastCache.entry.value.VectorLastCacheValue;
+import org.apache.iotdb.tsfile.read.TimeValuePair;
+
+public class LastCacheEntry implements ILastCacheEntry {
+
+  ILastCacheValue lastCacheValue;
+
+  @Override
+  public void init(int size) {
+    if (size > 1) {
+      lastCacheValue = new VectorLastCacheValue(size);
+    }
+  }
+
+  @Override
+  public TimeValuePair getCachedLast() {
+    return lastCacheValue == null ? null : lastCacheValue.getTimeValuePair();
+  }
+
+  @Override
+  public TimeValuePair getCachedLast(int index) {
+    return lastCacheValue == null ? null : lastCacheValue.getTimeValuePair(index);
+  }
+
+  @Override
+  public synchronized void updateCachedLast(
+      TimeValuePair timeValuePair, boolean highPriorityUpdate, Long latestFlushedTime) {
+    if (timeValuePair == null || timeValuePair.getValue() == null) {
+      return;
+    }
+
+    if (lastCacheValue == null) {
+      // If no cached last, (1) a last query (2) an unseq insertion or (3) a seq insertion will

Review comment:
       why update when comes an unseq insertion




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@iotdb.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org