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/10/09 05:28:29 UTC

[GitHub] [iotdb] MarcosZyk commented on a diff in pull request #7424: [IOTDB-4520] The tag schema region is implemented and supports the influxdb protocol

MarcosZyk commented on code in PR #7424:
URL: https://github.com/apache/iotdb/pull/7424#discussion_r990734926


##########
lsm/src/main/java/org/apache/iotdb/lsm/levelProcess/BasicLevelProcess.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.lsm.levelProcess;
+
+import org.apache.iotdb.lsm.context.RequestContext;
+
+import java.util.List;
+
+/** the processing method corresponding to each layer of memory nodes */
+public abstract class BasicLevelProcess<I, O, C extends RequestContext>
+    implements LevelProcess<I, O, C> {
+
+  // the next level process
+  LevelProcess<O, ?, C> next;
+
+  /**
+   * process the current layer memory node
+   *
+   * @param memNode memory node
+   * @param context request context
+   */
+  public abstract void handle(I memNode, C context);

Review Comment:
   Maybe we should define an interface named ```IMemNode```, and state some methods in it.



##########
lsm/src/main/java/org/apache/iotdb/lsm/wal/WALRecord.java:
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.lsm.wal;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+/** represents a wal record, which can be extended to implement more complex wal records */
+public abstract class WALRecord implements Cloneable {

Review Comment:
   Make this an Interface. The abstract class is not that convenient in Java, since Java only supports single extension.



##########
lsm/src/main/java/org/apache/iotdb/lsm/manager/LsmManager.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.lsm.manager;
+
+import org.apache.iotdb.lsm.context.RequestContext;
+import org.apache.iotdb.lsm.levelProcess.LevelProcess;
+
+// used to implement lsm manager
+public interface LsmManager<T, C extends RequestContext> {

Review Comment:
   Use upper case instead, ```LSMManager```.



##########
lsm/src/main/java/org/apache/iotdb/lsm/context/QueryRequestContext.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.lsm.context;
+
+import org.apache.iotdb.lsm.strategy.PostOrderAccessStrategy;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * represents the context of a query request, this class can be extended to implement a custom
+ * context
+ */
+public class QueryRequestContext extends RequestContext {
+
+  // save the key of each level
+  List<Object> keys;
+

Review Comment:
   We shall avoid direct use ```Object``` class. Is there any way to implement this feature with generics? 
   
   Besides, the attribute ```keys``` seems to be part of the query request, rather than the query context. The ```context``` usually means to provide some states of the system or module when executing the request.



##########
schema-engine-tag/src/main/java/org/apache/iotdb/db/metadata/tagSchemaRegion/deviceidlist/AppendOnlyDeviceIDListFileManager.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.tagSchemaRegion.deviceidlist;
+
+import org.apache.iotdb.commons.utils.TestOnly;
+import org.apache.iotdb.db.metadata.idtable.entry.DeviceIDFactory;
+import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+/** for append-only writing of device id list to disk */
+public class AppendOnlyDeviceIDListFileManager {

Review Comment:
   It seems we can recover the ```DeviceIDList``` by redo the file of IDTable.



##########
lsm/src/main/java/org/apache/iotdb/lsm/wal/WALReader.java:
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.lsm.wal;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.NoSuchElementException;
+
+/** get records in wal file */
+public class WALReader implements IWALReader {
+  private static final Logger logger = LoggerFactory.getLogger(WALReader.class);
+  // wal file
+  private final File logFile;
+  // wal record prototype, clone on read
+  private final WALRecord prototype;
+  private DataInputStream logStream;
+  // next wal record
+  private WALRecord nextRecord;
+  private boolean fileCorrupted = false;
+
+  public WALReader(File logFile, WALRecord prototype) throws IOException {
+    this.logFile = logFile;
+    this.logStream =
+        new DataInputStream(new BufferedInputStream(Files.newInputStream(logFile.toPath())));
+    this.prototype = prototype;
+  }
+
+  @Override
+  public void close() throws IOException {
+    logStream.close();
+    logStream = null;
+  }
+
+  @Override
+  public boolean hasNext() {
+    if (nextRecord != null) {
+      return true;
+    }
+    try {
+      if (fileCorrupted) {
+        return false;
+      }
+      int logSize = logStream.readInt();
+      if (logSize <= 0) {
+        return false;
+      }
+      // first clone the object through the prototype
+      nextRecord = prototype.clone();
+      // then perform deserialization and assign a value to the new object
+      nextRecord.deserialize(logStream);
+    } catch (EOFException e) {
+      logger.info(e.getMessage());
+      return false;
+    } catch (IOException e) {
+      logger.warn(e.getMessage());
+      fileCorrupted = true;

Review Comment:
   When file corrupted, we should recognize whether it is the middle of file or just the tail of the file. 
   
   If is tail, it maybe the last log is not completed yet, and we can truncate the corrupted file tail directly. 
   
   If is middle, it maybe complex and we should notice the user that the file is corrupted due to unknown cause.



##########
schema-engine-tag/src/main/java/org/apache/iotdb/db/metadata/tagSchemaRegion/TagSchemaConfig.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.tagSchemaRegion;
+
+/** tag schema region config */
+public class TagSchemaConfig {

Review Comment:
   Move this to package ```config```.



##########
schema-engine-tag/src/main/java/org/apache/iotdb/db/metadata/tagSchemaRegion/TagSchemaDescriptor.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.tagSchemaRegion;
+
+import org.apache.iotdb.commons.conf.IoTDBConstant;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Properties;
+
+/** manager tag schema config */
+public class TagSchemaDescriptor {

Review Comment:
   Move this to package ```config```.



##########
lsm/src/main/java/org/apache/iotdb/lsm/manager/LsmManager.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.lsm.manager;
+
+import org.apache.iotdb.lsm.context.RequestContext;
+import org.apache.iotdb.lsm.levelProcess.LevelProcess;
+
+// used to implement lsm manager
+public interface LsmManager<T, C extends RequestContext> {
+
+  /**
+   * use this method to process root memory node
+   *
+   * @param memNode memory node
+   * @param context request context
+   */
+  void process(T memNode, C context) throws Exception;
+
+  /**
+   * add the LevelProcess of the next layer of memory nodes
+   *
+   * @param next LevelProcess of the next layer
+   * @return LevelProcess of the next layer
+   */
+  <O> LevelProcess<T, O, C> nextLevel(LevelProcess<T, O, C> next);

Review Comment:
   It seems this method is used to build the process method of ```LSMManager```. What about separating these building methods into an individual  ```LSMManagerBuilder```. The ```LSMManager``` will only keep the executing methods. PLZ refer to the design pattern, Builder Pattern.



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