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 2022/03/14 08:07:24 UTC

[GitHub] [iotdb] qiaojialin commented on a change in pull request #5205: [IOTDB-2675] Manage Metadata by Storage Group

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



##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/StorageGroupManager.java
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.mtree.MTreeAboveSG;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.db.utils.TestOnly;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_ROOT;
+import static org.apache.iotdb.db.conf.IoTDBConstant.PATH_SEPARATOR;
+
+// This class implements all the interfaces for storage group management. The MTreeAboveSg is used
+// to manage all the storage groups and MNodes above storage group.
+public class StorageGroupManager implements IStorageGroupManager {

Review comment:
       This is duplicated with StorageGroupManager in StorageEngine, rename to StorageGroupSchemaManager

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/storagegroup/IStorageGroupManager.java
##########
@@ -0,0 +1,231 @@
+/*
+ * 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.storagegroup;
+
+import org.apache.iotdb.db.exception.metadata.MetadataException;
+import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException;
+import org.apache.iotdb.db.metadata.MManager;
+import org.apache.iotdb.db.metadata.mnode.IStorageGroupMNode;
+import org.apache.iotdb.db.metadata.path.PartialPath;
+import org.apache.iotdb.tsfile.utils.Pair;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+// This class declares all the interfaces for storage group management.
+public interface IStorageGroupManager {

Review comment:
       ```suggestion
   public interface IStorageGroupSchemaManager {
   ```

##########
File path: server/src/main/java/org/apache/iotdb/db/metadata/MManager.java
##########
@@ -170,35 +143,12 @@
 
   public static final String TIME_SERIES_TREE_HEADER = "===  Timeseries Tree  ===\n\n";
 
-  /** A thread will check whether the MTree is modified lately each such interval. Unit: second */
-  private static final long MTREE_SNAPSHOT_THREAD_CHECK_TIME = 600L;
-
   protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
-  /** threshold total size of MTree */
-  private static final long MTREE_SIZE_THRESHOLD = config.getAllocateMemoryForSchema();
-
-  private static final int ESTIMATED_SERIES_SIZE = config.getEstimatedSeriesSize();
 
-  private boolean isRecovering;
   private boolean initialized;
-  private boolean allowToCreateNewSeries = true;
-
-  private AtomicLong totalSeriesNumber = new AtomicLong();
-
-  private final int mtreeSnapshotInterval;
-  private final long mtreeSnapshotThresholdTime;
-  private ScheduledExecutorService timedCreateMTreeSnapshotThread;
-  private ScheduledExecutorService timedForceMLogThread;
 
-  // the log file seriesPath
-  private String logFilePath;
-  private File logFile;
-  private MLogWriter logWriter;
-
-  private MTree mtree;
-  // device -> DeviceMNode
-  private LoadingCache<PartialPath, IMNode> mNodeCache;
-  private TagManager tagManager = TagManager.getInstance();
+  private TimeseriesStatistics timeseriesStatistics = TimeseriesStatistics.getInstance();
+  private IStorageGroupManager storageGroupManager = StorageGroupManager.getInstance();

Review comment:
       StorageGroupManager will be removed later, you need to maintain each MTree of Storage Group inside MManager.




-- 
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