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 2019/08/22 03:14:12 UTC

[incubator-iotdb] 01/05: add sync new code framework

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

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

commit 85cd7783b87e4a01fafb6dccd45629b2d2e3dc9f
Author: lta <li...@163.com>
AuthorDate: Mon Aug 12 20:18:02 2019 +0800

    add sync new code framework
---
 server/iotdb/tools/start-sync-client.bat           |  2 +-
 server/iotdb/tools/start-sync-client.sh            |  2 +-
 .../org/apache/iotdb/db/sync/package-info.java     | 20 +++++++++++++
 .../iotdb/db/sync/receiver/SyncServerManager.java  |  1 +
 .../iotdb/db/sync/receiver/load/IFileLoader.java   | 25 ++++++++++++++++
 .../receiver/recover/ISyncReceiverLogAnalyzer.java | 27 +++++++++++++++++
 .../sync/receiver/recover/ISyncReceiverLogger.java | 23 ++++++++++++++
 .../receiver/{ => transfer}/SyncServiceImpl.java   |  4 +--
 .../iotdb/db/sync/sender/SyncFileManager.java      |  6 ++--
 .../iotdb/db/sync/{ => sender}/conf/Constans.java  |  2 +-
 .../sync/{ => sender}/conf/SyncSenderConfig.java   |  2 +-
 .../{ => sender}/conf/SyncSenderDescriptor.java    |  2 +-
 .../iotdb/db/sync/sender/manage/IFileManager.java  | 25 ++++++++++++++++
 .../sender/recover/ISyncSenderLogAnalyzer.java     | 33 ++++++++++++++++++++
 .../db/sync/sender/recover/ISyncSenderLogger.java  | 35 ++++++++++++++++++++++
 .../DataTransferManager.java}                      | 21 ++++++-------
 .../IDataTransferManager.java}                     |  4 +--
 .../java/org/apache/iotdb/db/utils/SyncUtils.java  |  2 +-
 .../iotdb/db/sync/sender/SingleClientSyncTest.java |  9 +++---
 .../iotdb/db/sync/sender/SyncFileManagerTest.java  |  2 +-
 20 files changed, 219 insertions(+), 28 deletions(-)

diff --git a/server/iotdb/tools/start-sync-client.bat b/server/iotdb/tools/start-sync-client.bat
index 49360ef..dc44fcd 100755
--- a/server/iotdb/tools/start-sync-client.bat
+++ b/server/iotdb/tools/start-sync-client.bat
@@ -29,7 +29,7 @@ set IOTDB_CONF=%IOTDB_HOME%\conf
 set IOTDB_LOGS=%IOTDB_HOME%\logs
 
 
-if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.db.sync.sender.SyncSenderImpl
+if NOT DEFINED MAIN_CLASS set MAIN_CLASS=org.apache.iotdb.db.sync.sender.transfer.DataTransferManager
 if NOT DEFINED JAVA_HOME goto :err
 
 @REM -----------------------------------------------------------------------------
diff --git a/server/iotdb/tools/start-sync-client.sh b/server/iotdb/tools/start-sync-client.sh
index c796197..a3ae56a 100755
--- a/server/iotdb/tools/start-sync-client.sh
+++ b/server/iotdb/tools/start-sync-client.sh
@@ -47,7 +47,7 @@ for f in ${IOTDB_HOME}/lib/*.jar; do
   CLASSPATH=${CLASSPATH}":"$f
 done
 
-MAIN_CLASS=org.apache.iotdb.db.sync.sender.SyncSenderImpl
+MAIN_CLASS=org.apache.iotdb.db.sync.sender.transfer.DataTransferManager
 
 "$JAVA" -DIOTDB_HOME=${IOTDB_HOME} -DTSFILE_HOME=${IOTDB_HOME} -DIOTDB_CONF=${IOTDB_CONF} -Dlogback.configurationFile=${IOTDB_CONF}/logback.xml $IOTDB_DERBY_OPTS $IOTDB_JMX_OPTS -Dname=SyncClient -cp "$CLASSPATH" "$MAIN_CLASS"
 
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/package-info.java b/server/src/main/java/org/apache/iotdb/db/sync/package-info.java
new file mode 100644
index 0000000..97a4ec5
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/package-info.java
@@ -0,0 +1,20 @@
+/**
+ * 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.sync;
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManager.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManager.java
index e4e220b..ee1fd82 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServerManager.java
@@ -27,6 +27,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor;
 import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.service.IService;
 import org.apache.iotdb.db.service.ServiceType;
+import org.apache.iotdb.db.sync.receiver.transfer.SyncServiceImpl;
 import org.apache.iotdb.service.sync.thrift.SyncService;
 import org.apache.iotdb.service.sync.thrift.SyncService.Processor;
 import org.apache.thrift.protocol.TBinaryProtocol;
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/IFileLoader.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/IFileLoader.java
new file mode 100644
index 0000000..aaa3ff7
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/load/IFileLoader.java
@@ -0,0 +1,25 @@
+/**
+ * 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.sync.receiver.load;
+
+public interface IFileLoader {
+
+  void loadTsFiles();
+
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
new file mode 100644
index 0000000..5d6351e
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogAnalyzer.java
@@ -0,0 +1,27 @@
+/**
+ * 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.sync.receiver.recover;
+
+public interface ISyncReceiverLogAnalyzer {
+  void recover();
+
+  void scanLogger(String path);
+
+  void clearLogger();
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogger.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogger.java
new file mode 100644
index 0000000..9fd3109
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/recover/ISyncReceiverLogger.java
@@ -0,0 +1,23 @@
+/**
+ * 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.sync.receiver.recover;
+
+public interface ISyncReceiverLogger {
+
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/sync/receiver/transfer/SyncServiceImpl.java
similarity index 99%
rename from server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServiceImpl.java
rename to server/src/main/java/org/apache/iotdb/db/sync/receiver/transfer/SyncServiceImpl.java
index 1dcd4f5..4866d41 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/receiver/SyncServiceImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/receiver/transfer/SyncServiceImpl.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.sync.receiver;
+package org.apache.iotdb.db.sync.receiver.transfer;
 
 import java.io.BufferedReader;
 import java.io.File;
@@ -54,7 +54,7 @@ import org.apache.iotdb.db.metadata.MetadataConstant;
 import org.apache.iotdb.db.metadata.MetadataOperationType;
 import org.apache.iotdb.db.qp.executor.QueryProcessExecutor;
 import org.apache.iotdb.db.qp.physical.crud.InsertPlan;
-import org.apache.iotdb.db.sync.conf.Constans;
+import org.apache.iotdb.db.sync.sender.conf.Constans;
 import org.apache.iotdb.db.utils.FilePathUtils;
 import org.apache.iotdb.db.utils.SyncUtils;
 import org.apache.iotdb.service.sync.thrift.SyncDataStatus;
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncFileManager.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncFileManager.java
index c7119aa..18c3b60 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncFileManager.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncFileManager.java
@@ -31,9 +31,9 @@ import java.util.Map.Entry;
 import java.util.Set;
 import org.apache.iotdb.db.conf.IoTDBConfig;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
-import org.apache.iotdb.db.sync.conf.Constans;
-import org.apache.iotdb.db.sync.conf.SyncSenderConfig;
-import org.apache.iotdb.db.sync.conf.SyncSenderDescriptor;
+import org.apache.iotdb.db.sync.sender.conf.Constans;
+import org.apache.iotdb.db.sync.sender.conf.SyncSenderConfig;
+import org.apache.iotdb.db.sync.sender.conf.SyncSenderDescriptor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/conf/Constans.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/conf/Constans.java
similarity index 97%
rename from server/src/main/java/org/apache/iotdb/db/sync/conf/Constans.java
rename to server/src/main/java/org/apache/iotdb/db/sync/sender/conf/Constans.java
index c810c4c..c8e2ce2 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/conf/Constans.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/conf/Constans.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.sync.conf;
+package org.apache.iotdb.db.sync.sender.conf;
 
 public class Constans {
 
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/sender/conf/SyncSenderConfig.java
similarity index 98%
rename from server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderConfig.java
rename to server/src/main/java/org/apache/iotdb/db/sync/sender/conf/SyncSenderConfig.java
index fd23b64..572e5df 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderConfig.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/conf/SyncSenderConfig.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.sync.conf;
+package org.apache.iotdb.db.sync.sender.conf;
 
 import java.io.File;
 import org.apache.iotdb.db.conf.IoTDBDescriptor;
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderDescriptor.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/conf/SyncSenderDescriptor.java
similarity index 99%
rename from server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderDescriptor.java
rename to server/src/main/java/org/apache/iotdb/db/sync/sender/conf/SyncSenderDescriptor.java
index 2587e26..2427c10 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/conf/SyncSenderDescriptor.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/conf/SyncSenderDescriptor.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.sync.conf;
+package org.apache.iotdb.db.sync.sender.conf;
 
 import java.io.File;
 import java.io.FileInputStream;
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/IFileManager.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/IFileManager.java
new file mode 100644
index 0000000..b1ba08f
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/manage/IFileManager.java
@@ -0,0 +1,25 @@
+/**
+ * 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.sync.sender.manage;
+
+public interface IFileManager {
+
+
+
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/ISyncSenderLogAnalyzer.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/ISyncSenderLogAnalyzer.java
new file mode 100644
index 0000000..3b33f78
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/ISyncSenderLogAnalyzer.java
@@ -0,0 +1,33 @@
+/**
+ * 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.sync.sender.recover;
+
+import java.util.Set;
+
+public interface ISyncSenderLogAnalyzer {
+
+  void loadLastLocalFiles(Set<String> set);
+
+  void loadLogger(Set<String> set);
+
+  void recover();
+
+  void clearLogger();
+
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/ISyncSenderLogger.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/ISyncSenderLogger.java
new file mode 100644
index 0000000..c516936
--- /dev/null
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/recover/ISyncSenderLogger.java
@@ -0,0 +1,35 @@
+/**
+ * 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.sync.sender.recover;
+
+public interface ISyncSenderLogger {
+
+  void startSyncDeletedFilesName();
+
+  void finishSyncDeletedFileName(String fileName);
+
+  void endSyncDeletedFilsName();
+
+  void startSyncTsFiles();
+
+  void finishSyncTsfile(String fileName);
+
+  void endSyncTsFiles();
+
+}
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncSenderImpl.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/DataTransferManager.java
similarity index 96%
rename from server/src/main/java/org/apache/iotdb/db/sync/sender/SyncSenderImpl.java
rename to server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/DataTransferManager.java
index 88d6705..59dc27e 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncSenderImpl.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/DataTransferManager.java
@@ -15,7 +15,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.sync.sender;
+package org.apache.iotdb.db.sync.sender.transfer;
 
 import java.io.BufferedReader;
 import java.io.ByteArrayOutputStream;
@@ -47,9 +47,10 @@ import org.apache.commons.io.FileUtils;
 import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory;
 import org.apache.iotdb.db.concurrent.ThreadName;
 import org.apache.iotdb.db.exception.SyncConnectionException;
-import org.apache.iotdb.db.sync.conf.Constans;
-import org.apache.iotdb.db.sync.conf.SyncSenderConfig;
-import org.apache.iotdb.db.sync.conf.SyncSenderDescriptor;
+import org.apache.iotdb.db.sync.sender.SyncFileManager;
+import org.apache.iotdb.db.sync.sender.conf.Constans;
+import org.apache.iotdb.db.sync.sender.conf.SyncSenderConfig;
+import org.apache.iotdb.db.sync.sender.conf.SyncSenderDescriptor;
 import org.apache.iotdb.db.utils.SyncUtils;
 import org.apache.iotdb.service.sync.thrift.SyncDataStatus;
 import org.apache.iotdb.service.sync.thrift.SyncService;
@@ -65,9 +66,9 @@ import org.slf4j.LoggerFactory;
 /**
  * SyncSenderImpl is used to transfer tsfiles that needs to sync to receiver.
  */
-public class SyncSenderImpl implements SyncSender {
+public class DataTransferManager implements IDataTransferManager {
 
-  private static final Logger logger = LoggerFactory.getLogger(SyncSenderImpl.class);
+  private static final Logger logger = LoggerFactory.getLogger(DataTransferManager.class);
 
   private TTransport transport;
 
@@ -101,11 +102,11 @@ public class SyncSenderImpl implements SyncSender {
 
   private ScheduledExecutorService executorService;
 
-  private SyncSenderImpl() {
+  private DataTransferManager() {
     init();
   }
 
-  public static final SyncSenderImpl getInstance() {
+  public static final DataTransferManager getInstance() {
     return InstanceHolder.INSTANCE;
   }
 
@@ -116,7 +117,7 @@ public class SyncSenderImpl implements SyncSender {
    */
   public static void main(String[] args) throws IOException {
     Thread.currentThread().setName(ThreadName.SYNC_CLIENT.getName());
-    SyncSenderImpl fileSenderImpl = new SyncSenderImpl();
+    DataTransferManager fileSenderImpl = new DataTransferManager();
     fileSenderImpl.verifySingleton();
     fileSenderImpl.startMonitor();
     fileSenderImpl.startTimedTask();
@@ -524,7 +525,7 @@ public class SyncSenderImpl implements SyncSender {
 
   private static class InstanceHolder {
 
-    private static final SyncSenderImpl INSTANCE = new SyncSenderImpl();
+    private static final DataTransferManager INSTANCE = new DataTransferManager();
   }
 
   public void setConfig(SyncSenderConfig config) {
diff --git a/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncSender.java b/server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/IDataTransferManager.java
similarity index 95%
rename from server/src/main/java/org/apache/iotdb/db/sync/sender/SyncSender.java
rename to server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/IDataTransferManager.java
index d45a09e..3941a90 100644
--- a/server/src/main/java/org/apache/iotdb/db/sync/sender/SyncSender.java
+++ b/server/src/main/java/org/apache/iotdb/db/sync/sender/transfer/IDataTransferManager.java
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.iotdb.db.sync.sender;
+package org.apache.iotdb.db.sync.sender.transfer;
 
 import java.io.IOException;
 import java.util.Set;
@@ -25,7 +25,7 @@ import org.apache.iotdb.db.exception.SyncConnectionException;
 /**
  * SyncSender defines the methods of a sender in sync module.
  */
-public interface SyncSender {
+public interface IDataTransferManager {
 
   /**
    * Init
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/SyncUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/SyncUtils.java
index d395038..5a056a9 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/SyncUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/SyncUtils.java
@@ -23,7 +23,7 @@ import java.text.DecimalFormat;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import org.apache.iotdb.db.sync.conf.SyncSenderDescriptor;
+import org.apache.iotdb.db.sync.sender.conf.SyncSenderDescriptor;
 
 public class SyncUtils {
 
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/sender/SingleClientSyncTest.java b/server/src/test/java/org/apache/iotdb/db/sync/sender/SingleClientSyncTest.java
index 86b82f6..ffb1f2b 100644
--- a/server/src/test/java/org/apache/iotdb/db/sync/sender/SingleClientSyncTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/sync/sender/SingleClientSyncTest.java
@@ -35,9 +35,10 @@ import org.apache.iotdb.db.exception.StartupException;
 import org.apache.iotdb.db.exception.SyncConnectionException;
 import org.apache.iotdb.db.integration.Constant;
 import org.apache.iotdb.db.service.IoTDB;
-import org.apache.iotdb.db.sync.conf.Constans;
-import org.apache.iotdb.db.sync.conf.SyncSenderConfig;
-import org.apache.iotdb.db.sync.conf.SyncSenderDescriptor;
+import org.apache.iotdb.db.sync.sender.conf.Constans;
+import org.apache.iotdb.db.sync.sender.conf.SyncSenderConfig;
+import org.apache.iotdb.db.sync.sender.conf.SyncSenderDescriptor;
+import org.apache.iotdb.db.sync.sender.transfer.DataTransferManager;
 import org.apache.iotdb.db.utils.EnvironmentUtils;
 import org.apache.iotdb.jdbc.Config;
 import org.slf4j.Logger;
@@ -49,7 +50,7 @@ import org.slf4j.LoggerFactory;
  */
 public class SingleClientSyncTest {
 
-  SyncSenderImpl fileSenderImpl = SyncSenderImpl.getInstance();
+  DataTransferManager fileSenderImpl = DataTransferManager.getInstance();
   private IoTDBConfig conf = IoTDBDescriptor.getInstance().getConfig();
   private String serverIpTest = "192.168.130.7";
   private SyncSenderConfig config = SyncSenderDescriptor.getInstance().getConfig();
diff --git a/server/src/test/java/org/apache/iotdb/db/sync/sender/SyncFileManagerTest.java b/server/src/test/java/org/apache/iotdb/db/sync/sender/SyncFileManagerTest.java
index 9398c12..4c8356a 100644
--- a/server/src/test/java/org/apache/iotdb/db/sync/sender/SyncFileManagerTest.java
+++ b/server/src/test/java/org/apache/iotdb/db/sync/sender/SyncFileManagerTest.java
@@ -26,7 +26,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Random;
 import java.util.Set;
-import org.apache.iotdb.db.sync.conf.Constans;
+import org.apache.iotdb.db.sync.sender.conf.Constans;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;