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/16 18:08:06 UTC

[GitHub] [iotdb] KeeProMise opened a new pull request, #7024: Idtable supports getting time series from disk and deleting time series

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

   ## Background
   Idtable currently does not support getting detailed time series information from disk, and does not support deleting time series
   ## Different
   Mainly added 2 interface methods:
   1. _**Pair<Integer, Set<String>> deleteTimeseries(List<PartialPath> fullPaths)**_: used to delete time series
   2. **_List<DiskSchemaEntry> getDiskSchemaEntries(List<SchemaEntry> schemaEntries)_**: Used to read time series information from disk files
   Detailed design:https://apache-iotdb.feishu.cn/docx/doxcn65XjYe64t8QyGiyLN2Ig0c


-- 
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 #7024: Idtable supports getting time series from disk and deleting time series

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


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/DiskSchemaEntry.java:
##########
@@ -73,37 +77,61 @@ public DiskSchemaEntry(
   }
 
   public int serialize(OutputStream outputStream) throws IOException {
-    int byteLen = 0;
-    byteLen += ReadWriteIOUtils.write(deviceID, outputStream);
-    byteLen += ReadWriteIOUtils.write(seriesKey, outputStream);
-    byteLen += ReadWriteIOUtils.write(measurementName, outputStream);
-    byteLen += ReadWriteIOUtils.write(type, outputStream);
-    byteLen += ReadWriteIOUtils.write(encoding, outputStream);
-    byteLen += ReadWriteIOUtils.write(compressor, outputStream);
-    byteLen += ReadWriteIOUtils.write(isAligned, outputStream);
-
-    byteLen += ReadWriteIOUtils.write(byteLen, outputStream);
-    entrySize = byteLen;
-
-    return byteLen;
+    this.entrySize = ReadWriteIOUtils.write(toByteBuffer(), outputStream);
+    return entrySize;

Review Comment:
   fixed



-- 
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 #7024: Idtable supports getting time series from disk and deleting time series

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


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/AppendOnlyDiskSchemaManager.java:
##########
@@ -190,6 +198,65 @@ public Collection<DiskSchemaEntry> getAllSchemaEntry() throws IOException {
     return res;
   }
 
+  /**
+   * get DiskSchemaEntries from disk file
+   *
+   * @param offsets the offset of each record on the disk file
+   * @return DiskSchemaEntries
+   */
+  @Override
+  public List<DiskSchemaEntry> getDiskSchemaEntriesByOffset(List<Long> offsets) {
+    List<DiskSchemaEntry> diskSchemaEntries = new ArrayList<>(offsets.size());
+    Collections.sort(offsets);
+    try (RandomAccessFile randomAccessFile = new RandomAccessFile(dataFile, "r")) {
+      for (long offset : offsets) {
+        diskSchemaEntries.add(getDiskSchemaEntryByOffset(randomAccessFile, offset));
+      }
+    } catch (FileNotFoundException e) {
+      logger.info(e.getMessage());

Review Comment:
   fixed



-- 
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 #7024: Idtable supports getting time series from disk and deleting time series

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


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/AppendOnlyDiskSchemaManager.java:
##########
@@ -54,12 +58,15 @@ public class AppendOnlyDiskSchemaManager implements IDiskSchemaManager {
 
   long loc;
 
+  private RandomAccessFile randomAccessFile;
+
   private static final Logger logger = LoggerFactory.getLogger(AppendOnlyDiskSchemaManager.class);
 
   public AppendOnlyDiskSchemaManager(File dir) {
     try {
       initFile(dir);
       outputStream = new FileOutputStream(dataFile, true);
+      randomAccessFile = new RandomAccessFile(dataFile, "rw");

Review Comment:
   Remember close this file when the manager is closed or cleared. Otherwise, some IT may be affected, since the file handler protection on windows is strict.



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/DiskSchemaEntry.java:
##########
@@ -73,37 +77,61 @@ public DiskSchemaEntry(
   }
 
   public int serialize(OutputStream outputStream) throws IOException {
-    int byteLen = 0;
-    byteLen += ReadWriteIOUtils.write(deviceID, outputStream);
-    byteLen += ReadWriteIOUtils.write(seriesKey, outputStream);
-    byteLen += ReadWriteIOUtils.write(measurementName, outputStream);
-    byteLen += ReadWriteIOUtils.write(type, outputStream);
-    byteLen += ReadWriteIOUtils.write(encoding, outputStream);
-    byteLen += ReadWriteIOUtils.write(compressor, outputStream);
-    byteLen += ReadWriteIOUtils.write(isAligned, outputStream);
-
-    byteLen += ReadWriteIOUtils.write(byteLen, outputStream);
-    entrySize = byteLen;
-
-    return byteLen;
+    this.entrySize = ReadWriteIOUtils.write(toByteBuffer(), outputStream);
+    return entrySize;

Review Comment:
   The byte data can be written directly into the ouputstream, which avoids the redundant data copy. 



-- 
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 #7024: Idtable supports getting time series from disk and deleting time series

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


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/DiskSchemaEntry.java:
##########
@@ -73,37 +77,61 @@ public DiskSchemaEntry(
   }
 
   public int serialize(OutputStream outputStream) throws IOException {
-    int byteLen = 0;
-    byteLen += ReadWriteIOUtils.write(deviceID, outputStream);
-    byteLen += ReadWriteIOUtils.write(seriesKey, outputStream);
-    byteLen += ReadWriteIOUtils.write(measurementName, outputStream);
-    byteLen += ReadWriteIOUtils.write(type, outputStream);
-    byteLen += ReadWriteIOUtils.write(encoding, outputStream);
-    byteLen += ReadWriteIOUtils.write(compressor, outputStream);
-    byteLen += ReadWriteIOUtils.write(isAligned, outputStream);
-
-    byteLen += ReadWriteIOUtils.write(byteLen, outputStream);
-    entrySize = byteLen;
-
-    return byteLen;
+    this.entrySize = ReadWriteIOUtils.write(toByteBuffer(), outputStream);
+    return entrySize;

Review Comment:
   I mean the process, that use a byteBuffer store the whole serialized binary data and then write it to outputStream, has some unnecessary operation. The data can be serialized to outputStream directly.



-- 
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 #7024: Idtable supports getting time series from disk and deleting time series

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


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java:
##########
@@ -129,6 +134,42 @@ public synchronized void createTimeseries(CreateTimeSeriesPlan plan) throws Meta
     deviceEntry.putSchemaEntry(plan.getPath().getMeasurement(), schemaEntry);
   }
 
+  /**
+   * Delete all timeseries matching the given paths
+   *
+   * @param fullPaths paths to be deleted
+   * @return deletion failed Timeseries
+   * @throws MetadataException
+   */
+  @Override
+  public Pair<Integer, Set<String>> deleteTimeseries(List<PartialPath> fullPaths)
+      throws MetadataException {
+    int deletedNum = 0;
+    Set<String> failedNames = new HashSet<>();
+    List<Pair<PartialPath, Long>> deletedPairs = new ArrayList<>(fullPaths.size());
+    for (PartialPath fullPath : fullPaths) {
+      try {
+        Map<String, SchemaEntry> map = getDeviceEntry(fullPath.getDevice()).getMeasurementMap();
+        Long offset = map.get(fullPath.getMeasurement()).getDiskPointer();
+        deletedPairs.add(new Pair<>(fullPath, offset));
+      } catch (NullPointerException e) {
+        failedNames.add(fullPath.getFullPath());
+      }

Review Comment:
   fixed



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/DiskSchemaEntry.java:
##########
@@ -100,10 +103,31 @@ public static DiskSchemaEntry deserialize(InputStream inputStream) throws IOExce
     // read byte len
     res.entrySize = ReadWriteIOUtils.readInt(inputStream);
     res.entrySize += Integer.BYTES;
-
+    if (isTombstone(res.deviceID)) res.deviceID = TOMBSTONE;
     return res;
   }
 
+  public static void writeTombstone(RandomAccessFile randomAccessFile, long offset)
+      throws IOException {
+    randomAccessFile.seek(offset);
+    int strLength = randomAccessFile.readInt();
+    byte[] bytes = new byte[strLength];
+    randomAccessFile.write(bytes, 0, strLength);
+  }
+
+  public static String readString(RandomAccessFile randomAccessFile) throws IOException {
+    int strLength = randomAccessFile.readInt();
+    byte[] bytes = new byte[strLength];
+    randomAccessFile.read(bytes, 0, strLength);
+    return new String(bytes, 0, strLength);
+  }

Review Comment:
   fixed



-- 
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 merged pull request #7024: Idtable supports getting time series from disk and deleting time series

Posted by GitBox <gi...@apache.org>.
MarcosZyk merged PR #7024:
URL: https://github.com/apache/iotdb/pull/7024


-- 
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 #7024: Idtable supports getting time series from disk and deleting time series

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


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/DiskSchemaEntry.java:
##########
@@ -73,37 +77,61 @@ public DiskSchemaEntry(
   }
 
   public int serialize(OutputStream outputStream) throws IOException {
-    int byteLen = 0;
-    byteLen += ReadWriteIOUtils.write(deviceID, outputStream);
-    byteLen += ReadWriteIOUtils.write(seriesKey, outputStream);
-    byteLen += ReadWriteIOUtils.write(measurementName, outputStream);
-    byteLen += ReadWriteIOUtils.write(type, outputStream);
-    byteLen += ReadWriteIOUtils.write(encoding, outputStream);
-    byteLen += ReadWriteIOUtils.write(compressor, outputStream);
-    byteLen += ReadWriteIOUtils.write(isAligned, outputStream);
-
-    byteLen += ReadWriteIOUtils.write(byteLen, outputStream);
-    entrySize = byteLen;
-
-    return byteLen;
+    this.entrySize = ReadWriteIOUtils.write(toByteBuffer(), outputStream);
+    return entrySize;

Review Comment:
   ```java
   public ByteBuffer toByteBuffer() {
       byte[] deviceIDBytes = deviceID.getBytes();
       byte[] seriesKeyBytes = seriesKey.getBytes();
       byte[] measurementNameBytes = measurementName.getBytes();
       int length =
           deviceIDBytes.length
               + seriesKeyBytes.length
               + measurementNameBytes.length
               + Integer.BYTES * 3
               + Byte.BYTES * 3
               + ReadWriteIOUtils.BOOLEAN_LEN;
       ByteBuffer byteBuffer = ByteBuffer.allocate(length);
       byteBuffer.put(BytesUtils.intToBytes(deviceIDBytes.length));
       byteBuffer.put(deviceIDBytes);
       byteBuffer.put(BytesUtils.intToBytes(seriesKeyBytes.length));
       byteBuffer.put(seriesKeyBytes);
       byteBuffer.put(BytesUtils.intToBytes(measurementNameBytes.length));
       byteBuffer.put(measurementNameBytes);
       byteBuffer.put(type);
       byteBuffer.put(encoding);
       byteBuffer.put(compressor);
       ReadWriteIOUtils.write(isAligned, byteBuffer);
       return byteBuffer;
     }
   ```
   Do you mean that the toByteBuffer() method has redundant data?
   



-- 
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 #7024: Idtable supports getting time series from disk and deleting time series

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


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/AppendOnlyDiskSchemaManager.java:
##########
@@ -54,12 +58,15 @@ public class AppendOnlyDiskSchemaManager implements IDiskSchemaManager {
 
   long loc;
 
+  private RandomAccessFile randomAccessFile;
+
   private static final Logger logger = LoggerFactory.getLogger(AppendOnlyDiskSchemaManager.class);
 
   public AppendOnlyDiskSchemaManager(File dir) {
     try {
       initFile(dir);
       outputStream = new FileOutputStream(dataFile, true);
+      randomAccessFile = new RandomAccessFile(dataFile, "rw");

Review Comment:
   I close randomAccessFile in the close method of this 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


[GitHub] [iotdb] MarcosZyk commented on a diff in pull request #7024: Idtable supports getting time series from disk and deleting time series

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


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTable.java:
##########
@@ -129,7 +141,7 @@ void updateLastCache(
    * @param deviceName device name of the time series
    * @return device entry of the timeseries
    */
-  public DeviceEntry getDeviceEntry(String deviceName);
+  DeviceEntry getDeviceEntry(String deviceName);

Review Comment:
   Why make this as ```default```?



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/AppendOnlyDiskSchemaManager.java:
##########
@@ -190,6 +198,65 @@ public Collection<DiskSchemaEntry> getAllSchemaEntry() throws IOException {
     return res;
   }
 
+  /**
+   * get DiskSchemaEntries from disk file
+   *
+   * @param offsets the offset of each record on the disk file
+   * @return DiskSchemaEntries
+   */
+  @Override
+  public List<DiskSchemaEntry> getDiskSchemaEntriesByOffset(List<Long> offsets) {
+    List<DiskSchemaEntry> diskSchemaEntries = new ArrayList<>(offsets.size());
+    Collections.sort(offsets);
+    try (RandomAccessFile randomAccessFile = new RandomAccessFile(dataFile, "r")) {
+      for (long offset : offsets) {

Review Comment:
   Make this an attribute of this class. Frequent open operation of file will cost serious performance loss.



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/AppendOnlyDiskSchemaManager.java:
##########
@@ -190,6 +198,65 @@ public Collection<DiskSchemaEntry> getAllSchemaEntry() throws IOException {
     return res;
   }
 
+  /**
+   * get DiskSchemaEntries from disk file
+   *
+   * @param offsets the offset of each record on the disk file
+   * @return DiskSchemaEntries
+   */
+  @Override
+  public List<DiskSchemaEntry> getDiskSchemaEntriesByOffset(List<Long> offsets) {
+    List<DiskSchemaEntry> diskSchemaEntries = new ArrayList<>(offsets.size());
+    Collections.sort(offsets);
+    try (RandomAccessFile randomAccessFile = new RandomAccessFile(dataFile, "r")) {
+      for (long offset : offsets) {
+        diskSchemaEntries.add(getDiskSchemaEntryByOffset(randomAccessFile, offset));
+      }
+    } catch (FileNotFoundException e) {
+      logger.info(e.getMessage());

Review Comment:
   Replace info with error.



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/AppendOnlyDiskSchemaManager.java:
##########
@@ -190,6 +198,65 @@ public Collection<DiskSchemaEntry> getAllSchemaEntry() throws IOException {
     return res;
   }
 
+  /**
+   * get DiskSchemaEntries from disk file
+   *
+   * @param offsets the offset of each record on the disk file
+   * @return DiskSchemaEntries
+   */
+  @Override
+  public List<DiskSchemaEntry> getDiskSchemaEntriesByOffset(List<Long> offsets) {
+    List<DiskSchemaEntry> diskSchemaEntries = new ArrayList<>(offsets.size());
+    Collections.sort(offsets);
+    try (RandomAccessFile randomAccessFile = new RandomAccessFile(dataFile, "r")) {
+      for (long offset : offsets) {
+        diskSchemaEntries.add(getDiskSchemaEntryByOffset(randomAccessFile, offset));
+      }
+    } catch (FileNotFoundException e) {
+      logger.info(e.getMessage());
+    } catch (IOException e) {
+      logger.error(e.getMessage());
+    }
+    return diskSchemaEntries;
+  }
+
+  /**
+   * delete DiskSchemaEntries on disk
+   *
+   * @param offset the offset of a record on the disk file
+   * @throws MetadataException
+   */
+  @Override
+  public void deleteDiskSchemaEntriesByOffset(long offset) throws MetadataException {
+    try (RandomAccessFile randomAccessFile = new RandomAccessFile(dataFile, "rw")) {
+      DiskSchemaEntry.writeTombstone(randomAccessFile, offset + FILE_VERSION.length() + 4);
+    } catch (IOException e) {
+      logger.error(e.getMessage());
+      throw new MetadataException(e.getMessage());
+    }
+  }
+
+  private DiskSchemaEntry getDiskSchemaEntryByOffset(RandomAccessFile randomAccessFile, long offset)
+      throws IOException {
+    randomAccessFile.seek(offset + FILE_VERSION.length() + 4);

Review Comment:
   What's the meaning of this ```4```? Make this a global variable or a static variable.



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTableHashmapImpl.java:
##########
@@ -129,6 +134,42 @@ public synchronized void createTimeseries(CreateTimeSeriesPlan plan) throws Meta
     deviceEntry.putSchemaEntry(plan.getPath().getMeasurement(), schemaEntry);
   }
 
+  /**
+   * Delete all timeseries matching the given paths
+   *
+   * @param fullPaths paths to be deleted
+   * @return deletion failed Timeseries
+   * @throws MetadataException
+   */
+  @Override
+  public Pair<Integer, Set<String>> deleteTimeseries(List<PartialPath> fullPaths)
+      throws MetadataException {
+    int deletedNum = 0;
+    Set<String> failedNames = new HashSet<>();
+    List<Pair<PartialPath, Long>> deletedPairs = new ArrayList<>(fullPaths.size());
+    for (PartialPath fullPath : fullPaths) {
+      try {
+        Map<String, SchemaEntry> map = getDeviceEntry(fullPath.getDevice()).getMeasurementMap();
+        Long offset = map.get(fullPath.getMeasurement()).getDiskPointer();
+        deletedPairs.add(new Pair<>(fullPath, offset));
+      } catch (NullPointerException e) {
+        failedNames.add(fullPath.getFullPath());
+      }

Review Comment:
   Replace this NullPointerException catch with ```if(xx==null)```.



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/entry/DiskSchemaEntry.java:
##########
@@ -100,10 +103,31 @@ public static DiskSchemaEntry deserialize(InputStream inputStream) throws IOExce
     // read byte len
     res.entrySize = ReadWriteIOUtils.readInt(inputStream);
     res.entrySize += Integer.BYTES;
-
+    if (isTombstone(res.deviceID)) res.deviceID = TOMBSTONE;
     return res;
   }
 
+  public static void writeTombstone(RandomAccessFile randomAccessFile, long offset)
+      throws IOException {
+    randomAccessFile.seek(offset);
+    int strLength = randomAccessFile.readInt();
+    byte[] bytes = new byte[strLength];
+    randomAccessFile.write(bytes, 0, strLength);
+  }
+
+  public static String readString(RandomAccessFile randomAccessFile) throws IOException {
+    int strLength = randomAccessFile.readInt();
+    byte[] bytes = new byte[strLength];
+    randomAccessFile.read(bytes, 0, strLength);
+    return new String(bytes, 0, strLength);
+  }

Review Comment:
   This class should only sense the byte data rather than the file object. Remove randomAccessFile from this class.



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTable.java:
##########
@@ -194,6 +206,14 @@ static PartialPath translateQueryPath(PartialPath fullPath) {
     }
   }
 
+  /**
+   * get DiskSchemaEntries from disk file
+   *
+   * @param schemaEntries get the disk pointers from schemaEntries
+   * @return DiskSchemaEntries
+   */
+  List<DiskSchemaEntry> getDiskSchemaEntries(List<SchemaEntry> schemaEntries);
+

Review Comment:
   It seems this is only used in test. Annotation ```@TestOnly``` could be 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] KeeProMise commented on a diff in pull request #7024: Idtable supports getting time series from disk and deleting time series

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


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/AppendOnlyDiskSchemaManager.java:
##########
@@ -190,6 +198,65 @@ public Collection<DiskSchemaEntry> getAllSchemaEntry() throws IOException {
     return res;
   }
 
+  /**
+   * get DiskSchemaEntries from disk file
+   *
+   * @param offsets the offset of each record on the disk file
+   * @return DiskSchemaEntries
+   */
+  @Override
+  public List<DiskSchemaEntry> getDiskSchemaEntriesByOffset(List<Long> offsets) {
+    List<DiskSchemaEntry> diskSchemaEntries = new ArrayList<>(offsets.size());
+    Collections.sort(offsets);
+    try (RandomAccessFile randomAccessFile = new RandomAccessFile(dataFile, "r")) {
+      for (long offset : offsets) {

Review Comment:
   fixed



##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/AppendOnlyDiskSchemaManager.java:
##########
@@ -190,6 +198,65 @@ public Collection<DiskSchemaEntry> getAllSchemaEntry() throws IOException {
     return res;
   }
 
+  /**
+   * get DiskSchemaEntries from disk file
+   *
+   * @param offsets the offset of each record on the disk file
+   * @return DiskSchemaEntries
+   */
+  @Override
+  public List<DiskSchemaEntry> getDiskSchemaEntriesByOffset(List<Long> offsets) {
+    List<DiskSchemaEntry> diskSchemaEntries = new ArrayList<>(offsets.size());
+    Collections.sort(offsets);
+    try (RandomAccessFile randomAccessFile = new RandomAccessFile(dataFile, "r")) {
+      for (long offset : offsets) {
+        diskSchemaEntries.add(getDiskSchemaEntryByOffset(randomAccessFile, offset));
+      }
+    } catch (FileNotFoundException e) {
+      logger.info(e.getMessage());
+    } catch (IOException e) {
+      logger.error(e.getMessage());
+    }
+    return diskSchemaEntries;
+  }
+
+  /**
+   * delete DiskSchemaEntries on disk
+   *
+   * @param offset the offset of a record on the disk file
+   * @throws MetadataException
+   */
+  @Override
+  public void deleteDiskSchemaEntriesByOffset(long offset) throws MetadataException {
+    try (RandomAccessFile randomAccessFile = new RandomAccessFile(dataFile, "rw")) {
+      DiskSchemaEntry.writeTombstone(randomAccessFile, offset + FILE_VERSION.length() + 4);
+    } catch (IOException e) {
+      logger.error(e.getMessage());
+      throw new MetadataException(e.getMessage());
+    }
+  }
+
+  private DiskSchemaEntry getDiskSchemaEntryByOffset(RandomAccessFile randomAccessFile, long offset)
+      throws IOException {
+    randomAccessFile.seek(offset + FILE_VERSION.length() + 4);

Review Comment:
   fixed



-- 
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 #7024: Idtable supports getting time series from disk and deleting time series

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


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTable.java:
##########
@@ -129,7 +141,7 @@ void updateLastCache(
    * @param deviceName device name of the time series
    * @return device entry of the timeseries
    */
-  public DeviceEntry getDeviceEntry(String deviceName);
+  DeviceEntry getDeviceEntry(String deviceName);

Review Comment:
   The default methods of java interfaces are public and abstract, so there is no need to add public.
   



-- 
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 #7024: Idtable supports getting time series from disk and deleting time series

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


##########
server/src/main/java/org/apache/iotdb/db/metadata/idtable/IDTable.java:
##########
@@ -194,6 +206,14 @@ static PartialPath translateQueryPath(PartialPath fullPath) {
     }
   }
 
+  /**
+   * get DiskSchemaEntries from disk file
+   *
+   * @param schemaEntries get the disk pointers from schemaEntries
+   * @return DiskSchemaEntries
+   */
+  List<DiskSchemaEntry> getDiskSchemaEntries(List<SchemaEntry> schemaEntries);
+

Review Comment:
   fixed



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