You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by lt...@apache.org on 2020/01/14 01:25:09 UTC

[incubator-iotdb] branch master updated: Change sync log and sync system dir (#715)

This is an automated email from the ASF dual-hosted git repository.

lta pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 3364f7c  Change sync log and sync system dir (#715)
3364f7c is described below

commit 3364f7c90e68f2c9bf68376ba1031c69e7cc4097
Author: Tianan Li <li...@163.com>
AuthorDate: Tue Jan 14 09:25:02 2020 +0800

    Change sync log and sync system dir (#715)
    
    * change lock file path and uuid file path to system folder
    
    * redirect sync log
    
    * catch all exception
    
    * merge log
---
 server/src/assembly/resources/conf/logback.xml     | 12 ++--
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java | 14 +++++
 .../org/apache/iotdb/db/conf/IoTDBConstant.java    |  7 +++
 .../org/apache/iotdb/db/conf/IoTDBDescriptor.java  |  8 ++-
 .../iotdb/db/sync/conf/SyncSenderConfig.java       | 18 ------
 .../db/sync/receiver/load/FileLoaderManager.java   |  9 ++-
 .../iotdb/db/sync/sender/transfer/SyncClient.java  | 66 ++++++++++++----------
 7 files changed, 71 insertions(+), 63 deletions(-)

diff --git a/server/src/assembly/resources/conf/logback.xml b/server/src/assembly/resources/conf/logback.xml
index e199641..dbae7b2 100644
--- a/server/src/assembly/resources/conf/logback.xml
+++ b/server/src/assembly/resources/conf/logback.xml
@@ -146,10 +146,10 @@
             <level>INFO</level>
         </filter>
     </appender>
-    <appender class="ch.qos.logback.core.rolling.RollingFileAppender" name="FILE_DYNAMIC_PARAMETER">
-        <file>${IOTDB_HOME}/logs/log_dynamic_adapter.log</file>
+    <appender class="ch.qos.logback.core.rolling.RollingFileAppender" name="SYNC">
+        <file>${IOTDB_HOME}/logs/log_sync.log</file>
         <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
-            <fileNamePattern>${IOTDB_HOME}/logs/log-measure-%d{yyyy-MM-dd}.%i.log</fileNamePattern>
+            <fileNamePattern>${IOTDB_HOME}/logs/log-sync-%d{yyyy-MM-dd}.%i.log</fileNamePattern>
             <timeBasedFileNamingAndTriggeringPolicy class="ch.qos.logback.core.rolling.SizeAndTimeBasedFNATP">
                 <maxFileSize>200MB</maxFileSize>
             </timeBasedFileNamingAndTriggeringPolicy>
@@ -160,7 +160,7 @@
             <charset>utf-8</charset>
         </encoder>
         <filter class="ch.qos.logback.classic.filter.ThresholdFilter">
-            <level>DEBUG</level>
+            <level>INFO</level>
         </filter>
     </appender>
     <root level="info">
@@ -173,7 +173,7 @@
     <logger level="info" name="org.apache.iotdb.db.cost.statistic">
         <appender-ref ref="FILE_COST_MEASURE"/>
     </logger>
-    <logger level="debug" name="org.apache.iotdb.db.conf.adapter.IoTDBConfigDynamicAdapter">
-        <appender-ref ref="FILE_DYNAMIC_PARAMETER"/>
+    <logger level="info" name="org.apache.iotdb.db.sync">
+        <appender-ref ref="SYNC"/>
     </logger>
 </configuration>
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
index b2c5974..92ac468 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java
@@ -128,6 +128,11 @@ public class IoTDBConfig {
   private String schemaDir = "data/system/schema";
 
   /**
+   * Sync directory, including the lock file, uuid file, device owner map
+   */
+  private String syncDir = "data/system/sync";
+
+  /**
    * Query directory, stores temporary files of query
    */
   private String queryDir = "data/query";
@@ -483,6 +488,7 @@ public class IoTDBConfig {
     baseDir = addHomeDir(baseDir);
     systemDir = addHomeDir(systemDir);
     schemaDir = addHomeDir(schemaDir);
+    syncDir = addHomeDir(syncDir);
     walFolder = addHomeDir(walFolder);
 
     if (TSFileDescriptor.getInstance().getConfig().getTSFileStorageFs().equals(FSType.HDFS)) {
@@ -630,6 +636,14 @@ public class IoTDBConfig {
     this.schemaDir = schemaDir;
   }
 
+  public String getSyncDir() {
+    return syncDir;
+  }
+
+  public void setSyncDir(String syncDir) {
+    this.syncDir = syncDir;
+  }
+
   public String getQueryDir() {
     return queryDir;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
index c995393..a4dc393 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java
@@ -81,4 +81,11 @@ public class IoTDBConstant {
   public static final String TSFILE_NAME_SEPARATOR = "-";
 
   public static final int MEMTABLE_NUM_IN_EACH_STORAGE_GROUP = 4;
+
+  // system folder name
+  public static final String SYSTEM_FOLDER_NAME = "system";
+  public static final String SCHEMA_FOLDER_NAME = "schema";
+  public static final String SYNC_FOLDER_NAME = "sync";
+  public static final String QUERY_FOLDER_NAME = "query";
+
 }
diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
index 91996e2..6d4be83 100644
--- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java
@@ -146,11 +146,13 @@ public class IoTDBDescriptor {
 
       conf.setBaseDir(properties.getProperty("base_dir", conf.getBaseDir()));
 
-      conf.setSystemDir(FilePathUtils.regularizePath(conf.getBaseDir()) + "system");
+      conf.setSystemDir(FilePathUtils.regularizePath(conf.getBaseDir()) + IoTDBConstant.SYSTEM_FOLDER_NAME);
 
-      conf.setSchemaDir(FilePathUtils.regularizePath(conf.getSystemDir()) + "schema");
+      conf.setSchemaDir(FilePathUtils.regularizePath(conf.getSystemDir()) + IoTDBConstant.SCHEMA_FOLDER_NAME);
 
-      conf.setQueryDir(FilePathUtils.regularizePath(conf.getBaseDir()) + "query");
+      conf.setSyncDir(FilePathUtils.regularizePath(conf.getSystemDir()) + IoTDBConstant.SYNC_FOLDER_NAME);
+
+      conf.setQueryDir(FilePathUtils.regularizePath(conf.getBaseDir()) + IoTDBConstant.QUERY_FOLDER_NAME);
 
       conf.setDataDirs(properties.getProperty("data_dirs", conf.getDataDirs()[0])
           .split(","));
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderConfig.java b/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderConfig.java
index e3ed539..e7d9466 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderConfig.java
@@ -32,10 +32,6 @@ public class SyncSenderConfig {
 
   private String senderFolderPath;
 
-  private String lockFilePath;
-
-  private String uuidPath;
-
   private String lastFileInfoPath;
 
   private String snapshotPath;
@@ -57,8 +53,6 @@ public class SyncSenderConfig {
     senderFolderPath =
         dataDirectory + File.separatorChar + SyncConstant.SYNC_SENDER + File.separatorChar +
             getSyncReceiverName();
-    lockFilePath = senderFolderPath + File.separatorChar + SyncConstant.LOCK_FILE_NAME;
-    uuidPath = senderFolderPath + File.separatorChar + SyncConstant.UUID_FILE_NAME;
     lastFileInfoPath = senderFolderPath + File.separatorChar + SyncConstant.LAST_LOCAL_FILE_NAME;
     snapshotPath = senderFolderPath + File.separatorChar + SyncConstant.DATA_SNAPSHOT_NAME;
     if (!new File(snapshotPath).exists()) {
@@ -98,14 +92,6 @@ public class SyncSenderConfig {
     this.senderFolderPath = senderFolderPath;
   }
 
-  public String getLockFilePath() {
-    return lockFilePath;
-  }
-
-  public void setLockFilePath(String lockFilePath) {
-    this.lockFilePath = lockFilePath;
-  }
-
   public String getLastFileInfoPath() {
     return lastFileInfoPath;
   }
@@ -122,10 +108,6 @@ public class SyncSenderConfig {
     this.snapshotPath = snapshotPath;
   }
 
-  public String getUuidPath() {
-    return uuidPath;
-  }
-
   public String getSyncReceiverName() {
     return serverIp + SyncConstant.SYNC_DIR_NAME_SEPARATOR + serverPort;
   }
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/FileLoaderManager.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/FileLoaderManager.java
index 811ce3c..289d477 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/FileLoaderManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/FileLoaderManager.java
@@ -60,15 +60,14 @@ public class FileLoaderManager {
   private File deviceOwnerTmpFile;
 
   private FileLoaderManager() {
-    File deviceOwnerMapDir = new File(IoTDBDescriptor.getInstance().getConfig().getSystemDir(),
-        SyncConstant.SYNC_NAME);
-    deviceOwnerFile = new File(deviceOwnerMapDir, SyncConstant.DEVICE_OWNER_FILE_NAME);
-    deviceOwnerTmpFile = new File(deviceOwnerMapDir, SyncConstant.DEVICE_OWNER_TMP_FILE_NAME);
+    String syncSystemDir = IoTDBDescriptor.getInstance().getConfig().getSyncDir();
+    deviceOwnerFile = new File(syncSystemDir, SyncConstant.DEVICE_OWNER_FILE_NAME);
+    deviceOwnerTmpFile = new File(syncSystemDir, SyncConstant.DEVICE_OWNER_TMP_FILE_NAME);
     try {
       recoverDeviceOwnerMap();
     } catch (IOException | ClassNotFoundException e) {
       LOGGER.error("Can not recover device owner map from file {}",
-          new File(deviceOwnerMapDir, SyncConstant.DEVICE_OWNER_FILE_NAME).getAbsolutePath());
+          new File(syncSystemDir, SyncConstant.DEVICE_OWNER_FILE_NAME).getAbsolutePath());
     }
   }
 
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/SyncClient.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/SyncClient.java
index 692509a..5b294d0 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/SyncClient.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/SyncClient.java
@@ -64,8 +64,8 @@ import org.apache.iotdb.db.sync.sender.recover.ISyncSenderLogger;
 import org.apache.iotdb.db.sync.sender.recover.SyncSenderLogAnalyzer;
 import org.apache.iotdb.db.sync.sender.recover.SyncSenderLogger;
 import org.apache.iotdb.db.utils.SyncUtils;
-import org.apache.iotdb.service.sync.thrift.SyncStatus;
 import org.apache.iotdb.service.sync.thrift.SyncService;
+import org.apache.iotdb.service.sync.thrift.SyncStatus;
 import org.apache.iotdb.tsfile.utils.BytesUtils;
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TBinaryProtocol;
@@ -139,32 +139,27 @@ public class SyncClient implements ISyncClient {
 
   @Override
   public void verifySingleton() throws IOException {
-    String[] dataDirs = IoTDBDescriptor.getInstance().getConfig().getDataDirs();
-    for (String dataDir : dataDirs) {
-      config.update(dataDir);
-      File lockFile = new File(config.getLockFilePath());
-      if (!lockFile.getParentFile().exists()) {
-        lockFile.getParentFile().mkdirs();
-      }
-      if (!lockFile.exists()) {
-        lockFile.createNewFile();
-      }
-      if (!lockInstance(config.getLockFilePath())) {
-        logger.error("Sync client is already running.");
-        System.exit(1);
-      }
+    File lockFile = getLockFile();
+    if (!lockFile.getParentFile().exists()) {
+      lockFile.getParentFile().mkdirs();
+    }
+    if (!lockFile.exists()) {
+      lockFile.createNewFile();
+    }
+    if (!lockInstance(lockFile)) {
+      logger.error("Sync client is already running.");
+      System.exit(1);
     }
   }
 
   /**
-   * Try to lock lockfile. if failed, it means that sync client has benn started.
+   * Try to lock lockfile. if failed, it means that sync client has been started.
    *
-   * @param lockFile path of lock file
+   * @param lockFile lock file
    */
-  private boolean lockInstance(final String lockFile) {
+  private boolean lockInstance(File lockFile) {
     try {
-      final File file = new File(lockFile);
-      final RandomAccessFile randomAccessFile = new RandomAccessFile(file, "rw");
+      final RandomAccessFile randomAccessFile = new RandomAccessFile(lockFile, "rw");
       final FileLock fileLock = randomAccessFile.getChannel().tryLock();
       if (fileLock != null) {
         Runtime.getRuntime().addShutdownHook(new Thread(() -> {
@@ -206,7 +201,7 @@ public class SyncClient implements ISyncClient {
     executorService.scheduleWithFixedDelay(() -> {
       try {
         syncAll();
-      } catch (SyncConnectionException | IOException | TException e) {
+      } catch (Exception e) {
         logger.error("Sync failed", e);
       }
     }, SyncConstant.SYNC_PROCESS_DELAY, SyncConstant.SYNC_PROCESS_PERIOD, TimeUnit.SECONDS);
@@ -288,7 +283,7 @@ public class SyncClient implements ISyncClient {
   public void confirmIdentity() throws SyncConnectionException {
     try (Socket socket = new Socket(config.getServerIp(), config.getServerPort())){
       SyncStatus status = serviceClient
-          .check(socket.getLocalAddress().getHostAddress(), getOrCreateUUID(config.getUuidPath()));
+          .check(socket.getLocalAddress().getHostAddress(), getOrCreateUUID(getUuidFile()));
       if (status.code != SUCCESS_CODE) {
         throw new SyncConnectionException(
             "The receiver rejected the synchronization task because " + status.msg);
@@ -302,25 +297,24 @@ public class SyncClient implements ISyncClient {
   /**
    * UUID marks the identity of sender for receiver.
    */
-  private String getOrCreateUUID(String uuidPath) throws IOException {
-    File file = new File(uuidPath);
+  private String getOrCreateUUID(File uuidFile) throws IOException {
     String uuid;
-    if (!file.getParentFile().exists()) {
-      file.getParentFile().mkdirs();
+    if (!uuidFile.getParentFile().exists()) {
+      uuidFile.getParentFile().mkdirs();
     }
-    if (!file.exists()) {
-      try (FileOutputStream out = new FileOutputStream(file)) {
+    if (!uuidFile.exists()) {
+      try (FileOutputStream out = new FileOutputStream(uuidFile)) {
         uuid = generateUUID();
         out.write(uuid.getBytes());
       } catch (IOException e) {
-        logger.error("Cannot insert UUID to file {}", file.getPath());
+        logger.error("Cannot insert UUID to file {}", uuidFile.getPath());
         throw new IOException(e);
       }
     } else {
-      try (BufferedReader bf = new BufferedReader((new FileReader(uuidPath)))) {
+      try (BufferedReader bf = new BufferedReader((new FileReader(uuidFile.getAbsolutePath())))) {
         uuid = bf.readLine();
       } catch (IOException e) {
-        logger.error("Cannot read UUID from file{}", file.getPath());
+        logger.error("Cannot read UUID from file{}", uuidFile.getPath());
         throw new IOException(e);
       }
     }
@@ -656,6 +650,16 @@ public class SyncClient implements ISyncClient {
         MetadataConstant.METADATA_LOG);
   }
 
+  private File getLockFile() {
+    return new File(IoTDBDescriptor.getInstance().getConfig().getSyncDir(),
+        SyncConstant.LOCK_FILE_NAME);
+  }
+
+  private File getUuidFile() {
+    return new File(IoTDBDescriptor.getInstance().getConfig().getSyncDir(),
+        SyncConstant.UUID_FILE_NAME);
+  }
+
   private static class InstanceHolder {
 
     private static final SyncClient INSTANCE = new SyncClient();