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/08/26 02:22:26 UTC

[GitHub] [iotdb] KeeProMise opened a new pull request, #7137: idTable supports auto-increment id, in the stand-alone environment

KeeProMise opened a new pull request, #7137:
URL: https://github.com/apache/iotdb/pull/7137

   ## Description
   provide a new deviceID, use a long variable composed of schemaRegion+int_32 instead of devicePath, which can easily build an index for the device and reduce the storage space occupied by storing long devicePath strings
   
   ## Different
   -  **DeviceIDFactory** : add the getDeviceIDWithAutoCreateFunction method to separate the writing and reading of deviceID, because writing requires a new deviceID object to be generated when a device is added for the first time, but reading does not need it. Continuing to use the same method cannot distinguish between reading and writing.
   -  **IStatefulDeviceID** : added IStatefulDeviceID interface: state recovery is required during restart
   -  **StandAloneAutoIncDeviceID** :  using auto-incrementing id as device id,A complete auto-increment id consists of schemaRegionID and autoIncrementID, where the upper 32 bits are schemaRegionID and the lower 32 bits are
   autoIncrementID
   
   Detailed documentation:https://apache-iotdb.feishu.cn/docx/doxcn0K2B6LI1bGr9QpvkYfFUCe


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


[GitHub] [iotdb] MarcosZyk closed pull request #7137: idTable supports auto-increment id, in the stand-alone environment

Posted by GitBox <gi...@apache.org>.
MarcosZyk closed pull request #7137: idTable supports auto-increment id, in the stand-alone environment
URL: https://github.com/apache/iotdb/pull/7137


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


[GitHub] [iotdb] MarcosZyk commented on a diff in pull request #7137: idTable supports auto-increment id, in the stand-alone environment

Posted by GitBox <gi...@apache.org>.
MarcosZyk commented on code in PR #7137:
URL: https://github.com/apache/iotdb/pull/7137#discussion_r956667331


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java:
##########
@@ -365,14 +382,44 @@ public List<DeviceEntry> getAllDeviceEntry() {
     return res;
   }
 
+  /**
+   * put schema entry to id table, currently used in recover

Review Comment:
   Keep this comment consistent with that in the IDTable inserface.



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java:
##########
@@ -485,6 +523,25 @@ private DeviceEntry getDeviceEntryWithAlignedCheck(String deviceName, boolean is
     return deviceEntry;
   }
 
+  /**
+   * get device entry
+   *
+   * @param deviceID
+   * @param isAligned
+   * @return
+   */
+  private DeviceEntry getDeviceEntryWithAutoCreate(IDeviceID deviceID, boolean isAligned) {
+    int slot = calculateSlot(deviceID);
+
+    DeviceEntry deviceEntry = idTables[slot].get(deviceID);
+    // new device
+    if (deviceEntry == null) {
+      deviceEntry = new DeviceEntry(deviceID);
+      deviceEntry.setAligned(isAligned);
+      idTables[slot].put(deviceID, deviceEntry);
+    }

Review Comment:
   There may exists some concurrent problem when concurrent creating deviceEntry.



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/AppendOnlyDiskSchemaManager.java:
##########
@@ -158,7 +157,12 @@ public void recover(IDTable idTable) {
                   TSEncoding.deserialize(cur.encoding),
                   CompressionType.deserialize(cur.compressor),
                   loc);
-          idTable.putSchemaEntry(cur.deviceID, cur.measurementName, schemaEntry, cur.isAligned);
+          idTable.putSchemaEntry(
+              cur.deviceID,
+              cur.seriesKey.substring(0, cur.seriesKey.length() - cur.measurementName.length() - 1),

Review Comment:
   Encapsulate this computation as a function like ```getDevicePathFromSeriesKey```.



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/deviceID/DeviceIDFactory.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.idtable.deviceID;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+
+import java.util.function.Function;
+
+/** factory to build device id according to configured algorithm */
+public class DeviceIDFactory {
+
+  /** used to obtain a IDeviceID instance in the query operation */
+  Function<String, IDeviceID> getDeviceIDFunction;
+
+  /** used to obtain a IDeviceID instance in the insert operation */
+  Function<String, IDeviceID> getDeviceIDWithAutoCreateFunction;
+
+  /** used to obtain a IDeviceID instance in the system restart */
+  Function<String[], IDeviceID> getDeviceIDWithRecoverFunction;

Review Comment:
   Consider encapsulate this three function into one strategy class. Each strategy takes care of there own init, reset and clear etc. behaviors.



##########
server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableRecoverTest.java:
##########
@@ -63,32 +63,32 @@ public void before() {
     isEnableIDTableLogFile = IoTDBDescriptor.getInstance().getConfig().isEnableIDTableLogFile();
 
     IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(true);
-    IoTDBDescriptor.getInstance().getConfig().setDeviceIDTransformationMethod("SHA256");
+    IoTDBDescriptor.getInstance().getConfig().setDeviceIDTransformationMethod("AutoIncrement");

Review Comment:
   Why change this? If you want to test the ```AutoIncrement``` mode, it's better to add new tests. If you want to reuse the test case of ```SHA256```, maybe test class extension will work.



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/deviceID/DeviceIDFactory.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.idtable.deviceID;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+
+import java.util.function.Function;
+
+/** factory to build device id according to configured algorithm */
+public class DeviceIDFactory {
+
+  /** used to obtain a IDeviceID instance in the query operation */
+  Function<String, IDeviceID> getDeviceIDFunction;
+
+  /** used to obtain a IDeviceID instance in the insert operation */
+  Function<String, IDeviceID> getDeviceIDWithAutoCreateFunction;
+
+  /** used to obtain a IDeviceID instance in the system restart */
+  Function<String[], IDeviceID> getDeviceIDWithRecoverFunction;
+
+  // region DeviceIDFactory Singleton
+  private static class DeviceIDFactoryHolder {
+
+    private DeviceIDFactoryHolder() {
+      // allowed to do nothing
+    }
+
+    private static final DeviceIDFactory INSTANCE = new DeviceIDFactory();
+  }
+
+  /**
+   * get instance
+   *
+   * @return instance of the factory
+   */
+  public static DeviceIDFactory getInstance() {
+    return DeviceIDFactoryHolder.INSTANCE;
+  }
+
+  private DeviceIDFactory() {
+    if (IoTDBDescriptor.getInstance().getConfig().isEnableIDTable()) {
+      if (IoTDBDescriptor.getInstance()
+          .getConfig()
+          .getDeviceIDTransformationMethod()
+          .equals("SHA256")) {

Review Comment:
   Make this transformationMethod an Enum.



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/deviceID/StandAloneAutoIncDeviceID.java:
##########
@@ -0,0 +1,310 @@
+/*
+ * 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.idtable.deviceID;
+
+import org.apache.iotdb.commons.consensus.SchemaRegionId;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.db.localconfignode.LocalConfigNode;
+import org.apache.iotdb.db.metadata.idtable.IDTable;
+import org.apache.iotdb.db.metadata.idtable.IDTableManager;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Using auto-incrementing id as device id,A complete auto-increment id consists of schemaRegionID
+ * and autoIncrementID, where the upper 32 bits are schemaRegionID and the lower 32 bits are
+ * autoIncrementID
+ */
+public class StandAloneAutoIncDeviceID extends SHA256DeviceID {

Review Comment:
   Rename to ```AutoIncDeviceID```. It will be applied in cluster too.



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/deviceID/StandAloneAutoIncDeviceID.java:
##########
@@ -0,0 +1,310 @@
+/*
+ * 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.idtable.deviceID;
+
+import org.apache.iotdb.commons.consensus.SchemaRegionId;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.db.localconfignode.LocalConfigNode;
+import org.apache.iotdb.db.metadata.idtable.IDTable;
+import org.apache.iotdb.db.metadata.idtable.IDTableManager;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Using auto-incrementing id as device id,A complete auto-increment id consists of schemaRegionID
+ * and autoIncrementID, where the upper 32 bits are schemaRegionID and the lower 32 bits are
+ * autoIncrementID
+ */
+public class StandAloneAutoIncDeviceID extends SHA256DeviceID {
+
+  /** logger */
+  private static Logger logger = LoggerFactory.getLogger(IDTable.class);
+
+  // stand-alone auto-increment id uses LocalConfigNode to obtain schemaRegionId
+  private static LocalConfigNode configManager;
+
+  // using map to maintain the mapping from schemaRegionId to list<deviceID>, each list<deviceID>
+  // maintains the auto-increment id of the schemaRegion
+  private static Map<Integer, List<IDeviceID>> deviceIDsMap;
+
+  // if the device represented by devicePath is not written to the metadata module, use this
+  // constant instead of devicePath to generate a sha266 value of StandAloneAutoIncDeviceID instance
+  private static final String INVALID_DEVICE_PATH = "invalid.device.path";
+
+  // if the schemaRegionId==-1 of a StandAloneAutoIncDeviceID instance, it means that the device
+  // corresponding to the StandAloneAutoIncDeviceID instance does not exist
+  // return the deviceIdOfNonExistentDevice object for all devicePaths of unmanaged devices in the
+  // metadata module, which can avoid unnecessary object creation while ensuring correctness
+  private static StandAloneAutoIncDeviceID deviceIdOfNonExistentDevice;

Review Comment:
   All the static attributes and methods in this class should be implemented as an independent ```AutoIncDeviceIdManager```. This class should be a simple entity class.



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/deviceID/StandAloneAutoIncDeviceID.java:
##########
@@ -0,0 +1,310 @@
+/*
+ * 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.idtable.deviceID;
+
+import org.apache.iotdb.commons.consensus.SchemaRegionId;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.db.localconfignode.LocalConfigNode;
+import org.apache.iotdb.db.metadata.idtable.IDTable;
+import org.apache.iotdb.db.metadata.idtable.IDTableManager;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Using auto-incrementing id as device id,A complete auto-increment id consists of schemaRegionID
+ * and autoIncrementID, where the upper 32 bits are schemaRegionID and the lower 32 bits are
+ * autoIncrementID
+ */
+public class StandAloneAutoIncDeviceID extends SHA256DeviceID {
+
+  /** logger */
+  private static Logger logger = LoggerFactory.getLogger(IDTable.class);
+
+  // stand-alone auto-increment id uses LocalConfigNode to obtain schemaRegionId
+  private static LocalConfigNode configManager;
+
+  // using map to maintain the mapping from schemaRegionId to list<deviceID>, each list<deviceID>
+  // maintains the auto-increment id of the schemaRegion
+  private static Map<Integer, List<IDeviceID>> deviceIDsMap;
+
+  // if the device represented by devicePath is not written to the metadata module, use this
+  // constant instead of devicePath to generate a sha266 value of StandAloneAutoIncDeviceID instance
+  private static final String INVALID_DEVICE_PATH = "invalid.device.path";
+
+  // if the schemaRegionId==-1 of a StandAloneAutoIncDeviceID instance, it means that the device
+  // corresponding to the StandAloneAutoIncDeviceID instance does not exist
+  // return the deviceIdOfNonExistentDevice object for all devicePaths of unmanaged devices in the
+  // metadata module, which can avoid unnecessary object creation while ensuring correctness
+  private static StandAloneAutoIncDeviceID deviceIdOfNonExistentDevice;
+
+  // starting with 0,the maximum value is Integer.MAX_VALUE
+  int schemaRegionId;
+
+  // starting with 0,the maximum value is Integer.MAX_VALUE
+  int autoIncrementID;
+
+  static {
+    deviceIDsMap = new ConcurrentHashMap<>();
+    configManager = LocalConfigNode.getInstance();
+    setDeviceIdOfNonExistentDevice();
+  }
+
+  /**
+   * use the sha256 value of INVALID_DEVICE_PATH and schemaRegionId=-1, autoIncrementID=1 to
+   * generate a deviceID of the devicePath of the device that is not managed by the metadata
+   * management module, which can ensure that the deviceID is used, and no information can be found
+   * as expected.
+   */
+  private static void setDeviceIdOfNonExistentDevice() {
+    deviceIdOfNonExistentDevice = new StandAloneAutoIncDeviceID(INVALID_DEVICE_PATH);
+    deviceIdOfNonExistentDevice.schemaRegionId = -1;
+    deviceIdOfNonExistentDevice.autoIncrementID = 0;
+  }
+
+  public StandAloneAutoIncDeviceID() {}
+
+  public StandAloneAutoIncDeviceID(String devicePath) {
+    super(devicePath);
+  }
+
+  public StandAloneAutoIncDeviceID(String deviceID, String devicePath) {
+    super(devicePath);
+    long id = parseFromDeviceID(deviceID);
+    this.schemaRegionId = (int) (id >>> 32);
+    this.autoIncrementID = (int) id;
+  }
+
+  /**
+   * get a StandAloneAutoIncDeviceID instance, create it if it doesn't exist
+   *
+   * @param deviceID device path for insert/query, and device id for query
+   * @return a StandAloneAutoIncDeviceID instance
+   */
+  public static StandAloneAutoIncDeviceID getDeviceIDWithAutoCreate(String deviceID) {
+    if (deviceID.startsWith("`") && deviceID.endsWith("`")) {
+      return fromAutoIncDeviceID(deviceID);
+    } else {
+      return buildDeviceID(deviceID);
+    }
+  }
+
+  /**
+   * get a StandAloneAutoIncDeviceID instance, only for query
+   *
+   * @param deviceID device path or device id for query
+   * @return if the device exists, return a StandAloneAutoIncDeviceID instance, if it does not
+   *     exist,return a StandAloneAutoIncDeviceID instance,the object is guaranteed to be different
+   *     from the deviceID object of any device managed by the system (equals==false).
+   */
+  public static StandAloneAutoIncDeviceID getDeviceID(String deviceID) {
+    if (deviceID.startsWith("`") && deviceID.endsWith("`")) {
+      return fromAutoIncDeviceID(deviceID);
+    } else {
+      return fromDevicePath(deviceID);
+    }
+  }
+
+  /**
+   * get a StandAloneAutoIncDeviceID instance, only for recover
+   *
+   * @param device union by deviceID and devicePath
+   * @return a StandAloneAutoIncDeviceID instance
+   */
+  public static StandAloneAutoIncDeviceID getDeviceIDWithRecover(String... device) {
+    String deviceID = device[0];
+    String devicePath = device[1];
+    StandAloneAutoIncDeviceID id = new StandAloneAutoIncDeviceID(deviceID, devicePath);
+    List<IDeviceID> deviceIDs =
+        deviceIDsMap.computeIfAbsent(id.schemaRegionId, integer -> new ArrayList<>());
+    // if there is out-of-order data, write the deviceID to the correct index of the array
+    synchronized (deviceIDs) {
+      if (id.autoIncrementID < deviceIDs.size() && deviceIDs.get(id.autoIncrementID) != null) {
+        return (StandAloneAutoIncDeviceID) deviceIDs.get(id.autoIncrementID);
+      } else {
+        for (int i = deviceIDs.size(); i < id.autoIncrementID; i++) {
+          deviceIDs.add(i, null);
+        }
+        deviceIDs.add(id.autoIncrementID, id);
+        return id;
+      }
+    }
+  }
+
+  /**
+   * get device id from a standAloneAutoIncDeviceID
+   *
+   * @param deviceID StandAloneAutoIncDeviceID deviceID, like: "`1`"
+   * @return a standAloneAutoIncDeviceID instance
+   */
+  private static StandAloneAutoIncDeviceID fromAutoIncDeviceID(String deviceID) {
+    long id = parseFromDeviceID(deviceID);
+    int schemaRegionId = (int) (id >>> 32);
+    int autoIncrementID = (int) id;
+    if (schemaRegionId == -1) {
+      return deviceIdOfNonExistentDevice;
+    }
+    List<IDeviceID> deviceIDs = deviceIDsMap.get(schemaRegionId);
+    synchronized (deviceIDs) {
+      return (StandAloneAutoIncDeviceID) deviceIDs.get(autoIncrementID);
+    }
+  }
+
+  /**
+   * get device id from a device path
+   *
+   * @param devicePath device path, like: "root.sg.x.d1"
+   * @return a standAloneAutoIncDeviceID instance
+   */
+  private static StandAloneAutoIncDeviceID fromDevicePath(String devicePath) {
+    try {
+      // use idTable to determine whether the device has been created
+      IDTable idTable = IDTableManager.getInstance().getIDTable(new PartialPath(devicePath));
+      StandAloneAutoIncDeviceID deviceID = new StandAloneAutoIncDeviceID(devicePath);
+      if (idTable.getDeviceEntry(deviceID) != null) {
+        deviceID = (StandAloneAutoIncDeviceID) idTable.getDeviceEntry(deviceID).getDeviceID();
+        return deviceID;
+      } else {
+        return deviceIdOfNonExistentDevice;
+      }
+    } catch (IllegalPathException e) {
+      logger.info(e.getMessage());
+      return deviceIdOfNonExistentDevice;
+    }
+  }
+
+  /**
+   * get device id from a device path, if the device represented by the path does not exist, a
+   * StandAloneAutoIncDeviceID instance is generated for the path
+   *
+   * @param devicePath device path, like: "root.sg.x.d1"
+   * @return a standAloneAutoIncDeviceID instance
+   */
+  private static StandAloneAutoIncDeviceID buildDeviceID(String devicePath) {
+    try {
+      PartialPath path = new PartialPath(devicePath);
+      // use idTable to determine whether the device has been created
+      IDTable idTable = IDTableManager.getInstance().getIDTable(path);
+      StandAloneAutoIncDeviceID deviceID = new StandAloneAutoIncDeviceID(devicePath);
+      // this device is added for the first time
+      if (idTable.getDeviceEntry(deviceID) == null) {
+        SchemaRegionId schemaRegionId = configManager.getBelongedSchemaRegionId(path);
+        deviceID.schemaRegionId = schemaRegionId.getId();

Review Comment:
   When there's need to create the deviceId, we should already know which schemaRegion it belongs rather than do computation again.



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


[GitHub] [iotdb] MarcosZyk commented on a diff in pull request #7137: idTable supports auto-increment id, in the stand-alone environment

Posted by GitBox <gi...@apache.org>.
MarcosZyk commented on code in PR #7137:
URL: https://github.com/apache/iotdb/pull/7137#discussion_r956668009


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/deviceID/DeviceIDFactory.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.idtable.deviceID;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+
+import java.util.function.Function;
+
+/** factory to build device id according to configured algorithm */
+public class DeviceIDFactory {
+
+  /** used to obtain a IDeviceID instance in the query operation */
+  Function<String, IDeviceID> getDeviceIDFunction;
+
+  /** used to obtain a IDeviceID instance in the insert operation */
+  Function<String, IDeviceID> getDeviceIDWithAutoCreateFunction;
+
+  /** used to obtain a IDeviceID instance in the system restart */
+  Function<String[], IDeviceID> getDeviceIDWithRecoverFunction;

Review Comment:
   This factory should only take the responsibility to parse DevicePath -> DeviceID, which should be treated as a stateless class.
   
   The ```getDeviceIDWithAutoCreateFunction``` and ```getDeviceIDWithRecoverFunction``` should be extracted and implemented into related IDTable.



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


[GitHub] [iotdb] KeeProMise commented on a diff in pull request #7137: idTable supports auto-increment id, in the stand-alone environment

Posted by GitBox <gi...@apache.org>.
KeeProMise commented on code in PR #7137:
URL: https://github.com/apache/iotdb/pull/7137#discussion_r958049409


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableAutoIncImpl.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.idtable;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.commons.path.PartialPath;
+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.localconfignode.LocalConfigNode;
+import org.apache.iotdb.db.metadata.idtable.deviceID.AutoIncDeviceID;
+import org.apache.iotdb.db.metadata.idtable.deviceID.IDeviceID;
+import org.apache.iotdb.db.metadata.idtable.deviceID.SHA256DeviceID;
+import org.apache.iotdb.db.metadata.idtable.entry.DeviceEntry;
+import org.apache.iotdb.db.metadata.idtable.entry.SchemaEntry;
+import org.apache.iotdb.db.metadata.idtable.entry.TimeseriesID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class IDTableAutoIncImpl extends IDTableHashmapImpl {
+
+  /** logger */
+  private static final Logger logger = LoggerFactory.getLogger(IDTableAutoIncImpl.class);
+
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  // if the schemaRegionId==-1 of a StandAloneAutoIncDeviceID instance, it means that the device
+  // corresponding to the StandAloneAutoIncDeviceID instance does not exist
+  // return the deviceIdOfNonExistentDevice object for all devicePaths of unmanaged devices in the
+  // metadata module, which can avoid unnecessary object creation while ensuring correctness
+  private static AutoIncDeviceID deviceIdOfNonExistentDevice;
+
+  // stand-alone auto-increment id uses LocalConfigNode to obtain schemaRegionId
+  private static LocalConfigNode configManager;
+
+  // using map to maintain the mapping from schemaRegionId to list<deviceID>, each list<deviceID>
+  // maintains the auto-increment id of the schemaRegion
+  private static Map<Integer, List<DeviceEntry>> deviceEntrysMap;
+
+  static {
+    deviceEntrysMap = new ConcurrentHashMap<>();
+    configManager = LocalConfigNode.getInstance();
+    deviceIdOfNonExistentDevice = new AutoIncDeviceID(-1, 0);
+  }
+
+  public static AutoIncDeviceID getDeviceID(String deviceID) {
+    if (deviceID.startsWith("`") && deviceID.endsWith("`")) {
+      return fromAutoIncDeviceID(deviceID);
+    } else {
+      return fromIdTable(deviceID);
+    }
+  }
+
+  public static AutoIncDeviceID fromAutoIncDeviceID(String deviceID) {
+    AutoIncDeviceID autoIncID = new AutoIncDeviceID(deviceID);
+    int schemaRegionId = autoIncID.getSchemaRegionId();
+    int autoIncrementID = autoIncID.getAutoIncrementID();
+    if (autoIncID.getSchemaRegionId() == -1) {
+      return deviceIdOfNonExistentDevice;
+    }
+    List<DeviceEntry> deviceEntries = deviceEntrysMap.get(schemaRegionId);
+    synchronized (deviceEntries) {
+      DeviceEntry deviceEntry = deviceEntries.get(autoIncrementID);
+      return (AutoIncDeviceID) deviceEntry.getDeviceID();
+    }
+  }
+
+  public static AutoIncDeviceID fromIdTable(String devicePath) {

Review Comment:
   renamed



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


[GitHub] [iotdb] MarcosZyk commented on a diff in pull request #7137: idTable supports auto-increment id, in the stand-alone environment

Posted by GitBox <gi...@apache.org>.
MarcosZyk commented on code in PR #7137:
URL: https://github.com/apache/iotdb/pull/7137#discussion_r956679487


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/deviceID/DeviceIDFactory.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.idtable.deviceID;
+
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.db.conf.IoTDBDescriptor;
+
+import java.util.function.Function;
+
+/** factory to build device id according to configured algorithm */
+public class DeviceIDFactory {
+
+  /** used to obtain a IDeviceID instance in the query operation */
+  Function<String, IDeviceID> getDeviceIDFunction;
+
+  /** used to obtain a IDeviceID instance in the insert operation */
+  Function<String, IDeviceID> getDeviceIDWithAutoCreateFunction;
+
+  /** used to obtain a IDeviceID instance in the system restart */
+  Function<String[], IDeviceID> getDeviceIDWithRecoverFunction;

Review Comment:
   Remove this ```String[]```.



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/deviceID/DeviceIDFactory.java:
##########
@@ -49,23 +50,26 @@ public static DeviceIDFactory getInstance() {
   }
 
   private DeviceIDFactory() {
-    if (IoTDBDescriptor.getInstance().getConfig().isEnableIDTable()
-        && IoTDBDescriptor.getInstance()
-            .getConfig()
-            .getDeviceIDTransformationMethod()
-            .equals("SHA256")) {
-      getDeviceIDFunction = SHA256DeviceID::new;
-    } else {
-      getDeviceIDFunction = PlainDeviceID::new;
+    if (IoTDBDescriptor.getInstance().getConfig().isEnableIDTable()) {
+      switch (DeviceIDMode.valueOf(
+          IoTDBDescriptor.getInstance().getConfig().getDeviceIDTransformationMethod())) {
+        case SHA256:
+          getDeviceIDFunction = SHA256DeviceID::new;
+          return;
+        case AutoIncrement:
+          getDeviceIDFunction = IDTableAutoIncImpl::getDeviceID;
+          return;

Review Comment:
   Add default branch.



##########
server/src/test/java/org/apache/iotdb/db/metadata/idtable/IDTableRecoverTest.java:
##########
@@ -63,32 +63,32 @@ public void before() {
     isEnableIDTableLogFile = IoTDBDescriptor.getInstance().getConfig().isEnableIDTableLogFile();
 
     IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(true);
-    IoTDBDescriptor.getInstance().getConfig().setDeviceIDTransformationMethod("SHA256");
+    IoTDBDescriptor.getInstance().getConfig().setDeviceIDTransformationMethod("AutoIncrement");
     IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(true);
     EnvironmentUtils.envSetUp();
   }
 
   @After
   public void clean() throws IOException, StorageEngineException {
+    EnvironmentUtils.cleanEnv();
     IoTDBDescriptor.getInstance().getConfig().setEnableIDTable(isEnableIDTable);
     IoTDBDescriptor.getInstance()
         .getConfig()
         .setDeviceIDTransformationMethod(originalDeviceIDTransformationMethod);
     IoTDBDescriptor.getInstance().getConfig().setEnableIDTableLogFile(isEnableIDTableLogFile);
-
-    EnvironmentUtils.cleanEnv();
   }

Review Comment:
   The ```EnvironmentUtils.cleanEnv()``` is better to executed as the last step, otherwise something caused by config change won't be cleaned clear.



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableAutoIncImpl.java:
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.idtable;
+
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.commons.path.PartialPath;
+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.localconfignode.LocalConfigNode;
+import org.apache.iotdb.db.metadata.idtable.deviceID.AutoIncDeviceID;
+import org.apache.iotdb.db.metadata.idtable.deviceID.IDeviceID;
+import org.apache.iotdb.db.metadata.idtable.deviceID.SHA256DeviceID;
+import org.apache.iotdb.db.metadata.idtable.entry.DeviceEntry;
+import org.apache.iotdb.db.metadata.idtable.entry.SchemaEntry;
+import org.apache.iotdb.db.metadata.idtable.entry.TimeseriesID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class IDTableAutoIncImpl extends IDTableHashmapImpl {
+
+  /** logger */
+  private static final Logger logger = LoggerFactory.getLogger(IDTableAutoIncImpl.class);
+
+  protected static IoTDBConfig config = IoTDBDescriptor.getInstance().getConfig();
+
+  // if the schemaRegionId==-1 of a StandAloneAutoIncDeviceID instance, it means that the device
+  // corresponding to the StandAloneAutoIncDeviceID instance does not exist
+  // return the deviceIdOfNonExistentDevice object for all devicePaths of unmanaged devices in the
+  // metadata module, which can avoid unnecessary object creation while ensuring correctness
+  private static AutoIncDeviceID deviceIdOfNonExistentDevice;
+
+  // stand-alone auto-increment id uses LocalConfigNode to obtain schemaRegionId
+  private static LocalConfigNode configManager;
+
+  // using map to maintain the mapping from schemaRegionId to list<deviceID>, each list<deviceID>
+  // maintains the auto-increment id of the schemaRegion
+  private static Map<Integer, List<DeviceEntry>> deviceEntrysMap;
+
+  static {
+    deviceEntrysMap = new ConcurrentHashMap<>();
+    configManager = LocalConfigNode.getInstance();
+    deviceIdOfNonExistentDevice = new AutoIncDeviceID(-1, 0);
+  }
+
+  public static AutoIncDeviceID getDeviceID(String deviceID) {
+    if (deviceID.startsWith("`") && deviceID.endsWith("`")) {
+      return fromAutoIncDeviceID(deviceID);
+    } else {
+      return fromIdTable(deviceID);
+    }
+  }
+
+  public static AutoIncDeviceID fromAutoIncDeviceID(String deviceID) {
+    AutoIncDeviceID autoIncID = new AutoIncDeviceID(deviceID);
+    int schemaRegionId = autoIncID.getSchemaRegionId();
+    int autoIncrementID = autoIncID.getAutoIncrementID();
+    if (autoIncID.getSchemaRegionId() == -1) {
+      return deviceIdOfNonExistentDevice;
+    }
+    List<DeviceEntry> deviceEntries = deviceEntrysMap.get(schemaRegionId);
+    synchronized (deviceEntries) {
+      DeviceEntry deviceEntry = deviceEntries.get(autoIncrementID);
+      return (AutoIncDeviceID) deviceEntry.getDeviceID();
+    }
+  }
+
+  public static AutoIncDeviceID fromIdTable(String devicePath) {

Review Comment:
   Rename this method to some name like ```getDeviceIdFromIdTable```. Add a verb.



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


[GitHub] [iotdb] KeeProMise commented on a diff in pull request #7137: idTable supports auto-increment id, in the stand-alone environment

Posted by GitBox <gi...@apache.org>.
KeeProMise commented on code in PR #7137:
URL: https://github.com/apache/iotdb/pull/7137#discussion_r958048469


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/deviceID/DeviceIDFactory.java:
##########
@@ -49,23 +50,26 @@ public static DeviceIDFactory getInstance() {
   }
 
   private DeviceIDFactory() {
-    if (IoTDBDescriptor.getInstance().getConfig().isEnableIDTable()
-        && IoTDBDescriptor.getInstance()
-            .getConfig()
-            .getDeviceIDTransformationMethod()
-            .equals("SHA256")) {
-      getDeviceIDFunction = SHA256DeviceID::new;
-    } else {
-      getDeviceIDFunction = PlainDeviceID::new;
+    if (IoTDBDescriptor.getInstance().getConfig().isEnableIDTable()) {
+      switch (DeviceIDMode.valueOf(
+          IoTDBDescriptor.getInstance().getConfig().getDeviceIDTransformationMethod())) {
+        case SHA256:
+          getDeviceIDFunction = SHA256DeviceID::new;
+          return;
+        case AutoIncrement:
+          getDeviceIDFunction = IDTableAutoIncImpl::getDeviceID;
+          return;

Review Comment:
   added



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


[GitHub] [iotdb] MarcosZyk commented on a diff in pull request #7137: idTable supports auto-increment id, in the stand-alone environment

Posted by GitBox <gi...@apache.org>.
MarcosZyk commented on code in PR #7137:
URL: https://github.com/apache/iotdb/pull/7137#discussion_r956670121


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/deviceID/StandAloneAutoIncDeviceID.java:
##########
@@ -0,0 +1,310 @@
+/*
+ * 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.idtable.deviceID;
+
+import org.apache.iotdb.commons.consensus.SchemaRegionId;
+import org.apache.iotdb.commons.exception.IllegalPathException;
+import org.apache.iotdb.commons.exception.MetadataException;
+import org.apache.iotdb.commons.path.PartialPath;
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.db.localconfignode.LocalConfigNode;
+import org.apache.iotdb.db.metadata.idtable.IDTable;
+import org.apache.iotdb.db.metadata.idtable.IDTableManager;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Using auto-incrementing id as device id,A complete auto-increment id consists of schemaRegionID
+ * and autoIncrementID, where the upper 32 bits are schemaRegionID and the lower 32 bits are
+ * autoIncrementID
+ */
+public class StandAloneAutoIncDeviceID extends SHA256DeviceID {
+
+  /** logger */
+  private static Logger logger = LoggerFactory.getLogger(IDTable.class);
+
+  // stand-alone auto-increment id uses LocalConfigNode to obtain schemaRegionId
+  private static LocalConfigNode configManager;
+
+  // using map to maintain the mapping from schemaRegionId to list<deviceID>, each list<deviceID>
+  // maintains the auto-increment id of the schemaRegion
+  private static Map<Integer, List<IDeviceID>> deviceIDsMap;
+
+  // if the device represented by devicePath is not written to the metadata module, use this
+  // constant instead of devicePath to generate a sha266 value of StandAloneAutoIncDeviceID instance
+  private static final String INVALID_DEVICE_PATH = "invalid.device.path";
+
+  // if the schemaRegionId==-1 of a StandAloneAutoIncDeviceID instance, it means that the device
+  // corresponding to the StandAloneAutoIncDeviceID instance does not exist
+  // return the deviceIdOfNonExistentDevice object for all devicePaths of unmanaged devices in the
+  // metadata module, which can avoid unnecessary object creation while ensuring correctness
+  private static StandAloneAutoIncDeviceID deviceIdOfNonExistentDevice;

Review Comment:
   All the static attributes and methods in this class should be implemented as an independent ```IDTableAutoIncImpl```. This class should be a simple entity class.



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