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

[incubator-iotdb] 01/01: Merge branch 'cluster_new' into cluster_data_snapshot

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

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

commit 9a8ff35ed414ff537d0376352b740fb1ddf26a3f
Merge: d9e49c1 3215776
Author: jt2594838 <jt...@163.com>
AuthorDate: Tue Apr 14 17:07:47 2020 +0800

    Merge branch 'cluster_new' into cluster_data_snapshot
    
    # Conflicts:
    #	cluster/src/test/java/org/apache/iotdb/cluster/common/TestRemoteFileSnapshot.java

 .travis.yml                                        |  32 +-
 Jenkinsfile                                        |   2 +-
 LICENSE                                            |   8 -
 README.md                                          |   9 +-
 README_ZH.md                                       |   8 +-
 cluster/pom.xml                                    |   4 +
 .../resources/conf/iotdb-cluster.properties        |   3 +
 cluster/src/assembly/resources/sbin/start-node.bat |  17 +-
 cluster/src/assembly/resources/sbin/start-node.sh  |  21 +-
 .../java/org/apache/iotdb/cluster/ClusterMain.java |   9 +-
 .../apache/iotdb/cluster/config/ClusterConfig.java |  12 +
 .../iotdb/cluster/config/ClusterDescriptor.java    |  98 +++-
 .../cluster/exception/MemberReadOnlyException.java |  19 +
 .../java/org/apache/iotdb/cluster/log/Log.java     |   1 +
 .../org/apache/iotdb/cluster/log/LogManager.java   |   1 -
 .../iotdb/cluster/log/logtypes/AddNodeLog.java     |   1 +
 .../cluster/log/logtypes/PhysicalPlanLog.java      |   1 +
 .../iotdb/cluster/log/logtypes/RemoveNodeLog.java  |  19 +
 .../iotdb/cluster/log/manage/DiskLogManager.java   | 144 +++++
 .../manage/FilePartitionedSnapshotLogManager.java  |   3 +-
 .../iotdb/cluster/log/manage/MemoryLogManager.java |  65 ++-
 .../log/manage/MetaSingleSnapshotLogManager.java   |   4 +-
 .../manage/serializable/LogDequeSerializer.java    |  79 +++
 .../log/manage/serializable/LogManagerMeta.java    | 111 ++++
 .../serializable/SyncLogDequeSerializer.java       | 482 ++++++++++++++++
 .../iotdb/cluster/partition/NodeRemovalResult.java |  19 +
 .../iotdb/cluster/partition/PartitionTable.java    |  13 +-
 .../cluster/query/ClusterAggregateExecutor.java    |  19 +
 .../cluster/query/ClusterConcatPathOptimizer.java  |  19 +
 .../iotdb/cluster/query/ClusterFillExecutor.java   |  19 +
 .../iotdb/cluster/query/ClusterPlanExecutor.java   |  18 +-
 .../cluster/query/filter/SlotTsFileFilter.java     |  19 +
 .../groupby/ClusterGroupByNoVFilterDataSet.java    |  19 +
 .../groupby/ClusterGroupByVFilterDataSet.java      |  19 +
 .../query/groupby/MergeGroupByExecutor.java        |  19 +
 .../query/groupby/RemoteGroupByExecutor.java       |  19 +
 .../iotdb/cluster/query/reader/EmptyReader.java    |  19 +
 .../cluster/query/reader/ManagedMergeReader.java   |  19 +
 .../cluster/query/reader/MergedReaderByTime.java   |  19 +
 .../iotdb/cluster/server/DataClusterServer.java    |   4 +
 .../apache/iotdb/cluster/server/NodeReport.java    |  19 +
 .../cluster/server/member/MetaGroupMember.java     |  89 ++-
 .../apache/iotdb/cluster/utils/PartitionUtils.java |  17 +-
 .../iotdb/cluster/utils/nodetool/NodeTool.java     |   6 +-
 .../cluster/utils/nodetool/function/LogView.java   |  65 +++
 .../org/apache/iotdb/cluster/common/TestUtils.java |  15 +
 .../cluster/log/manage/DiskLogManagerTest.java     | 131 +++++
 .../serializable/SyncLogDequeSerializerTest.java   | 584 +++++++++++++++++++
 .../cluster/partition/SlotPartitionTableTest.java  |  21 +-
 .../query/ClusterAggregateExecutorTest.java        |  19 +
 .../cluster/query/ClusterFillExecutorTest.java     |  19 +
 .../cluster/query/ClusterPlanExecutorTest.java     |  20 +-
 .../ClusterGroupByNoVFilterDataSetTest.java        |  19 +
 .../groupby/ClusterGroupByVFilterDataSetTest.java  |  19 +
 .../query/groupby/MergeGroupByExecutorTest.java    |  19 +
 .../query/groupby/RemoteGroupByExecutorTest.java   |  19 +
 distribution/pom.xml                               |  13 +
 docs/Community/ASF.md                              |  33 --
 docs/Community/Community-Powered By.md             |  13 -
 docs/Community/Community-Project Committers.md     |   2 -
 docs/Community/Wiki.md                             |  30 -
 docs/Download/README.md                            |   2 -
 docs/README.md                                     |   8 +-
 .../4-StorageEngine/6-DataManipulation.md          |  17 +-
 docs/UserGuide/0-Get Started/1-QuickStart.md       |   9 +-
 docs/UserGuide/3-Server/2-Single Node Setup.md     |  10 +-
 docs/zh/Community/ASF.md                           |  33 --
 docs/zh/Community/Community-Powered By.md          |  32 +-
 docs/zh/Community/Community-Project Committers.md  |   2 -
 docs/zh/Community/Wiki.md                          |  30 -
 docs/zh/Download/README.md                         |   2 -
 docs/zh/README.md                                  |   4 +-
 .../4-StorageEngine/6-DataManipulation.md          |  16 +-
 docs/zh/UserGuide/0-Get Started/1-QuickStart.md    |   8 +-
 docs/zh/UserGuide/3-Server/2-Single Node Setup.md  |  11 +-
 docs/zh/UserGuide/4-Client/6-Programming - MQTT.md | 101 ++++
 .../{6-Status Codes.md => 7-Status Codes.md}       |   0
 .../main/java/org/apache/iotdb/SessionExample.java |  16 +-
 .../apache/iotdb/tsfile/TsFileSequenceRead.java    |   4 +
 .../apache/iotdb/hadoop/fileSystem/HDFSFile.java   |   2 +-
 pom.xml                                            |  21 +-
 server/pom.xml                                     |  14 +
 .../resources/conf/iotdb-engine.properties         |   4 +-
 .../java/io/moquette/broker/MQTTConnection.java    | 503 ----------------
 .../iotdb/db/concurrent/WrappedRunnable.java       |  29 +-
 .../java/org/apache/iotdb/db/conf/IoTDBConfig.java |   4 +-
 .../iotdb/db/cost/statistic/Measurement.java       |   9 +-
 .../apache/iotdb/db/engine/flush/FlushManager.java |   6 +-
 .../iotdb/db/engine/flush/MemTableFlushTask.java   |   2 +-
 .../iotdb/db/engine/merge/task/MergeFileTask.java  |   6 +-
 .../db/engine/merge/task/MergeMultiChunkTask.java  |   3 +-
 .../engine/storagegroup/StorageGroupProcessor.java |   3 +-
 .../db/engine/storagegroup/TsFileResource.java     |   2 +-
 .../iotdb/db/engine/upgrade/UpgradeTask.java       |   5 +-
 .../org/apache/iotdb/db/monitor/StatMonitor.java   |   5 +-
 .../apache/iotdb/db/qp/executor/PlanExecutor.java  |  18 +-
 .../apache/iotdb/db/qp/physical/PhysicalPlan.java  |  23 +-
 .../iotdb/db/qp/physical/crud/DeletePlan.java      |  15 +-
 .../iotdb/db/qp/physical/sys/DataAuthPlan.java     |  54 +-
 .../iotdb/db/qp/physical/sys/SetTTLPlan.java       |  29 +
 .../db/query/dataset/NonAlignEngineDataSet.java    |   5 +-
 .../dataset/RawQueryDataSetWithoutValueFilter.java |   5 +-
 .../iotdb/db/query/executor/FillQueryExecutor.java |  12 +-
 .../java/org/apache/iotdb/db/query/fill/IFill.java |  19 +-
 .../org/apache/iotdb/db/query/fill/LinearFill.java |  27 +-
 .../apache/iotdb/db/query/fill/PreviousFill.java   | 222 +++++++-
 .../db/query/reader/chunk/MemChunkReader.java      |   1 -
 .../org/apache/iotdb/db/service/MQTTService.java   |   1 +
 .../apache/iotdb/db/service/MetricsService.java    |   5 +-
 .../db/tools/memestimation/MemEstToolCmd.java      |   5 +-
 .../writelog/recover/TsFileRecoverPerformer.java   |   1 +
 .../db/concurrent/IoTDBThreadPoolFactoryTest.java  |   4 +-
 .../conf/adapter/ActiveTimeSeriesCounterTest.java  |   9 +-
 .../engine/storagegroup/TsFileProcessorTest.java   |   3 +-
 .../iotdb/db/integration/IoTDBAlignByDeviceIT.java |   2 +-
 .../apache/iotdb/db/integration/IoTDBFillIT.java   | 316 ++++++++++-
 .../iotdb/db/integration/IoTDBRestartIT.java       |  58 ++
 .../apache/iotdb/db/qp/plan/PhysicalPlanTest.java  |  65 ++-
 .../query/reader/series/SeriesReaderTestUtil.java  |   4 +-
 site/pom.xml                                       |   4 +-
 site/src/main/.vuepress/components/FooterFixed.vue |  50 --
 site/src/main/.vuepress/config.js                  | 631 +++++++++++----------
 site/src/main/.vuepress/enhanceApp.js              |   6 +-
 site/src/main/.vuepress/public/css/index.css       |  27 -
 .../public/img/contributor-avatar/user.svg         |   2 +-
 .../.vuepress/{override.styl => styles/index.styl} |   9 +-
 site/src/main/.vuepress/styles/palette.styl        |   7 +-
 .../theme/components/AlgoliaSearchBox.vue          | 172 ++++++
 .../.vuepress/theme/components/DropdownLink.vue    | 196 +++++++
 .../theme/components/DropdownTransition.vue        |  50 ++
 site/src/main/.vuepress/theme/components/Home.vue  | 179 ++++++
 .../main/.vuepress/theme/components/NavLink.vue    |  65 +++
 .../main/.vuepress/theme/components/NavLinks.vue   | 166 ++++++
 .../src/main/.vuepress/theme/components/Navbar.vue | 139 +++++
 site/src/main/.vuepress/theme/components/Page.vue  | 278 +++++++++
 .../main/.vuepress/theme/components/Sidebar.vue    |  76 +++
 .../.vuepress/theme/components/SidebarButton.vue   |  44 ++
 .../.vuepress/theme/components/SidebarGroup.vue    | 146 +++++
 .../.vuepress/theme/components/SidebarLink.vue     | 126 ++++
 .../.vuepress/theme/components/SidebarLinks.vue    | 103 ++++
 .../.vuepress/theme/global-components/Badge.vue    |  61 ++
 .../global-components}/Contributor.vue             |   0
 .../Home.vue => theme/global-components/IoTDB.vue} |   2 +-
 .../global-components/IoTDBZH.vue}                 |   2 +-
 site/src/main/.vuepress/theme/index.js             |  63 ++
 site/src/main/.vuepress/theme/layouts/404.vue      |  43 ++
 site/src/main/.vuepress/theme/layouts/Layout.vue   | 166 ++++++
 .../{override.styl => theme/noopModule.js}         |   4 +-
 .../{override.styl => theme/styles/arrow.styl}     |  23 +-
 site/src/main/.vuepress/theme/styles/code.styl     | 166 ++++++
 .../styles/custom-blocks.styl}                     |  44 +-
 site/src/main/.vuepress/theme/styles/index.styl    | 227 ++++++++
 .../{enhanceApp.js => theme/styles/mobile.styl}    |  51 +-
 .../{styles/palette.styl => theme/styles/toc.styl} |   6 +-
 .../palette.styl => theme/styles/wrapper.styl}     |  10 +-
 site/src/main/.vuepress/theme/util/index.js        | 256 +++++++++
 site/src/main/package.json                         |  21 +-
 tsfile/format-changelist.md                        |   3 +-
 .../org/apache/iotdb/tsfile/file/MetaMarker.java   |   1 +
 .../iotdb/tsfile/read/TsFileSequenceReader.java    |  42 +-
 .../apache/iotdb/tsfile/read/common/BatchData.java |  12 +
 .../iotdb/tsfile/utils/ReadWriteIOUtils.java       |   9 +
 .../apache/iotdb/tsfile/utils/VersionUtils.java    |  34 +-
 .../apache/iotdb/tsfile/write/TsFileWriter.java    |  10 +-
 .../write/writer/RestorableTsFileIOWriter.java     |   6 +-
 .../iotdb/tsfile/write/writer/TsFileIOWriter.java  |  16 +-
 .../tsfile/read/TsFileSequenceReaderTest.java      |   3 +
 .../iotdb/tsfile/write/TsFileIOWriterTest.java     |   6 +-
 .../iotdb/tsfile/write/TsFileWriterTest.java       |   2 +-
 .../write/writer/RestorableTsFileIOWriterTest.java |   6 +-
 170 files changed, 6718 insertions(+), 1467 deletions(-)

diff --cc cluster/src/main/java/org/apache/iotdb/cluster/log/manage/MemoryLogManager.java
index 2dd1261,40e69ba..acee173
--- a/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/MemoryLogManager.java
+++ b/cluster/src/main/java/org/apache/iotdb/cluster/log/manage/MemoryLogManager.java
@@@ -162,18 -185,23 +185,28 @@@ public abstract class MemoryLogManager 
      return logApplier;
    }
  
-   @Override
-   public void setLastLogId(long lastLogId) {
-     this.lastLogId = lastLogId;
+   public void removeFromHead(int length) {
+     logBuffer.subList(0, length).clear();
    }
  
-   @Override
-   public void setLastLogTerm(long lastLogTerm) {
-     this.lastLogTerm = lastLogTerm;
+   @TestOnly
+   public LogManagerMeta getMeta() {
+     LogManagerMeta managerMeta = new LogManagerMeta();
+     managerMeta.setCommitLogIndex(commitLogIndex);
+     managerMeta.setLastLogId(lastLogId);
+     managerMeta.setLastLogTerm(lastLogTerm);
+     return managerMeta;
+   }
+ 
+   @TestOnly
+   public void setMeta(LogManagerMeta meta) {
+     commitLogIndex = meta.getCommitLogIndex();
+     lastLogId = meta.getLastLogId();
+     lastLogTerm = meta.getLastLogTerm();
    }
 +
 +  @Override
 +  public void setCommitIndex(long commitIndex) {
 +    this.commitLogIndex = commitIndex;
 +  }
  }