You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@rocketmq.apache.org by GitBox <gi...@apache.org> on 2021/09/15 14:44:02 UTC

[GitHub] [rocketmq] Jason918 opened a new pull request #3357: RIP-7 Multiple Directories Storage Suppor

Jason918 opened a new pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357


   **Make sure set the target branch to `develop`**
   
   ## What is the purpose of the change
   
   Follow up PR of #3348 
   Update according to  comments, see change log
   
   ## Brief changelog
   
   1. remove isMultiCommitlogPathEnable
   2. In org.apache.rocketmq.store.DefaultMessageStore.CleanCommitLogService#isSpaceToDelete, use minPhysicRatio to determin when the broker's disk is full. And mark all store path when it's full.
   3. In org.apache.rocketmq.store.MultiPathMappedFileQueue#tryCreateMappedFile, we ignore readonly paths and  paths marked full in step 2.
   
   ## Verifying this change
   
   Follow this checklist to help us incorporate your contribution quickly and easily. Notice, `it would be helpful if you could finish the following 5 checklist(the last one is not necessary)before request the community to review your PR`.
   
   - [x] Make sure there is a [Github issue](https://github.com/apache/rocketmq/issues) filed for the change (usually before you start working on it). Trivial changes like typos do not require a Github issue. Your pull request should address just this issue, without pulling in other changes - one PR resolves one issue. 
   - [x] Format the pull request title like `[ISSUE #123] Fix UnknownException when host config not exist`. Each commit in the pull request should have a meaningful subject line and body.
   - [x] Write a pull request description that is detailed enough to understand what the pull request does, how, and why.
   - [x] Write necessary unit-test(over 80% coverage) to verify your logic correction, more mock a little better when cross module dependency exist. If the new feature or significant change is committed, please remember to add integration-test in [test module](https://github.com/apache/rocketmq/tree/master/test).
   - [x] Run `mvn -B clean apache-rat:check findbugs:findbugs checkstyle:checkstyle` to make sure basic checks pass. Run `mvn clean install -DskipITs` to make sure unit-test pass. Run `mvn clean test-compile failsafe:integration-test`  to make sure integration-test pass.
   - [ ] If this contribution is large, please file an [Apache Individual Contributor License Agreement](http://www.apache.org/licenses/#clas).
   


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 edited a comment on pull request #3357: RIP-7 Multiple Directories Storage Suppor

Posted by GitBox <gi...@apache.org>.
Jason918 edited a comment on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-921424971


   > DiskCheckScheduledThread1
   
   @francisoliverlee Thank you for the notice.
   Error in `DiskCheckScheduledThread1` is due to  storePathCommitLog usage in "DefaultMessageStore.CleanCommitLogService#isSpaceFull".  I fixed it and added test in last commit.
   Sorry for missing that. I have checked all the usage of `storePathCommitLog`. 


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] yuz10 commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
yuz10 commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r713760173



##########
File path: store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
##########
@@ -144,35 +145,39 @@ void deleteExpiredFile(List<MappedFile> files) {
         }
     }
 
+
     public boolean load() {
         File dir = new File(this.storePath);
-        File[] files = dir.listFiles();
-        if (files != null) {
-            // ascending order
-            Arrays.sort(files);
-            for (File file : files) {
-
-                if (file.length() != this.mappedFileSize) {
-                    log.warn(file + "\t" + file.length()
-                        + " length not matched message store config value, please check it manually");
-                    return false;
-                }
+        File[] ls = dir.listFiles();
+        if (ls != null) {
+            return doLoad(Arrays.asList(ls));
+        }
+        return true;
+    }
 
-                try {
-                    MappedFile mappedFile = new MappedFile(file.getPath(), mappedFileSize);
-
-                    mappedFile.setWrotePosition(this.mappedFileSize);
-                    mappedFile.setFlushedPosition(this.mappedFileSize);
-                    mappedFile.setCommittedPosition(this.mappedFileSize);
-                    this.mappedFiles.add(mappedFile);
-                    log.info("load " + file.getPath() + " OK");
-                } catch (IOException e) {
-                    log.error("load file " + file + " error", e);
-                    return false;
-                }
+    public boolean doLoad(List<File> files) {
+        // ascending order
+        Collections.sort(files);

Review comment:
       According to the code :
   ![image](https://user-images.githubusercontent.com/14816818/134317644-4061cb3f-5470-43bf-bf52-ee974458db10.png)
   if paths are a/, b/ then files will be created such as:
   a/000000   b/000001  a/000002  b/000003
   so sort these can cause wrong order.
   a/000000   a/000002  b/000001  b/000003




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r712892164



##########
File path: store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java
##########
@@ -17,10 +17,14 @@
 package org.apache.rocketmq.store.config;
 
 import java.io.File;
+
 import org.apache.rocketmq.common.annotation.ImportantField;
 import org.apache.rocketmq.store.ConsumeQueue;
 
 public class MessageStoreConfig {
+
+    public static final String MULTI_PATH_SPLITTER = ":";

Review comment:
       YES, it would be confusing. Thank you for the reminding.
   A common reserved character would be a nice fit for this splitter.
   According to[ Comparison of file Systems](https://en.wikipedia.org/wiki/Comparison_of_file_systems), different files system have different reserved characters. So there is no best character for this splitter,  and I am using ',' as default splitter, user can change it using system property `rocketmq.broker.multiPathSplitter`
   




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 commented on pull request #3357: RIP-7 Multiple Directories Storage Suppor

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-921426239


   > AdminBrokerThread error when no any topics
   
   The log shows that the error in AdminBrokerThread is due to the directory for storing CQs  is not created. 
   It's a little bit strange, this PR should have nothing to do with CQs.


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] lollipopjin commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
lollipopjin commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r714427691



##########
File path: store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.rocketmq.store;
+
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.function.Supplier;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.common.UtilAll;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class MultiPathMappedFileQueue extends MappedFileQueue {
+
+    private final MessageStoreConfig config;
+    private final Supplier<Set<String>> fullStorePathsSupplier;
+
+    public MultiPathMappedFileQueue(MessageStoreConfig messageStoreConfig, int mappedFileSize,
+                                    AllocateMappedFileService allocateMappedFileService,
+                                    Supplier<Set<String>> fullStorePathsSupplier) {
+        super(messageStoreConfig.getStorePathCommitLog(), mappedFileSize, allocateMappedFileService);
+        this.config = messageStoreConfig;
+        this.fullStorePathsSupplier = fullStorePathsSupplier;
+    }
+
+    private Set<String> getPaths() {
+        String[] paths = config.getStorePathCommitLog().trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+        return new HashSet<>(Arrays.asList(paths));
+    }
+
+    private Set<String> getReadonlyPaths() {
+        String pathStr = config.getReadOnlyCommitLogStorePaths();
+        if (StringUtils.isBlank(pathStr)) {
+            return Collections.emptySet();
+        }
+        String[] paths = pathStr.trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+        return new HashSet<>(Arrays.asList(paths));
+    }
+
+    @Override
+    public boolean load() {
+        Set<String> storePathSet = getPaths();
+        storePathSet.addAll(getReadonlyPaths());
+
+        List<File> files = new ArrayList<>();
+        for (String path : storePathSet) {
+            File dir = new File(path);
+            File[] ls = dir.listFiles();
+            if (ls != null) {
+                Collections.addAll(files, ls);
+            }
+        }
+
+        return doLoad(files);
+    }
+
+    @Override
+    protected MappedFile tryCreateMappedFile(long createOffset) {
+        long fileIdx = createOffset / this.mappedFileSize;
+        Set<String> storePath = getPaths();
+        Set<String> readonlyPathSet = getReadonlyPaths();
+        Set<String> fullStorePaths =
+                fullStorePathsSupplier == null ? Collections.emptySet() : fullStorePathsSupplier.get();
+
+
+        HashSet<String> availableStorePath = new HashSet<>(storePath);
+        //do not create file in readonly store path.
+        availableStorePath.removeAll(readonlyPathSet);
+
+        //do not create file is space is nearly full.
+        availableStorePath.removeAll(fullStorePaths);
+
+        //if no store path left, fall back to wriable store path.

Review comment:
       wriable --> writable




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] caigy commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Suppor

Posted by GitBox <gi...@apache.org>.
caigy commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r709701419



##########
File path: broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java
##########
@@ -635,8 +636,16 @@ public SocketAddress getStoreHost() {
     }
 
     private String diskUtil() {
-        String storePathPhysic = this.brokerController.getMessageStoreConfig().getStorePathCommitLog();
-        double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathPhysic);
+        double physicRatio = -1;
+        String storePath =this.brokerController.getMessageStoreConfig().getStorePathCommitLog();
+        if (storePath.contains(MessageStoreConfig.MULTI_PATH_SPLITTER)) {
+            String[] paths = storePath.trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+            for (String storePathPhysic : paths) {
+                physicRatio = Math.min(physicRatio, UtilAll.getDiskPartitionSpaceUsedPercent(storePathPhysic));

Review comment:
       Is `Math.min()` correct here? `physicRatio` is initialized as `-1` and `UtilAll.getDiskPartitionSpaceUsedPercent()` always returns value greater or equal to `-1`.

##########
File path: store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
##########
@@ -782,10 +783,20 @@ private String getStorePathPhysic() {
     public HashMap<String, String> getRuntimeInfo() {
         HashMap<String, String> result = this.storeStatsService.getRuntimeInfo();
 
-        {
-            double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(getStorePathPhysic());
+        String commitLogStorePath = DefaultMessageStore.this.getMessageStoreConfig().getStorePathCommitLog();
+        if (commitLogStorePath.contains(MessageStoreConfig.MULTI_PATH_SPLITTER)) {
+            double maxValue = Double.MIN_VALUE;

Review comment:
       Is `maxPhysicsUsedRatio` more appropriate?




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] yuz10 commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
yuz10 commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r713696805



##########
File path: broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java
##########
@@ -635,8 +636,16 @@ public SocketAddress getStoreHost() {
     }
 
     private String diskUtil() {
-        String storePathPhysic = this.brokerController.getMessageStoreConfig().getStorePathCommitLog();
-        double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathPhysic);
+        double physicRatio = 100;
+        String storePath = this.brokerController.getMessageStoreConfig().getStorePathCommitLog();
+        if (storePath.contains(MessageStoreConfig.MULTI_PATH_SPLITTER)) {
+            String[] paths = storePath.trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+            for (String storePathPhysic : paths) {
+                physicRatio = Math.min(physicRatio, UtilAll.getDiskPartitionSpaceUsedPercent(storePathPhysic));
+            }
+        } else {
+            physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePath);

Review comment:
       this if branch is not necessary, could use the same code with MULTI_PATH branch




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 commented on pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-925529140


   > If we add a cloud disk to the running ecs, can the new cloud disk be recognized by updating broker.conf dynamically?
   
   Yes, It should work. You can add path after `storePathCommitLog`. 
   But, one thing to be noticed, if your server is started with only one commit log path, you have to restart the server to change it to multi-path-commit-log mode. Because `mappedFileQueue` in class `Commitlog` is initialized during start up.
   One way to avoid this restart is to add a `MessageStoreConfig#MULTI_PATH_SPLITTER`(default is ',' and can be changed by system property "rocketmq.broker.multiPathSplitter") after origin single storePathCommitLog, e.g. "~/store/commitlog,".


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] yuz10 commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
yuz10 commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r713713618



##########
File path: store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
##########
@@ -782,16 +783,28 @@ private String getStorePathPhysic() {
     public HashMap<String, String> getRuntimeInfo() {
         HashMap<String, String> result = this.storeStatsService.getRuntimeInfo();
 
-        {
-            double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(getStorePathPhysic());
+        String commitLogStorePath = DefaultMessageStore.this.getMessageStoreConfig().getStorePathCommitLog();

Review comment:
       Dose it support dledger mode? the original code getStorePathPhysic() has different value of  commitLogStorePath in dledger mode




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 commented on pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-1050455472


   > `readOnlyCommitLogStorePaths`: I have a question to ask. What is the meaning of this configuration, is it used for smooth upgrade? @Jason918
   
   Yep, it can be used to decommission some broken disks.


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] francisoliverlee commented on pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
francisoliverlee commented on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-924522139


   > > AdminBrokerThread error when no any topics
   > 
   > @francisoliverlee
   > The log shows that the error in AdminBrokerThread is due to the directory for storing CQs is not created.
   > It's a little bit strange, this PR should have nothing to do with CQs.
   > I would try to reproduce it.
   
   checked and works OK 👍


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] francisoliverlee edited a comment on pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
francisoliverlee edited a comment on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-924529808


   > > > DiskCheckScheduledThread1
   > > 
   > > 
   > > @francisoliverlee Thank you for the notice.
   > > Error in `DiskCheckScheduledThread1` is due to storePathCommitLog usage in "DefaultMessageStore.CleanCommitLogService#isSpaceFull". I fixed it and added test in last commit.
   > > Sorry for missing that. I have checked all the usage of `storePathCommitLog`.
   > 
   > It seems this error still exists ,and maybe you discarded this method _org.apache.rocketmq.common.UtilAll#getDiskPartitionSpaceUsedPercent_, or should we check the existence of a directory at the start of the broker?
   
   error stills. 
   ![image](https://user-images.githubusercontent.com/5908412/134273780-29068ec0-9537-425b-91c8-009f7db8093f.png)
   
   when doing getDiskPartitionSpaceUsedPercent(),  how about a WARNING for NO consume queue dir ?
   @Jason918 @duhenglucky 


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] duhenglucky commented on pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
duhenglucky commented on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-923128542


   > > DiskCheckScheduledThread1
   > 
   > @francisoliverlee Thank you for the notice.
   > Error in `DiskCheckScheduledThread1` is due to storePathCommitLog usage in "DefaultMessageStore.CleanCommitLogService#isSpaceFull". I fixed it and added test in last commit.
   > Sorry for missing that. I have checked all the usage of `storePathCommitLog`.
   
   Maybe you discarded this method _org.apache.rocketmq.common.UtilAll#getDiskPartitionSpaceUsedPercent_, or should we check the existence of a directory at the start of the broker?


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] yuz10 commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
yuz10 commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r713760173



##########
File path: store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
##########
@@ -144,35 +145,39 @@ void deleteExpiredFile(List<MappedFile> files) {
         }
     }
 
+
     public boolean load() {
         File dir = new File(this.storePath);
-        File[] files = dir.listFiles();
-        if (files != null) {
-            // ascending order
-            Arrays.sort(files);
-            for (File file : files) {
-
-                if (file.length() != this.mappedFileSize) {
-                    log.warn(file + "\t" + file.length()
-                        + " length not matched message store config value, please check it manually");
-                    return false;
-                }
+        File[] ls = dir.listFiles();
+        if (ls != null) {
+            return doLoad(Arrays.asList(ls));
+        }
+        return true;
+    }
 
-                try {
-                    MappedFile mappedFile = new MappedFile(file.getPath(), mappedFileSize);
-
-                    mappedFile.setWrotePosition(this.mappedFileSize);
-                    mappedFile.setFlushedPosition(this.mappedFileSize);
-                    mappedFile.setCommittedPosition(this.mappedFileSize);
-                    this.mappedFiles.add(mappedFile);
-                    log.info("load " + file.getPath() + " OK");
-                } catch (IOException e) {
-                    log.error("load file " + file + " error", e);
-                    return false;
-                }
+    public boolean doLoad(List<File> files) {
+        // ascending order
+        Collections.sort(files);

Review comment:
       According to the code :
   ![image](https://user-images.githubusercontent.com/14816818/134317644-4061cb3f-5470-43bf-bf52-ee974458db10.png)
   if paths are a/, b/ then files will be created such as:
   a/000000   b/000001  a/000002  b/000003
   so sort these can cause wrong order.
   a/000000   a/000002  b/000001  b/000003
   
   this is test case:
   ![image](https://user-images.githubusercontent.com/14816818/134319811-7d986dcf-0ffa-42bb-8aae-796421d86ad4.png)
   




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 commented on pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-923844209


   > LGTM~, but only one disk is written at the same time, which does not take advantage of the increased write bandwidth after multi-disk deployment.
   
   After this multi-disk deployment, read bandwidth will be fully utilized. 
   In current one big commit log writing pattern, writing concurrently requires much more work. Maybe soft raid is more suitable for now, if write bandwidth is the physical bottleneck. 


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] caigy commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Suppor

Posted by GitBox <gi...@apache.org>.
caigy commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r710674452



##########
File path: store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
##########
@@ -782,10 +783,20 @@ private String getStorePathPhysic() {
     public HashMap<String, String> getRuntimeInfo() {
         HashMap<String, String> result = this.storeStatsService.getRuntimeInfo();
 
-        {
-            double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(getStorePathPhysic());
+        String commitLogStorePath = DefaultMessageStore.this.getMessageStoreConfig().getStorePathCommitLog();
+        if (commitLogStorePath.contains(MessageStoreConfig.MULTI_PATH_SPLITTER)) {
+            double maxValue = Double.MIN_VALUE;

Review comment:
       @Jason918 I agree. Using `minPhysicsUsedRatio` is more suitable for this scenario. 




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 edited a comment on pull request #3357: RIP-7 Multiple Directories Storage Suppor

Posted by GitBox <gi...@apache.org>.
Jason918 edited a comment on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-921424971


   > DiskCheckScheduledThread1
   @francisoliverlee Thank you for the notice.
   Error in `DiskCheckScheduledThread1` is due to  storePathCommitLog usage in "DefaultMessageStore.CleanCommitLogService#isSpaceFull".  I fixed it and added test in last commit.
   Sorry for missing that. I have checked all the usage of `storePathCommitLog`. 


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] francisoliverlee commented on pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
francisoliverlee commented on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-924529808


   > > > DiskCheckScheduledThread1
   > > 
   > > 
   > > @francisoliverlee Thank you for the notice.
   > > Error in `DiskCheckScheduledThread1` is due to storePathCommitLog usage in "DefaultMessageStore.CleanCommitLogService#isSpaceFull". I fixed it and added test in last commit.
   > > Sorry for missing that. I have checked all the usage of `storePathCommitLog`.
   > 
   > It seems this error still exists ,and maybe you discarded this method _org.apache.rocketmq.common.UtilAll#getDiskPartitionSpaceUsedPercent_, or should we check the existence of a directory at the start of the broker?
   
   when doing getDiskPartitionSpaceUsedPercent(),  how about a WARNING for NO consume queue dir ?
   @Jason918 @duhenglucky 


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] yuz10 commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
yuz10 commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r713760173



##########
File path: store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
##########
@@ -144,35 +145,39 @@ void deleteExpiredFile(List<MappedFile> files) {
         }
     }
 
+
     public boolean load() {
         File dir = new File(this.storePath);
-        File[] files = dir.listFiles();
-        if (files != null) {
-            // ascending order
-            Arrays.sort(files);
-            for (File file : files) {
-
-                if (file.length() != this.mappedFileSize) {
-                    log.warn(file + "\t" + file.length()
-                        + " length not matched message store config value, please check it manually");
-                    return false;
-                }
+        File[] ls = dir.listFiles();
+        if (ls != null) {
+            return doLoad(Arrays.asList(ls));
+        }
+        return true;
+    }
 
-                try {
-                    MappedFile mappedFile = new MappedFile(file.getPath(), mappedFileSize);
-
-                    mappedFile.setWrotePosition(this.mappedFileSize);
-                    mappedFile.setFlushedPosition(this.mappedFileSize);
-                    mappedFile.setCommittedPosition(this.mappedFileSize);
-                    this.mappedFiles.add(mappedFile);
-                    log.info("load " + file.getPath() + " OK");
-                } catch (IOException e) {
-                    log.error("load file " + file + " error", e);
-                    return false;
-                }
+    public boolean doLoad(List<File> files) {
+        // ascending order
+        Collections.sort(files);

Review comment:
       According to the code :
   ![image](https://user-images.githubusercontent.com/14816818/134317644-4061cb3f-5470-43bf-bf52-ee974458db10.png)
   if paths are a/, b/ then files will be created such as:
   a/000000   b/000001  a/000002  b/000003
   so sort these can cause wrong order.




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r714489880



##########
File path: store/src/main/java/org/apache/rocketmq/store/MultiPathMappedFileQueue.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.rocketmq.store;
+
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.function.Supplier;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.rocketmq.common.UtilAll;
+import org.apache.rocketmq.store.config.MessageStoreConfig;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class MultiPathMappedFileQueue extends MappedFileQueue {
+
+    private final MessageStoreConfig config;
+    private final Supplier<Set<String>> fullStorePathsSupplier;
+
+    public MultiPathMappedFileQueue(MessageStoreConfig messageStoreConfig, int mappedFileSize,
+                                    AllocateMappedFileService allocateMappedFileService,
+                                    Supplier<Set<String>> fullStorePathsSupplier) {
+        super(messageStoreConfig.getStorePathCommitLog(), mappedFileSize, allocateMappedFileService);
+        this.config = messageStoreConfig;
+        this.fullStorePathsSupplier = fullStorePathsSupplier;
+    }
+
+    private Set<String> getPaths() {
+        String[] paths = config.getStorePathCommitLog().trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+        return new HashSet<>(Arrays.asList(paths));
+    }
+
+    private Set<String> getReadonlyPaths() {
+        String pathStr = config.getReadOnlyCommitLogStorePaths();
+        if (StringUtils.isBlank(pathStr)) {
+            return Collections.emptySet();
+        }
+        String[] paths = pathStr.trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+        return new HashSet<>(Arrays.asList(paths));
+    }
+
+    @Override
+    public boolean load() {
+        Set<String> storePathSet = getPaths();
+        storePathSet.addAll(getReadonlyPaths());
+
+        List<File> files = new ArrayList<>();
+        for (String path : storePathSet) {
+            File dir = new File(path);
+            File[] ls = dir.listFiles();
+            if (ls != null) {
+                Collections.addAll(files, ls);
+            }
+        }
+
+        return doLoad(files);
+    }
+
+    @Override
+    protected MappedFile tryCreateMappedFile(long createOffset) {
+        long fileIdx = createOffset / this.mappedFileSize;
+        Set<String> storePath = getPaths();
+        Set<String> readonlyPathSet = getReadonlyPaths();
+        Set<String> fullStorePaths =
+                fullStorePathsSupplier == null ? Collections.emptySet() : fullStorePathsSupplier.get();
+
+
+        HashSet<String> availableStorePath = new HashSet<>(storePath);
+        //do not create file in readonly store path.
+        availableStorePath.removeAll(readonlyPathSet);
+
+        //do not create file is space is nearly full.
+        availableStorePath.removeAll(fullStorePaths);
+
+        //if no store path left, fall back to wriable store path.

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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r713842297



##########
File path: store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
##########
@@ -782,16 +783,28 @@ private String getStorePathPhysic() {
     public HashMap<String, String> getRuntimeInfo() {
         HashMap<String, String> result = this.storeStatsService.getRuntimeInfo();
 
-        {
-            double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(getStorePathPhysic());
+        String commitLogStorePath = DefaultMessageStore.this.getMessageStoreConfig().getStorePathCommitLog();

Review comment:
       Thank you for the reminding. 
   Changed to `getStorePathPhysic()`




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r713840154



##########
File path: broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java
##########
@@ -635,8 +636,16 @@ public SocketAddress getStoreHost() {
     }
 
     private String diskUtil() {
-        String storePathPhysic = this.brokerController.getMessageStoreConfig().getStorePathCommitLog();
-        double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathPhysic);
+        double physicRatio = 100;
+        String storePath = this.brokerController.getMessageStoreConfig().getStorePathCommitLog();
+        if (storePath.contains(MessageStoreConfig.MULTI_PATH_SPLITTER)) {
+            String[] paths = storePath.trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+            for (String storePathPhysic : paths) {
+                physicRatio = Math.min(physicRatio, UtilAll.getDiskPartitionSpaceUsedPercent(storePathPhysic));
+            }
+        } else {
+            physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePath);

Review comment:
       updated.

##########
File path: store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
##########
@@ -1650,25 +1663,49 @@ private boolean isSpaceToDelete() {
             cleanImmediately = false;
 
             {
-                double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(getStorePathPhysic());
-                if (physicRatio > diskSpaceWarningLevelRatio) {
+                String[] storePaths;
+                String commitLogStorePath = DefaultMessageStore.this.getMessageStoreConfig().getStorePathCommitLog();
+                if (commitLogStorePath.contains(MessageStoreConfig.MULTI_PATH_SPLITTER)) {
+                    storePaths = commitLogStorePath.trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+                } else {
+                    storePaths = new String[]{commitLogStorePath};

Review comment:
       updated.




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] francisoliverlee merged pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
francisoliverlee merged pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357


   


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] duhenglucky edited a comment on pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
duhenglucky edited a comment on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-923128542


   > > DiskCheckScheduledThread1
   > 
   > @francisoliverlee Thank you for the notice.
   > Error in `DiskCheckScheduledThread1` is due to storePathCommitLog usage in "DefaultMessageStore.CleanCommitLogService#isSpaceFull". I fixed it and added test in last commit.
   > Sorry for missing that. I have checked all the usage of `storePathCommitLog`.
   
   It seems this error still exists ,and maybe you discarded this method _org.apache.rocketmq.common.UtilAll#getDiskPartitionSpaceUsedPercent_, or should we check the existence of a directory at the start of the broker?


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] duhenglucky commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
duhenglucky commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r712308669



##########
File path: store/src/main/java/org/apache/rocketmq/store/config/MessageStoreConfig.java
##########
@@ -17,10 +17,14 @@
 package org.apache.rocketmq.store.config;
 
 import java.io.File;
+
 import org.apache.rocketmq.common.annotation.ImportantField;
 import org.apache.rocketmq.store.ConsumeQueue;
 
 public class MessageStoreConfig {
+
+    public static final String MULTI_PATH_SPLITTER = ":";

Review comment:
       IMHO, ":"  can be confusing sometimes, especially on Windows, where the storage path contains C: XXX, eg.




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] yuz10 commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
yuz10 commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r713720169



##########
File path: store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
##########
@@ -1650,25 +1663,49 @@ private boolean isSpaceToDelete() {
             cleanImmediately = false;
 
             {
-                double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(getStorePathPhysic());
-                if (physicRatio > diskSpaceWarningLevelRatio) {
+                String[] storePaths;
+                String commitLogStorePath = DefaultMessageStore.this.getMessageStoreConfig().getStorePathCommitLog();
+                if (commitLogStorePath.contains(MessageStoreConfig.MULTI_PATH_SPLITTER)) {
+                    storePaths = commitLogStorePath.trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+                } else {
+                    storePaths = new String[]{commitLogStorePath};

Review comment:
       not necessary if branch, could use the same code with MULTI_PATH branch




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r713856997



##########
File path: store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java
##########
@@ -144,35 +145,39 @@ void deleteExpiredFile(List<MappedFile> files) {
         }
     }
 
+
     public boolean load() {
         File dir = new File(this.storePath);
-        File[] files = dir.listFiles();
-        if (files != null) {
-            // ascending order
-            Arrays.sort(files);
-            for (File file : files) {
-
-                if (file.length() != this.mappedFileSize) {
-                    log.warn(file + "\t" + file.length()
-                        + " length not matched message store config value, please check it manually");
-                    return false;
-                }
+        File[] ls = dir.listFiles();
+        if (ls != null) {
+            return doLoad(Arrays.asList(ls));
+        }
+        return true;
+    }
 
-                try {
-                    MappedFile mappedFile = new MappedFile(file.getPath(), mappedFileSize);
-
-                    mappedFile.setWrotePosition(this.mappedFileSize);
-                    mappedFile.setFlushedPosition(this.mappedFileSize);
-                    mappedFile.setCommittedPosition(this.mappedFileSize);
-                    this.mappedFiles.add(mappedFile);
-                    log.info("load " + file.getPath() + " OK");
-                } catch (IOException e) {
-                    log.error("load file " + file + " error", e);
-                    return false;
-                }
+    public boolean doLoad(List<File> files) {
+        // ascending order
+        Collections.sort(files);

Review comment:
       Great point.
   Changed `Collections.sort(files)` to `files.sort(Comparator.comparing(File::getName))`,  sort the file using simple filenames, aka, the last part of the path.
   
   And added test in org.apache.rocketmq.store.MultiPathMappedFileQueueTest#testLoadReadOnlyMappedFiles, PTAL.




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] ni-ze commented on pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
ni-ze commented on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-924074682


   > LGTM~, but only one disk is written at the same time, which does not take advantage of the increased write bandwidth after multi-disk deployment.
   
   yes, it is. Appending messages write last MappedFile only, other files can not be written at all.


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] lollipopjin commented on pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
lollipopjin commented on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-925463652


   If we add a cloud disk to the running ecs, can the new cloud disk be recognized by updating broker.conf dynamically?


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] francisoliverlee removed a comment on pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
francisoliverlee removed a comment on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-924522139


   > > AdminBrokerThread error when no any topics
   > 
   > @francisoliverlee
   > The log shows that the error in AdminBrokerThread is due to the directory for storing CQs is not created.
   > It's a little bit strange, this PR should have nothing to do with CQs.
   > I would try to reproduce it.
   
   checked and works OK 👍


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] yuz10 commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
yuz10 commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r713713618



##########
File path: store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
##########
@@ -782,16 +783,28 @@ private String getStorePathPhysic() {
     public HashMap<String, String> getRuntimeInfo() {
         HashMap<String, String> result = this.storeStatsService.getRuntimeInfo();
 
-        {
-            double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(getStorePathPhysic());
+        String commitLogStorePath = DefaultMessageStore.this.getMessageStoreConfig().getStorePathCommitLog();

Review comment:
       Dose it support dledge mode? the original code getStorePathPhysic() has different value of  commitLogStorePath in dledge mode




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] cserwen commented on pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
cserwen commented on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-1049547693


   `readOnlyCommitLogStorePaths`: I have a question to ask. What is the meaning of this configuration, is it used for smooth upgrade? @Jason918 


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 commented on pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-923832494


   > It seems this error still exists ,and maybe you discarded this method _org.apache.rocketmq.common.UtilAll#getDiskPartitionSpaceUsedPercent_, or should we check the existence of a directory at the start of the broker?
   
   yes, for now, user have to put existing file paths in `storePathCommitLog`.
   
   I think auto creation is better, as we can update it the online.


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 commented on pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-922244792


   @francisoliverlee Added path empty check to avoid error log. Checked with `bin/mqadmin brokerStatus  -n 127.1:9876 -b 127.1:10911`, no error logs appears in broker.log.
   Please take a look.
   


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 commented on pull request #3357: RIP-7 Multiple Directories Storage Suppor

Posted by GitBox <gi...@apache.org>.
Jason918 commented on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-921424971


   > DiskCheckScheduledThread1
   
   Error in `DiskCheckScheduledThread1` is due to  storePathCommitLog usage in "DefaultMessageStore.CleanCommitLogService#isSpaceFull".  I fixed it and added test in last commit.
   Sorry for missing that. I have checked all the usage of `storePathCommitLog`. 


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 edited a comment on pull request #3357: RIP-7 Multiple Directories Storage Suppor

Posted by GitBox <gi...@apache.org>.
Jason918 edited a comment on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-921426239


   > AdminBrokerThread error when no any topics
   
   @francisoliverlee 
   The log shows that the error in AdminBrokerThread is due to the directory for storing CQs  is not created. 
   It's a little bit strange, this PR should have nothing to do with CQs.
   I would try to reproduce it. 


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Suppor

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r709717805



##########
File path: store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java
##########
@@ -782,10 +783,20 @@ private String getStorePathPhysic() {
     public HashMap<String, String> getRuntimeInfo() {
         HashMap<String, String> result = this.storeStatsService.getRuntimeInfo();
 
-        {
-            double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(getStorePathPhysic());
+        String commitLogStorePath = DefaultMessageStore.this.getMessageStoreConfig().getStorePathCommitLog();
+        if (commitLogStorePath.contains(MessageStoreConfig.MULTI_PATH_SPLITTER)) {
+            double maxValue = Double.MIN_VALUE;

Review comment:
       @caigy Yes, `maxPhysicsUsedRatio` is better .
   But after some consideration, I think we should use `minPhysicsUsedRatio`. It's more useful for admins, because we use min value of all store path to determine whether disk is full and we should clean disk immediately.




-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 edited a comment on pull request #3357: RIP-7 Multiple Directories Storage Support

Posted by GitBox <gi...@apache.org>.
Jason918 edited a comment on pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#issuecomment-923832494


   > It seems this error still exists ,and maybe you discarded this method _org.apache.rocketmq.common.UtilAll#getDiskPartitionSpaceUsedPercent_, or should we check the existence of a directory at the start of the broker?
   
   Yes, for now, user have to put existing file paths in `storePathCommitLog`.
   
   Added isPathExists check in 'DiskCheckScheduledThread'. 
   
   PTAL


-- 
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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [rocketmq] Jason918 commented on a change in pull request #3357: RIP-7 Multiple Directories Storage Suppor

Posted by GitBox <gi...@apache.org>.
Jason918 commented on a change in pull request #3357:
URL: https://github.com/apache/rocketmq/pull/3357#discussion_r709716591



##########
File path: broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java
##########
@@ -635,8 +636,16 @@ public SocketAddress getStoreHost() {
     }
 
     private String diskUtil() {
-        String storePathPhysic = this.brokerController.getMessageStoreConfig().getStorePathCommitLog();
-        double physicRatio = UtilAll.getDiskPartitionSpaceUsedPercent(storePathPhysic);
+        double physicRatio = -1;
+        String storePath =this.brokerController.getMessageStoreConfig().getStorePathCommitLog();
+        if (storePath.contains(MessageStoreConfig.MULTI_PATH_SPLITTER)) {
+            String[] paths = storePath.trim().split(MessageStoreConfig.MULTI_PATH_SPLITTER);
+            for (String storePathPhysic : paths) {
+                physicRatio = Math.min(physicRatio, UtilAll.getDiskPartitionSpaceUsedPercent(storePathPhysic));

Review comment:
       @caigy Yes, you are right. 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: dev-unsubscribe@rocketmq.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org