You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2021/07/10 15:36:50 UTC

[GitHub] [incubator-doris] caiconghui opened a new pull request #6200: [Performance] Optimize the performance of tabletReport

caiconghui opened a new pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200


   ## Proposed changes
   1. Use parallelStream to speed up tabletReport.
   2. Add partitionIdInMemorySet to speed up tabletToInMemory check.
   3. Add disable_storage_medium_check to disable storage medium check when user doesn't care what tablet's storage medium is.
   
   ## Types of changes
   
   What types of changes does your code introduce to Doris?
   _Put an `x` in the boxes that apply_
   
   - [ ] Bugfix (non-breaking change which fixes an issue)
   - [ ] New feature (non-breaking change which adds functionality)
   - [ ] Breaking change (fix or feature that would cause existing functionality to not work as expected)
   - [ ] Documentation Update (if none of the other choices apply)
   - [ ] Code refactor (Modify the code structure, format the code, etc...)
   - [x] Optimization. Including functional usability improvements and performance improvements.
   - [ ] Dependency. Such as changes related to third-party components.
   - [ ] Other.
   
   ## Checklist
   
   _Put an `x` in the boxes that apply. You can also fill these out after creating the PR. If you're unsure about any of them, don't hesitate to ask. We're here to help! This is simply a reminder of what we are going to look for before merging your code._
   
   - [ ] I have created an issue on (Fix #ISSUE) and described the bug/feature there in detail
   - [ ] Compiling and unit tests pass locally with my changes
   - [ ] I have added tests that prove my fix is effective or that my feature works
   - [ ] If these changes need document changes, I have updated the document
   - [ ] Any dependent changes have been merged
   
   ## Further comments
   
   If this is a relatively large or complex change, kick off the discussion at dev@doris.apache.org by explaining why you chose the solution you did and what alternatives you considered, etc...
   


-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r671862506



##########
File path: docs/en/administrator-guide/config/fe_config.md
##########
@@ -366,20 +366,16 @@ MasterOnly:true
 
 Whether to enable spark load temporarily, it is not enabled by default
 
-### enable_strict_storage_medium_check
+### disable_storage_medium_check

Review comment:
       Actually, now, this config is a bad design, so I forcibly modified this configuration. For users, it can be divided into two categories, one is concerned about the storage medium, the other is not concerned. if enable_strict_storage_medium_check is true, the behavior is same as that disable_storage_medium_check is false. If  enable_strict_storage_medium_check is false, it would cause continuous migration task failure which is a bug that should be 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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui merged pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui merged pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200


   


-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r670424687



##########
File path: fe/fe-core/src/main/java/org/apache/doris/common/Config.java
##########
@@ -735,9 +735,17 @@
      */
     @ConfField(mutable = true, masterOnly = true)
     public static int max_backend_down_time_second = 3600; // 1h
+
+    /**
+     * If disable_storage_medium_check is true, ReportHandler would not check tablet's storage medium
+     * and disable storage cool down function, the default value is false.
+     * You can set the value true when you don't care what the storage medium of the tablet is.
+     */
+    @ConfField(mutable = true, masterOnly = true)
+    public static boolean disable_storage_medium_check = false;

Review comment:
       enable_strict_storage_medium_check make the user confused, if we really need to check storage_medium, we should not create table without consider storage medium and then produce many migration task usually failed because that user not use this feature and all storage medium are same. If we not care the storage medium, just disable this check, which can save a lot of time for tablet report, because it is meanless to distinguish ssd from hdd for some users while user not need cool down feature.




-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r671866833



##########
File path: docs/en/administrator-guide/config/fe_config.md
##########
@@ -366,20 +366,16 @@ MasterOnly:true
 
 Whether to enable spark load temporarily, it is not enabled by default
 
-### enable_strict_storage_medium_check
+### disable_storage_medium_check

Review comment:
       enable_strict_storage_medium_check is not same as  enable_storage_medium_check, if we don't remove this config, it is still a bug, you could see the comment about this config. and enable_strict_storage_medium_check -> enable_storage_medium_check or disable_storage_medium_check, I think it is all right. for older user, new config would let user know the wrong usage for the storage medium, and for new user, for new user, we have already correct usage




-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r670424687



##########
File path: fe/fe-core/src/main/java/org/apache/doris/common/Config.java
##########
@@ -735,9 +735,17 @@
      */
     @ConfField(mutable = true, masterOnly = true)
     public static int max_backend_down_time_second = 3600; // 1h
+
+    /**
+     * If disable_storage_medium_check is true, ReportHandler would not check tablet's storage medium
+     * and disable storage cool down function, the default value is false.
+     * You can set the value true when you don't care what the storage medium of the tablet is.
+     */
+    @ConfField(mutable = true, masterOnly = true)
+    public static boolean disable_storage_medium_check = false;

Review comment:
       enable_strict_storage_medium_check make the user confused, if we really need to check storage_medium, we should not create table without consider storage medium and then produce many migration task usually failed because that user not use this feature. If we not care the storage medium, just disable this check, which can save a lot of time for tablet report, because it is meanless to distinguish ssd from hdd for some users while user not need cool down feature.




-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r670383438



##########
File path: fe/fe-core/src/main/java/org/apache/doris/common/Config.java
##########
@@ -735,9 +735,17 @@
      */
     @ConfField(mutable = true, masterOnly = true)
     public static int max_backend_down_time_second = 3600; // 1h
+
+    /**
+     * If disable_storage_medium_check is true, ReportHandler would not check tablet's storage medium
+     * and disable storage cool down function, the default value is false.
+     * You can set the value true when you don't care what the storage medium of the tablet is.
+     */
+    @ConfField(mutable = true, masterOnly = true)
+    public static boolean disable_storage_medium_check = false;

Review comment:
       Better not changing the default value.
   And why not using origin config name?

##########
File path: fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java
##########
@@ -484,40 +512,40 @@ public void setNewSchemaHash(long partitionId, long indexId, int newSchemaHash)
         if (Catalog.isCheckpointThread()) {
             return;
         }
-        writeLock();
+        readLock();

Review comment:
       Why change to write lock?

##########
File path: fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java
##########
@@ -17,6 +17,7 @@
 
 package org.apache.doris.master;
 
+import com.google.common.collect.Sets;

Review comment:
       import order

##########
File path: fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java
##########
@@ -660,51 +679,64 @@ private static void deleteFromBackend(Map<Long, TTablet> backendTablets,
         int deleteFromBackendCounter = 0;
         int addToMetaCounter = 0;
         AgentBatchTask batchTask = new AgentBatchTask();
-        for (Long tabletId : backendTablets.keySet()) {
-            TTablet backendTablet = backendTablets.get(tabletId);
-            for (TTabletInfo backendTabletInfo : backendTablet.getTabletInfos()) {
-                boolean needDelete = false;
-                if (!foundTabletsWithValidSchema.contains(tabletId)) {
-                    if (isBackendReplicaHealthy(backendTabletInfo)) {
-                        // if this tablet is not in meta. try adding it.
-                        // if add failed. delete this tablet from backend.
-                        try {
-                            addReplica(tabletId, backendTabletInfo, backendId);
-                            // update counter
-                            needDelete = false;
-                            ++addToMetaCounter;
-                        } catch (MetaNotFoundException e) {
-                            LOG.warn("failed add to meta. tablet[{}], backend[{}]. {}",
-                                    tabletId, backendId, e.getMessage());
-                            needDelete = true;
-                        }
-                    } else {
-                        needDelete = true;
-                    }
-                }
-
-                if (needDelete) {
-                    // drop replica
-                    DropReplicaTask task = new DropReplicaTask(backendId, tabletId, backendTabletInfo.getSchemaHash());
-                    batchTask.addTask(task);
-                    LOG.warn("delete tablet[" + tabletId + " - " + backendTabletInfo.getSchemaHash()
-                            + "] from backend[" + backendId + "] because not found in meta");
-                    ++deleteFromBackendCounter;
-                }
-            } // end for tabletInfos
-
-            if (foundTabletsWithInvalidSchema.containsKey(tabletId)) {
-                // this tablet is found in meta but with invalid schema hash.
-                // delete it.
+        if (foundTabletsWithValidSchema.size() + foundTabletsWithInvalidSchema.size() == backendTablets.size()) {

Review comment:
       Add comment to explain this `if` condition




-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui merged pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui merged pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200


   


-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r670438530



##########
File path: fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java
##########
@@ -660,51 +679,64 @@ private static void deleteFromBackend(Map<Long, TTablet> backendTablets,
         int deleteFromBackendCounter = 0;
         int addToMetaCounter = 0;
         AgentBatchTask batchTask = new AgentBatchTask();
-        for (Long tabletId : backendTablets.keySet()) {
-            TTablet backendTablet = backendTablets.get(tabletId);
-            for (TTabletInfo backendTabletInfo : backendTablet.getTabletInfos()) {
-                boolean needDelete = false;
-                if (!foundTabletsWithValidSchema.contains(tabletId)) {
-                    if (isBackendReplicaHealthy(backendTabletInfo)) {
-                        // if this tablet is not in meta. try adding it.
-                        // if add failed. delete this tablet from backend.
-                        try {
-                            addReplica(tabletId, backendTabletInfo, backendId);
-                            // update counter
-                            needDelete = false;
-                            ++addToMetaCounter;
-                        } catch (MetaNotFoundException e) {
-                            LOG.warn("failed add to meta. tablet[{}], backend[{}]. {}",
-                                    tabletId, backendId, e.getMessage());
-                            needDelete = true;
-                        }
-                    } else {
-                        needDelete = true;
-                    }
-                }
-
-                if (needDelete) {
-                    // drop replica
-                    DropReplicaTask task = new DropReplicaTask(backendId, tabletId, backendTabletInfo.getSchemaHash());
-                    batchTask.addTask(task);
-                    LOG.warn("delete tablet[" + tabletId + " - " + backendTabletInfo.getSchemaHash()
-                            + "] from backend[" + backendId + "] because not found in meta");
-                    ++deleteFromBackendCounter;
-                }
-            } // end for tabletInfos
-
-            if (foundTabletsWithInvalidSchema.containsKey(tabletId)) {
-                // this tablet is found in meta but with invalid schema hash.
-                // delete it.
+        if (foundTabletsWithValidSchema.size() + foundTabletsWithInvalidSchema.size() == backendTablets.size()) {

Review comment:
       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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] acelyc111 commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r671946122



##########
File path: docs/en/administrator-guide/config/fe_config.md
##########
@@ -366,20 +366,16 @@ MasterOnly:true
 
 Whether to enable spark load temporarily, it is not enabled by default
 
-### enable_strict_storage_medium_check
+### disable_storage_medium_check

Review comment:
       Does the bug exist in all cases?




-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r671264720



##########
File path: fe/fe-core/src/main/java/org/apache/doris/master/PartitionInMemoryInfoCollector.java
##########
@@ -0,0 +1,95 @@
+// 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.doris.master;
+
+import org.apache.doris.catalog.Catalog;
+import org.apache.doris.catalog.Database;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Partition;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.catalog.TabletInvertedIndex;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.util.MasterDaemon;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.List;
+
+public class PartitionInMemoryInfoCollector extends MasterDaemon {
+
+    private static final Logger LOG = LogManager.getLogger(PartitionInMemoryInfoCollector.class);
+
+    public PartitionInMemoryInfoCollector() {
+        super("PartitionInMemoryInfoCollector", Config.partition_in_memory_update_interval_secs * 1000);
+    }
+
+    @Override
+    protected void runAfterCatalogReady() {
+        updatePartitionInMemoryInfo();
+    }
+
+    private void updatePartitionInMemoryInfo() {
+        Catalog catalog = Catalog.getCurrentCatalog();

Review comment:
       I think we can just clear the `partitionIdInMemorySet` and set it again .
   So that we don't need to care about the partition's adding and dropping operation.
   That will make things more easy.

##########
File path: fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java
##########
@@ -282,6 +282,14 @@ public void moveFromTempToFormal(long tempPartitionId) {
         }
     }
 
+    public void resetPartitionIdForRestore(long newPartitionId, long oldPartitionId, short restoreReplicationNum) {
+        idToDataProperty.put(newPartitionId, idToDataProperty.remove(oldPartitionId));
+        idToReplicationNum.remove(oldPartitionId);
+        idToReplicationNum.put(newPartitionId, restoreReplicationNum);
+        idToItem.put(newPartitionId, idToItem.remove(oldPartitionId));

Review comment:
       For unpartitioned table, the `idToItem` is empty.




-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] acelyc111 commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r671865186



##########
File path: docs/en/administrator-guide/config/fe_config.md
##########
@@ -366,20 +366,16 @@ MasterOnly:true
 
 Whether to enable spark load temporarily, it is not enabled by default
 
-### enable_strict_storage_medium_check
+### disable_storage_medium_check

Review comment:
       It's not necessary to change the config name when you fix the bug.




-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui commented on pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui commented on pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#issuecomment-877927222


   > How much improvement does this new implementation have?
   
   About seven to ten times improvement for tabletReport of TabletInvertedIndex, and operation outside this function cost less than 1 ms when all tablet are normal and we skip tablet storage medium check.


-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r671866833



##########
File path: docs/en/administrator-guide/config/fe_config.md
##########
@@ -366,20 +366,16 @@ MasterOnly:true
 
 Whether to enable spark load temporarily, it is not enabled by default
 
-### enable_strict_storage_medium_check
+### disable_storage_medium_check

Review comment:
       enable_strict_storage_medium_check is not same as  enable_storage_medium_check, if we don't remove this config, it is still a bug, you could see the comment about this config. and enable_strict_storage_medium_check -> enable_storage_medium_check or disable_storage_medium_check, I think it is all right. for older user, new config would let user know the wrong usage for the storage medium, and for new user, we have correct usage




-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui merged pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui merged pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200


   


-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] acelyc111 commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r671946988



##########
File path: docs/en/administrator-guide/config/fe_config.md
##########
@@ -366,20 +366,16 @@ MasterOnly:true
 
 Whether to enable spark load temporarily, it is not enabled by default
 
-### enable_strict_storage_medium_check
+### disable_storage_medium_check

Review comment:
       BTW, if it's a bug, better to describe it on PR title or description, or add a `bug` lable.




-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r670425732



##########
File path: fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java
##########
@@ -484,40 +512,40 @@ public void setNewSchemaHash(long partitionId, long indexId, int newSchemaHash)
         if (Catalog.isCheckpointThread()) {
             return;
         }
-        writeLock();
+        readLock();

Review comment:
       actually here not modify any structure for TabletInvertedIndex, and when we modify schemahash we hold the tablet meta write lock of tablet meta




-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r671862506



##########
File path: docs/en/administrator-guide/config/fe_config.md
##########
@@ -366,20 +366,16 @@ MasterOnly:true
 
 Whether to enable spark load temporarily, it is not enabled by default
 
-### enable_strict_storage_medium_check
+### disable_storage_medium_check

Review comment:
       Actually, now, this config is a bad design, so I forcibly modified this configuration. For users, it can be divided into two categories, one is concerned about the storage medium, the other is not concerned. if enable_strict_storage_medium_check is true, the behavior is same as that disable_storage_medium_check is false. If  enable_strict_storage_medium_check is false, it may cause continuous migration task failure which is a bug that should be 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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r671954463



##########
File path: docs/en/administrator-guide/config/fe_config.md
##########
@@ -366,20 +366,16 @@ MasterOnly:true
 
 Whether to enable spark load temporarily, it is not enabled by default
 
-### enable_strict_storage_medium_check
+### disable_storage_medium_check

Review comment:
       > Does the bug exist in all cases?
   
   for enable_strict_storage_medium_check=false, on condition that most user cannot be aware this problem, but it may cause unstabitily of fe.




-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
morningman commented on pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#issuecomment-877904367


   How much improvement does this new implementation have?


-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r670432387



##########
File path: fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java
##########
@@ -17,6 +17,7 @@
 
 package org.apache.doris.master;
 
+import com.google.common.collect.Sets;

Review comment:
       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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r671955319



##########
File path: docs/en/administrator-guide/config/fe_config.md
##########
@@ -366,20 +366,16 @@ MasterOnly:true
 
 Whether to enable spark load temporarily, it is not enabled by default
 
-### enable_strict_storage_medium_check
+### disable_storage_medium_check

Review comment:
       > BTW, if it's a bug, better to describe it on PR title or description, or add a `bug` lable.
   
   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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] acelyc111 commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r671860885



##########
File path: docs/en/administrator-guide/config/fe_config.md
##########
@@ -366,20 +366,16 @@ MasterOnly:true
 
 Whether to enable spark load temporarily, it is not enabled by default
 
-### enable_strict_storage_medium_check
+### disable_storage_medium_check

Review comment:
       It's not a good idea to change a config key for compatiable reason. Suppose a user set `enable_strict_storage_medium_check` to true, after he/she upgraded, the old config has no effect and would complain.




-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] acelyc111 commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r671946988



##########
File path: docs/en/administrator-guide/config/fe_config.md
##########
@@ -366,20 +366,16 @@ MasterOnly:true
 
 Whether to enable spark load temporarily, it is not enabled by default
 
-### enable_strict_storage_medium_check
+### disable_storage_medium_check

Review comment:
       further more, if it's a bug, better to describe it on PR title or description, or lable.




-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] acelyc111 commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
acelyc111 commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r671946122



##########
File path: docs/en/administrator-guide/config/fe_config.md
##########
@@ -366,20 +366,16 @@ MasterOnly:true
 
 Whether to enable spark load temporarily, it is not enabled by default
 
-### enable_strict_storage_medium_check
+### disable_storage_medium_check

Review comment:
       Does the bug exits in all cases?




-- 
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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] caiconghui commented on a change in pull request #6200: [Performance] Optimize the performance of tabletReport

Posted by GitBox <gi...@apache.org>.
caiconghui commented on a change in pull request #6200:
URL: https://github.com/apache/incubator-doris/pull/6200#discussion_r671862506



##########
File path: docs/en/administrator-guide/config/fe_config.md
##########
@@ -366,20 +366,16 @@ MasterOnly:true
 
 Whether to enable spark load temporarily, it is not enabled by default
 
-### enable_strict_storage_medium_check
+### disable_storage_medium_check

Review comment:
       Actually, now, this config is a bad design, so I forcibly modified this configuration. For users, it can be divided into two categories, one is concerned about the storage media, the other is not concerned. if enable_strict_storage_medium_check is true, the behavior is same as that disable_storage_medium_check is false. If  enable_strict_storage_medium_check is false, it would cause continuous migration task failure which is a bug that should be 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: commits-unsubscribe@doris.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org