This is an automated email from the ASF dual-hosted git repository. yiguolei pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/master by this push: new 6757d2f361e Revert "[Enhancement](show-backends-disks) Add show backends disks (#24229)" (#25389) 6757d2f361e is described below commit 6757d2f361ed5ae36c0a46c805d2c468ccaa5d1e Author: yiguolei <676222...@qq.com> AuthorDate: Fri Oct 13 14:08:45 2023 +0800 Revert "[Enhancement](show-backends-disks) Add show backends disks (#24229)" (#25389) This reverts commit 21223e65c59c23cfcb9e8ab610ea321168bcb75a. --- be/src/agent/task_worker_pool.cpp | 21 +----- be/src/agent/task_worker_pool.h | 2 - be/src/olap/olap_common.h | 9 --- be/src/olap/special_dir.cpp | 67 -------------------- be/src/olap/special_dir.h | 57 ----------------- be/src/olap/storage_engine.cpp | 22 +------ be/src/olap/storage_engine.h | 5 -- .../Show-Statements/SHOW-BACKENDS-DISKS.md | 66 ------------------- .../Show-Statements/SHOW-FRONTENDS-DISKS.md | 2 +- .../Show-Statements/SHOW-BACKENDS-DISKS.md | 66 ------------------- .../Show-Statements/SHOW-FRONTENDS-DISKS.md | 2 +- fe/fe-core/src/main/cup/sql_parser.cup | 4 -- .../apache/doris/analysis/ShowBackendsStmt.java | 24 +------ .../java/org/apache/doris/catalog/DiskInfo.java | 13 +--- .../apache/doris/common/proc/BackendsProcDir.java | 74 ---------------------- .../java/org/apache/doris/qe/ShowExecutor.java | 2 +- .../main/java/org/apache/doris/system/Backend.java | 50 ++++----------- .../org/apache/doris/system/SystemInfoService.java | 8 +-- .../java/org/apache/doris/catalog/BackendTest.java | 4 -- .../org/apache/doris/clone/DecommissionTest.java | 3 - .../doris/clone/TabletRepairAndBalanceTest.java | 3 - .../doris/clone/TabletReplicaTooSlowTest.java | 3 - .../doris/common/util/AutoBucketUtilsTest.java | 2 - .../apache/doris/planner/ResourceTagQueryTest.java | 3 - .../doris/utframe/DemoMultiBackendsTest.java | 3 - gensrc/thrift/MasterService.thrift | 1 - gensrc/thrift/Types.thrift | 6 -- 27 files changed, 20 insertions(+), 502 deletions(-) diff --git a/be/src/agent/task_worker_pool.cpp b/be/src/agent/task_worker_pool.cpp index 898fa584ba7..d80a49bdbb6 100644 --- a/be/src/agent/task_worker_pool.cpp +++ b/be/src/agent/task_worker_pool.cpp @@ -20,6 +20,7 @@ #include <fmt/format.h> #include <gen_cpp/AgentService_types.h> #include <gen_cpp/HeartbeatService_types.h> +#include <gen_cpp/MasterService_types.h> #include <gen_cpp/Status_types.h> #include <gen_cpp/Types_types.h> #include <unistd.h> @@ -54,7 +55,6 @@ #include "olap/olap_common.h" #include "olap/rowset/rowset_meta.h" #include "olap/snapshot_manager.h" -#include "olap/special_dir.h" #include "olap/storage_engine.h" #include "olap/storage_policy.h" #include "olap/tablet.h" @@ -677,13 +677,8 @@ void TaskWorkerPool::_report_disk_state_worker_thread_callback() { disk.__set_disk_available_capacity(root_path_info.available); disk.__set_trash_used_capacity(root_path_info.trash_used_capacity); disk.__set_used(root_path_info.is_used); - disk.__set_dir_type(TDiskType::STORAGE); request.disks[root_path_info.path] = disk; } - - _set_disk_infos(request, TDiskType::LOG); - _set_disk_infos(request, TDiskType::DEPLOY); - request.__set_num_cores(CpuInfo::num_cores()); request.__set_pipeline_executor_size(config::pipeline_executor_size > 0 ? config::pipeline_executor_size @@ -1101,20 +1096,6 @@ void TaskWorkerPool::_handle_report(const TReportRequest& request, ReportType ty } } -void TaskWorkerPool::_set_disk_infos(TReportRequest& request, TDiskType::type type) { - SpecialDirInfo dir_info; - StorageEngine::instance()->get_special_dir_info(&dir_info, type); - - TDisk special_disk; - special_disk.__set_root_path(dir_info.path); - special_disk.__set_data_used_capacity(0); - special_disk.__set_disk_total_capacity(dir_info.capacity); - special_disk.__set_disk_available_capacity(dir_info.available); - special_disk.__set_used(dir_info.is_used); - special_disk.__set_dir_type(type); - request.disks[dir_info.path] = special_disk; -} - void TaskWorkerPool::_random_sleep(int second) { Random rnd(UnixMillis()); sleep(rnd.Uniform(second) + 1); diff --git a/be/src/agent/task_worker_pool.h b/be/src/agent/task_worker_pool.h index ca90edac51d..50c8842166f 100644 --- a/be/src/agent/task_worker_pool.h +++ b/be/src/agent/task_worker_pool.h @@ -19,7 +19,6 @@ #include <butil/macros.h> #include <gen_cpp/AgentService_types.h> -#include <gen_cpp/MasterService_types.h> #include <gen_cpp/Types_types.h> #include <stdint.h> @@ -212,7 +211,6 @@ protected: TFinishTaskRequest* finish_task_request); void _handle_report(const TReportRequest& request, ReportType type); - void _set_disk_infos(TReportRequest& request, TDiskType::type type); Status _get_tablet_info(const TTabletId tablet_id, const TSchemaHash schema_hash, int64_t signature, TTabletInfo* tablet_info); diff --git a/be/src/olap/olap_common.h b/be/src/olap/olap_common.h index e5ca13bec99..130d65e7ef4 100644 --- a/be/src/olap/olap_common.h +++ b/be/src/olap/olap_common.h @@ -61,20 +61,11 @@ struct DataDirInfo { bool is_used = false; // whether available mark TStorageMedium::type storage_medium = TStorageMedium::HDD; // Storage medium type: SSD|HDD }; - -struct SpecialDirInfo { - std::string path; - int64_t capacity = 1; // actual disk capacity - int64_t available = 0; // available space, in bytes unit - bool is_used = false; -}; - struct PredicateFilterInfo { int type = 0; uint64_t input_row = 0; uint64_t filtered_row = 0; }; - // Sort DataDirInfo by available space. struct DataDirInfoLessAvailability { bool operator()(const DataDirInfo& left, const DataDirInfo& right) const { diff --git a/be/src/olap/special_dir.cpp b/be/src/olap/special_dir.cpp deleted file mode 100644 index 8e8e0384763..00000000000 --- a/be/src/olap/special_dir.cpp +++ /dev/null @@ -1,67 +0,0 @@ -// 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. - -#include "olap/special_dir.h" - -#include <string> - -#include "io/fs/file_reader_writer_fwd.h" -#include "io/fs/file_writer.h" -#include "io/fs/local_file_system.h" -#include "io/fs/path.h" -#include "olap/utils.h" // for check_dir_existed - -namespace doris { -using namespace ErrorCode; - -static const char* const kTestFilePath = ".testfile"; - -SpecialDir::SpecialDir(const std::string& path) - : _path(path), - _fs(io::LocalFileSystem::create(path)), - _available_bytes(0), - _capacity_bytes(0), - _is_used(true) {} - -SpecialDir::~SpecialDir() = default; - -Status SpecialDir::update_capacity() { - RETURN_IF_ERROR(io::global_local_filesystem()->get_space_info(_path, &_capacity_bytes, - &_available_bytes)); - LOG(INFO) << "path: " << _path << " total capacity: " << _capacity_bytes - << ", available capacity: " << _available_bytes; - - return Status::OK(); -} - -void SpecialDir::health_check() { - _is_used = true; - // check disk - Status res = _read_and_write_test_file(); - if (!res) { - LOG(WARNING) << "log read/write test file occur IO Error. path=" << _path - << ", err: " << res; - _is_used = !res.is<IO_ERROR>(); - } -} - -Status SpecialDir::_read_and_write_test_file() { - auto test_file = fmt::format("{}/{}", _path, kTestFilePath); - return read_write_test_file(test_file); -} - -} // namespace doris diff --git a/be/src/olap/special_dir.h b/be/src/olap/special_dir.h deleted file mode 100644 index 4768a6bf66a..00000000000 --- a/be/src/olap/special_dir.h +++ /dev/null @@ -1,57 +0,0 @@ -// 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. - -#pragma once - -#include <string> - -#include "common/status.h" -#include "io/fs/file_system.h" -#include "olap/olap_common.h" - -namespace doris { - -class SpecialDir { -public: - SpecialDir(const std::string& path); - ~SpecialDir(); - - void get_dir_info(SpecialDirInfo* special_dir_info) { - special_dir_info->path = _path; - special_dir_info->capacity = _capacity_bytes; - special_dir_info->available = _available_bytes; - special_dir_info->is_used = _is_used; - return; - } - - Status update_capacity(); - - void health_check(); - -private: - Status _read_and_write_test_file(); - - std::string _path; - io::FileSystemSPtr _fs; - // the actual available capacity of the disk of this data dir - size_t _available_bytes; - // the actual capacity of the disk of this data dir - size_t _capacity_bytes; - bool _is_used; -}; - -} // namespace doris diff --git a/be/src/olap/storage_engine.cpp b/be/src/olap/storage_engine.cpp index 062a5c9d322..f17a6de8414 100644 --- a/be/src/olap/storage_engine.cpp +++ b/be/src/olap/storage_engine.cpp @@ -125,9 +125,7 @@ StorageEngine::StorageEngine(const EngineOptions& options) _calc_delete_bitmap_executor(nullptr), _default_rowset_type(BETA_ROWSET), _heartbeat_flags(nullptr), - _stream_load_recorder(nullptr), - _log_dir(new SpecialDir(config::sys_log_dir)), - _deploy_dir(new SpecialDir(std::string(std::getenv("DORIS_HOME")))) { + _stream_load_recorder(nullptr) { REGISTER_HOOK_METRIC(unused_rowsets_count, [this]() { // std::lock_guard<std::mutex> lock(_gc_mutex); return _unused_rowsets.size(); @@ -362,24 +360,6 @@ Status StorageEngine::get_all_data_dir_info(std::vector<DataDirInfo>* data_dir_i return res; } -void StorageEngine::get_special_dir_info(SpecialDirInfo* special_dir_infos, TDiskType::type type) { - switch (type) { - case TDiskType::LOG: - _log_dir->health_check(); - static_cast<void>(_log_dir->update_capacity()); - _log_dir->get_dir_info(special_dir_infos); - break; - case TDiskType::DEPLOY: - _deploy_dir->health_check(); - static_cast<void>(_deploy_dir->update_capacity()); - _deploy_dir->get_dir_info(special_dir_infos); - break; - default: - break; - } - return; -} - int64_t StorageEngine::get_file_or_directory_size(const std::string& file_path) { if (!std::filesystem::exists(file_path)) { return 0; diff --git a/be/src/olap/storage_engine.h b/be/src/olap/storage_engine.h index aafd5951a2e..6017354ef46 100644 --- a/be/src/olap/storage_engine.h +++ b/be/src/olap/storage_engine.h @@ -45,7 +45,6 @@ #include "olap/rowset/rowset.h" #include "olap/rowset/rowset_id_generator.h" #include "olap/rowset/segment_v2/segment.h" -#include "olap/special_dir.h" #include "olap/tablet.h" #include "olap/task/index_builder.h" #include "runtime/exec_env.h" @@ -55,7 +54,6 @@ namespace doris { class DataDir; -class SpecialDir; class EngineTask; class MemTableFlushExecutor; class SegcompactionWorker; @@ -103,7 +101,6 @@ public: // get all info of root_path Status get_all_data_dir_info(std::vector<DataDirInfo>* data_dir_infos, bool need_update); - void get_special_dir_info(SpecialDirInfo* dir_infos, TDiskType::type type); int64_t get_file_or_directory_size(const std::string& file_path); @@ -474,8 +471,6 @@ private: std::condition_variable _compaction_producer_sleep_cv; std::shared_ptr<StreamLoadRecorder> _stream_load_recorder; - std::unique_ptr<SpecialDir> _log_dir; - std::unique_ptr<SpecialDir> _deploy_dir; // we use unordered_map to store all cumulative compaction policy sharded ptr std::unordered_map<std::string_view, std::shared_ptr<CumulativeCompactionPolicy>> diff --git a/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-BACKENDS-DISKS.md b/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-BACKENDS-DISKS.md deleted file mode 100644 index 0d8b8514215..00000000000 --- a/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-BACKENDS-DISKS.md +++ /dev/null @@ -1,66 +0,0 @@ ---- -{ - "title": "SHOW-BACKENDS-DISKS", - "language": "en" -} ---- - -<!-- -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. ---> - -## SHOW-BACKENDS-DISKS - -### Name - -SHOW BACKENDS DISKS - -### Description - - This statement is used to query disk information corresponding to data directory of BE node. - - 语法: - -```sql -SHOW BACKENDS DISKS; -``` - -说明: -1. Name indicates id of BE node. -2. Host indicates ip of BE node. -3. RootPath indicates data directory of BE node. -4. DirType indicates the type of directory -5. DiskState indicates state of disk. -6. TotalCapacity indicates total capacity of the disk. -7. UsedCapacity indicates used space of the disk. -8. AvailableCapacity indicates available space of the disk. -9. UsedPct indicates percentage of the disk used. - -### Example -` -mysql> show backends disks; -+-----------+-------------+------------------------------+---------+----------+---------------+-------------+-------------------+---------+ -| BackendId | Host | RootPath | DirType | DiskState| TotalCapacity | UsedCapacity| AvailableCapacity | UsedPct | -+-----------+-------------+------------------------------+---------+----------+---------------+-------------+-------------------+---------+ -| 10002 | 10.xx.xx.90 | /home/work/output/be/storage | STORAGE | ONLINE | 7.049 TB | 2.478 TB | 4.571 TB | 35.16 % | -| 10002 | 10.xx.xx.90 | /home/work/output/be | DEPLOY | ONLINE | 7.049 TB | 2.478 TB | 4.571 TB | 35.16 % | -| 10002 | 10.xx.xx.90 | /home/work/output/be/log | LOG | ONLINE | 7.049 TB | 2.478 TB | 4.571 TB | 35.16 % | -+-----------+-------------+------------------------------+---------+----------+---------------+-------------+-------------------+---------+ -2 rows in set (0.00 sec) -` -### Keywords - - SHOW, BACKENDS, DISK, DISKS \ No newline at end of file diff --git a/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-FRONTENDS-DISKS.md b/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-FRONTENDS-DISKS.md index 2180b99c952..fad9e33dd44 100644 --- a/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-FRONTENDS-DISKS.md +++ b/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-FRONTENDS-DISKS.md @@ -70,7 +70,7 @@ mysql> show frontends disks; ### Keywords - SHOW, FRONTENDS, DISK, DISKS + SHOW, FRONTENDS ### Best Practice diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-BACKENDS-DISKS.md b/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-BACKENDS-DISKS.md deleted file mode 100644 index dcb35ef42e0..00000000000 --- a/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-BACKENDS-DISKS.md +++ /dev/null @@ -1,66 +0,0 @@ ---- -{ - "title": "SHOW-BACKENDS-DISKS", - "language": "zh-CN" -} ---- - -<!-- -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. ---> - -## SHOW-BACKENDS-DISKS - -### Name - -SHOW BACKENDS DISKS - -### Description - - 该语句用于查看 BE 节点的数据目录对应的磁盘信息。 - - 语法: - -```sql -SHOW BACKENDS DISKS; -``` - -说明: -1. Name 表示该 BE 节点的 ID。 -2. Host 表示该 BE 节点的 IP。 -3. RootPath 表示该 BE 节点的数据目录。 -4. DirType 表示目录类型 -5. DiskState 表示磁盘状态。 -6. TotalCapacity 表示数据目录对应磁盘的总容量。 -7. UsedCapacity 表示磁盘的已使用空间。 -8. AvailableCapacity 表示磁盘的可使用空间。 -9. UsedPct 表示磁盘的使用百分比。 - -### Example -` -mysql> show backends disks; -+-----------+-------------+------------------------------+---------+----------+---------------+-------------+-------------------+---------+ -| BackendId | Host | RootPath | DirType | DiskState| TotalCapacity | UsedCapacity| AvailableCapacity | UsedPct | -+-----------+-------------+------------------------------+---------+----------+---------------+-------------+-------------------+---------+ -| 10002 | 10.xx.xx.90 | /home/work/output/be/storage | STORAGE | ONLINE | 7.049 TB | 2.478 TB | 4.571 TB | 35.16 % | -| 10002 | 10.xx.xx.90 | /home/work/output/be | DEPLOY | ONLINE | 7.049 TB | 2.478 TB | 4.571 TB | 35.16 % | -| 10002 | 10.xx.xx.90 | /home/work/output/be/log | LOG | ONLINE | 7.049 TB | 2.478 TB | 4.571 TB | 35.16 % | -+-----------+-------------+------------------------------+---------+----------+---------------+-------------+-------------------+---------+ -2 rows in set (0.00 sec) -` -### Keywords - - SHOW, BACKENDS, DISK, DISKS \ No newline at end of file diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-FRONTENDS-DISKS.md b/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-FRONTENDS-DISKS.md index 4531688464b..3570ee8302e 100644 --- a/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-FRONTENDS-DISKS.md +++ b/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-FRONTENDS-DISKS.md @@ -69,7 +69,7 @@ mysql> show frontends disks; ### Keywords - SHOW, FRONTENDS, DISK, DISKS + SHOW, FRONTENDS ### Best Practice diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup index e2d7a2ca5d5..5675e11b495 100644 --- a/fe/fe-core/src/main/cup/sql_parser.cup +++ b/fe/fe-core/src/main/cup/sql_parser.cup @@ -4150,10 +4150,6 @@ show_param ::= {: RESULT = new ShowBackendsStmt(); :} - | KW_BACKENDS ident:name - {: - RESULT = new ShowBackendsStmt(name); - :} | KW_TRASH KW_ON STRING_LITERAL:backend {: RESULT = new ShowTrashDiskStmt(backend); diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackendsStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackendsStmt.java index ce1ca8d26b6..46009bd1c09 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackendsStmt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowBackendsStmt.java @@ -20,7 +20,6 @@ package org.apache.doris.analysis; import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.ScalarType; -import org.apache.doris.common.AnalysisException; import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; @@ -29,22 +28,11 @@ import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.ShowResultSetMetaData; -import com.google.common.collect.ImmutableList; - public class ShowBackendsStmt extends ShowStmt { - private String type; public ShowBackendsStmt() { } - public ShowBackendsStmt(String type) { - this.type = type; - } - - public String getType() { - return type; - } - @Override public void analyze(Analyzer analyzer) throws UserException { super.analyze(analyzer); @@ -54,22 +42,12 @@ public class ShowBackendsStmt extends ShowStmt { PrivPredicate.OPERATOR)) { ErrorReport.reportAnalysisException(ErrorCode.ERR_SPECIFIC_ACCESS_DENIED_ERROR, "ADMIN/OPERATOR"); } - - if (type != null && !type.equals("disks")) { - throw new AnalysisException("Show backends with extra info only support show backends disks"); - } } @Override public ShowResultSetMetaData getMetaData() { ShowResultSetMetaData.Builder builder = ShowResultSetMetaData.builder(); - - ImmutableList<String> titles = BackendsProcDir.TITLE_NAMES; - if (type != null && type.equals("disks")) { - titles = BackendsProcDir.DISK_TITLE_NAMES; - } - - for (String title : titles) { + for (String title : BackendsProcDir.TITLE_NAMES) { builder.addColumn(new Column(title, ScalarType.createVarchar(30))); } return builder.build(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/DiskInfo.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/DiskInfo.java index c54afda8345..b49acb2ff83 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/DiskInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/DiskInfo.java @@ -55,8 +55,6 @@ public class DiskInfo implements Writable { private long diskAvailableCapacityB; @SerializedName("state") private DiskState state; - @SerializedName("dirType") - private String dirType; // path hash and storage medium are reported from Backend and no need to persist private long pathHash = 0; private TStorageMedium storageMedium; @@ -72,7 +70,6 @@ public class DiskInfo implements Writable { this.trashUsedCapacityB = 0; this.diskAvailableCapacityB = DEFAULT_CAPACITY_B; this.state = DiskState.ONLINE; - this.dirType = "STORAGE"; this.pathHash = 0; this.storageMedium = TStorageMedium.HDD; } @@ -133,10 +130,6 @@ public class DiskInfo implements Writable { return state; } - public String getDirType() { - return dirType; - } - // return true if changed public boolean setState(DiskState state) { if (this.state != state) { @@ -146,10 +139,6 @@ public class DiskInfo implements Writable { return false; } - public void setDirType(String dirType) { - this.dirType = dirType; - } - public long getPathHash() { return pathHash; } @@ -196,7 +185,7 @@ public class DiskInfo implements Writable { return "DiskInfo [rootPath=" + rootPath + "(" + pathHash + "), totalCapacityB=" + totalCapacityB + ", dataUsedCapacityB=" + dataUsedCapacityB + ", trashUsedCapacityB=" + trashUsedCapacityB + ", diskAvailableCapacityB=" + diskAvailableCapacityB + ", state=" + state - + ", dirType=" + dirType + ", medium: " + storageMedium + "]"; + + ", medium: " + storageMedium + "]"; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java index 25d6e312cbd..2acf93d1651 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java @@ -17,7 +17,6 @@ package org.apache.doris.common.proc; -import org.apache.doris.catalog.DiskInfo; import org.apache.doris.catalog.Env; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Pair; @@ -31,7 +30,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Stopwatch; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.gson.Gson; import org.apache.logging.log4j.LogManager; @@ -80,17 +78,6 @@ public class BackendsProcDir implements ProcDirInterface { return result; } - public static List<List<String>> getBackendInfos(String type) { - List<List<String>> backendInfos = new LinkedList<>(); - - if (type == null) { - backendInfos = getBackendInfos(); - } else if (type.equals("disks")) { - backendInfos = getBackendsDiskInfos(); - } - return backendInfos; - } - /** * get backends info * @@ -201,67 +188,6 @@ public class BackendsProcDir implements ProcDirInterface { return backendInfos; } - /** - * get backends disk info - * - * @return - */ - public static List<List<String>> getBackendsDiskInfos() { - final SystemInfoService systemInfoService = Env.getCurrentSystemInfo(); - List<List<String>> backendsDiskInfos = new LinkedList<>(); - List<Long> backendIds = systemInfoService.getAllBackendIds(false); - if (backendIds == null) { - return backendsDiskInfos; - } - - List<List<Comparable>> comparableBackendsDiskInfos = new LinkedList<>(); - for (long backendId : backendIds) { - Backend backend = systemInfoService.getBackend(backendId); - if (backend == null) { - continue; - } - - ImmutableMap<String, DiskInfo> disksRef = backend.getAllDisks(); - for (DiskInfo disk : disksRef.values()) { - List<Comparable> backendsDiskInfo = Lists.newArrayList(); - backendsDiskInfo.add(String.valueOf(backendId)); - backendsDiskInfo.add(backend.getHost()); - // add disk info to backendsDiskInfo - backendsDiskInfo.add(disk.getRootPath()); - backendsDiskInfo.add(disk.getDirType()); - backendsDiskInfo.add(disk.getState()); - long totalCapacityB = disk.getTotalCapacityB(); - Pair<Double, String> totalCapacity = DebugUtil.getByteUint(totalCapacityB); - backendsDiskInfo.add(DebugUtil.DECIMAL_FORMAT_SCALE_3.format( - totalCapacity.first) + " " + totalCapacity.second); - long diskUsedCapacityB = disk.getDiskUsedCapacityB(); - Pair<Double, String> diskUsedCapacity = DebugUtil.getByteUint(diskUsedCapacityB); - backendsDiskInfo.add(DebugUtil.DECIMAL_FORMAT_SCALE_3.format( - diskUsedCapacity.first) + " " + diskUsedCapacity.second); - long availableCapacityB = disk.getAvailableCapacityB(); - Pair<Double, String> availableCapacity = DebugUtil.getByteUint(availableCapacityB); - backendsDiskInfo.add(DebugUtil.DECIMAL_FORMAT_SCALE_3.format( - availableCapacity.first) + " " + availableCapacity.second); - backendsDiskInfo.add(String.format("%.2f", disk.getUsedPct() * 100) + " %"); - comparableBackendsDiskInfos.add(backendsDiskInfo); - } - } - - // sort by host name - ListComparator<List<Comparable>> comparator = new ListComparator<List<Comparable>>(1); - comparableBackendsDiskInfos.sort(comparator); - - for (List<Comparable> backendsDiskInfo : comparableBackendsDiskInfos) { - List<String> oneInfo = new ArrayList<String>(backendsDiskInfo.size()); - for (Comparable element : backendsDiskInfo) { - oneInfo.add(element.toString()); - } - backendsDiskInfos.add(oneInfo); - } - - return backendsDiskInfos; - } - @Override public boolean register(String name, ProcNodeInterface node) { return false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java index f66d76eca8b..04644c0c920 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java @@ -1964,7 +1964,7 @@ public class ShowExecutor { private void handleShowBackends() { final ShowBackendsStmt showStmt = (ShowBackendsStmt) stmt; - List<List<String>> backendInfos = BackendsProcDir.getBackendInfos(showStmt.getType()); + List<List<String>> backendInfos = BackendsProcDir.getBackendInfos(); backendInfos.sort(new Comparator<List<String>>() { @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java b/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java index 582567ae156..fcb5e63e838 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java @@ -59,7 +59,6 @@ public class Backend implements Writable { // Represent a meaningless IP public static final String DUMMY_IP = "0.0.0.0"; - public static final String DATA_DIR_TYPE = "STORAGE"; @SerializedName("id") private long id; @@ -375,44 +374,22 @@ public class Backend implements Writable { } public ImmutableMap<String, DiskInfo> getDisks() { - Map<String, DiskInfo> disks = Maps.newHashMap(); - for (Map.Entry<String, DiskInfo> entry : disksRef.entrySet()) { - if (entry.getValue().getDirType().equals(DATA_DIR_TYPE)) { - disks.put(entry.getKey(), entry.getValue()); - } - } - return ImmutableMap.copyOf(disks); - } - - public ImmutableMap<String, DiskInfo> getAllDisks() { return this.disksRef; } public boolean hasPathHash() { - Map<String, DiskInfo> disks = Maps.newHashMap(); - for (Map.Entry<String, DiskInfo> entry : disksRef.entrySet()) { - if (entry.getValue().getDirType().equals(DATA_DIR_TYPE)) { - disks.put(entry.getKey(), entry.getValue()); - } - } - return disks.values().stream().allMatch(DiskInfo::hasPathHash); + return disksRef.values().stream().allMatch(DiskInfo::hasPathHash); } public boolean hasSpecifiedStorageMedium(TStorageMedium storageMedium) { - Map<String, DiskInfo> disks = Maps.newHashMap(); - for (Map.Entry<String, DiskInfo> entry : disksRef.entrySet()) { - if (entry.getValue().getDirType().equals(DATA_DIR_TYPE)) { - disks.put(entry.getKey(), entry.getValue()); - } - } - return disks.values().stream().anyMatch(d -> d.isStorageMediumMatch(storageMedium)); + return disksRef.values().stream().anyMatch(d -> d.isStorageMediumMatch(storageMedium)); } public long getTotalCapacityB() { ImmutableMap<String, DiskInfo> disks = disksRef; long totalCapacityB = 0L; for (DiskInfo diskInfo : disks.values()) { - if (diskInfo.getState() == DiskState.ONLINE && diskInfo.getDirType().equals(DATA_DIR_TYPE)) { + if (diskInfo.getState() == DiskState.ONLINE) { totalCapacityB += diskInfo.getTotalCapacityB(); } } @@ -424,7 +401,7 @@ public class Backend implements Writable { ImmutableMap<String, DiskInfo> disks = disksRef; long availableCapacityB = 1L; for (DiskInfo diskInfo : disks.values()) { - if (diskInfo.getState() == DiskState.ONLINE && diskInfo.getDirType().equals(DATA_DIR_TYPE)) { + if (diskInfo.getState() == DiskState.ONLINE) { availableCapacityB += diskInfo.getAvailableCapacityB(); } } @@ -435,7 +412,7 @@ public class Backend implements Writable { ImmutableMap<String, DiskInfo> disks = disksRef; long dataUsedCapacityB = 0L; for (DiskInfo diskInfo : disks.values()) { - if (diskInfo.getState() == DiskState.ONLINE && diskInfo.getDirType().equals(DATA_DIR_TYPE)) { + if (diskInfo.getState() == DiskState.ONLINE) { dataUsedCapacityB += diskInfo.getDataUsedCapacityB(); } } @@ -446,7 +423,7 @@ public class Backend implements Writable { ImmutableMap<String, DiskInfo> disks = disksRef; long trashUsedCapacityB = 0L; for (DiskInfo diskInfo : disks.values()) { - if (diskInfo.getState() == DiskState.ONLINE && diskInfo.getDirType().equals(DATA_DIR_TYPE)) { + if (diskInfo.getState() == DiskState.ONLINE) { trashUsedCapacityB += diskInfo.getTrashUsedCapacityB(); } } @@ -457,7 +434,7 @@ public class Backend implements Writable { ImmutableMap<String, DiskInfo> disks = disksRef; long totalRemoteUsedCapacityB = 0L; for (DiskInfo diskInfo : disks.values()) { - if (diskInfo.getState() == DiskState.ONLINE && diskInfo.getDirType().equals(DATA_DIR_TYPE)) { + if (diskInfo.getState() == DiskState.ONLINE) { totalRemoteUsedCapacityB += diskInfo.getRemoteUsedCapacity(); } } @@ -468,7 +445,7 @@ public class Backend implements Writable { ImmutableMap<String, DiskInfo> disks = disksRef; double maxPct = 0.0; for (DiskInfo diskInfo : disks.values()) { - if (diskInfo.getState() == DiskState.ONLINE && diskInfo.getDirType().equals(DATA_DIR_TYPE)) { + if (diskInfo.getState() == DiskState.ONLINE) { double percent = diskInfo.getUsedPct(); if (percent > maxPct) { maxPct = percent; @@ -486,7 +463,7 @@ public class Backend implements Writable { boolean exceedLimit = true; for (DiskInfo diskInfo : diskInfos.values()) { if (diskInfo.getState() == DiskState.ONLINE && diskInfo.getStorageMedium() - == storageMedium && !diskInfo.exceedLimit(true) && diskInfo.getDirType().equals(DATA_DIR_TYPE)) { + == storageMedium && !diskInfo.exceedLimit(true)) { exceedLimit = false; break; } @@ -501,8 +478,7 @@ public class Backend implements Writable { ImmutableMap<String, DiskInfo> diskInfos = disksRef; boolean exceedLimit = true; for (DiskInfo diskInfo : diskInfos.values()) { - if (diskInfo.getState() == DiskState.ONLINE && diskInfo.getDirType().equals(DATA_DIR_TYPE) - && !diskInfo.exceedLimit(true)) { + if (diskInfo.getState() == DiskState.ONLINE && !diskInfo.exceedLimit(true)) { exceedLimit = false; break; } @@ -516,7 +492,7 @@ public class Backend implements Writable { if (!initPathInfo) { boolean allPathHashUpdated = true; for (DiskInfo diskInfo : disks.values()) { - if (diskInfo.getDirType().equals(DATA_DIR_TYPE) && diskInfo.getPathHash() == 0) { + if (diskInfo.getPathHash() == 0) { allPathHashUpdated = false; break; } @@ -543,7 +519,6 @@ public class Backend implements Writable { long trashUsedCapacityB = tDisk.getTrashUsedCapacity(); long diskAvailableCapacityB = tDisk.getDiskAvailableCapacity(); boolean isUsed = tDisk.isUsed(); - String dirType = tDisk.getDirType().toString(); DiskInfo diskInfo = disks.get(rootPath); if (diskInfo == null) { diskInfo = new DiskInfo(rootPath); @@ -557,8 +532,6 @@ public class Backend implements Writable { diskInfo.setDataUsedCapacityB(dataUsedCapacityB); diskInfo.setTrashUsedCapacityB(trashUsedCapacityB); diskInfo.setAvailableCapacityB(diskAvailableCapacityB); - diskInfo.setDirType(dirType); - if (tDisk.isSetRemoteUsedCapacity()) { diskInfo.setRemoteUsedCapacity(tDisk.getRemoteUsedCapacity()); } @@ -580,7 +553,6 @@ public class Backend implements Writable { isChanged = true; } } - LOG.debug("update disk info. backendId: {}, diskInfo: {}", id, diskInfo.toString()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java index 28c3a6a7ba7..69af28dffda 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java @@ -868,14 +868,10 @@ public class SystemInfoService { public void updatePathInfo(List<DiskInfo> addedDisks, List<DiskInfo> removedDisks) { Map<Long, DiskInfo> copiedPathInfos = Maps.newHashMap(pathHashToDiskInfoRef); for (DiskInfo diskInfo : addedDisks) { - if (diskInfo.getDirType().equals("STORAGE")) { - copiedPathInfos.put(diskInfo.getPathHash(), diskInfo); - } + copiedPathInfos.put(diskInfo.getPathHash(), diskInfo); } for (DiskInfo diskInfo : removedDisks) { - if (diskInfo.getDirType().equals("STORAGE")) { - copiedPathInfos.remove(diskInfo.getPathHash()); - } + copiedPathInfos.remove(diskInfo.getPathHash()); } ImmutableMap<Long, DiskInfo> newPathInfos = ImmutableMap.copyOf(copiedPathInfos); pathHashToDiskInfoRef = newPathInfos; diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/BackendTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/BackendTest.java index 2aa5d59ffd7..86d95482fa6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/catalog/BackendTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/BackendTest.java @@ -22,7 +22,6 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.resource.Tag; import org.apache.doris.system.Backend; import org.apache.doris.thrift.TDisk; -import org.apache.doris.thrift.TDiskType; import org.apache.doris.thrift.TStorageMedium; import com.google.common.collect.ImmutableMap; @@ -93,11 +92,8 @@ public class BackendTest { Map<String, TDisk> diskInfos = new HashMap<String, TDisk>(); TDisk disk1 = new TDisk("/data1/", 1000, 800, true); - disk1.setDirType(TDiskType.STORAGE); TDisk disk2 = new TDisk("/data2/", 2000, 700, true); - disk2.setDirType(TDiskType.STORAGE); TDisk disk3 = new TDisk("/data3/", 3000, 600, false); - disk3.setDirType(TDiskType.STORAGE); diskInfos.put(disk1.getRootPath(), disk1); diskInfos.put(disk2.getRootPath(), disk2); diff --git a/fe/fe-core/src/test/java/org/apache/doris/clone/DecommissionTest.java b/fe/fe-core/src/test/java/org/apache/doris/clone/DecommissionTest.java index f74dd3a3f27..1764573d0fc 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/clone/DecommissionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/clone/DecommissionTest.java @@ -29,7 +29,6 @@ import org.apache.doris.qe.ConnectContext; import org.apache.doris.system.Backend; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TDisk; -import org.apache.doris.thrift.TDiskType; import org.apache.doris.thrift.TStorageMedium; import org.apache.doris.utframe.UtFrameUtils; @@ -92,7 +91,6 @@ public class DecommissionTest { tDisk1.setDiskAvailableCapacity(tDisk1.disk_total_capacity - tDisk1.data_used_capacity); tDisk1.setPathHash(random.nextLong()); tDisk1.setStorageMedium(TStorageMedium.HDD); - tDisk1.setDirType(TDiskType.STORAGE); backendDisks.put(tDisk1.getRootPath(), tDisk1); TDisk tDisk2 = new TDisk(); @@ -103,7 +101,6 @@ public class DecommissionTest { tDisk2.setDiskAvailableCapacity(tDisk2.disk_total_capacity - tDisk2.data_used_capacity); tDisk2.setPathHash(random.nextLong()); tDisk2.setStorageMedium(TStorageMedium.HDD); - tDisk2.setDirType(TDiskType.STORAGE); backendDisks.put(tDisk2.getRootPath(), tDisk2); be.updateDisks(backendDisks); diff --git a/fe/fe-core/src/test/java/org/apache/doris/clone/TabletRepairAndBalanceTest.java b/fe/fe-core/src/test/java/org/apache/doris/clone/TabletRepairAndBalanceTest.java index 2c98decef94..3e178ef0903 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/clone/TabletRepairAndBalanceTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/clone/TabletRepairAndBalanceTest.java @@ -49,7 +49,6 @@ import org.apache.doris.resource.Tag; import org.apache.doris.system.Backend; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TDisk; -import org.apache.doris.thrift.TDiskType; import org.apache.doris.thrift.TStorageMedium; import org.apache.doris.utframe.UtFrameUtils; @@ -135,7 +134,6 @@ public class TabletRepairAndBalanceTest { tDisk1.setDiskAvailableCapacity(tDisk1.disk_total_capacity - tDisk1.data_used_capacity); tDisk1.setPathHash(random.nextLong()); tDisk1.setStorageMedium(TStorageMedium.HDD); - tDisk1.setDirType(TDiskType.STORAGE); backendDisks.put(tDisk1.getRootPath(), tDisk1); TDisk tDisk2 = new TDisk(); @@ -146,7 +144,6 @@ public class TabletRepairAndBalanceTest { tDisk2.setDiskAvailableCapacity(tDisk2.disk_total_capacity - tDisk2.data_used_capacity); tDisk2.setPathHash(random.nextLong()); tDisk2.setStorageMedium(TStorageMedium.SSD); - tDisk2.setDirType(TDiskType.STORAGE); backendDisks.put(tDisk2.getRootPath(), tDisk2); be.updateDisks(backendDisks); diff --git a/fe/fe-core/src/test/java/org/apache/doris/clone/TabletReplicaTooSlowTest.java b/fe/fe-core/src/test/java/org/apache/doris/clone/TabletReplicaTooSlowTest.java index cb980b4051e..6a38985b73f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/clone/TabletReplicaTooSlowTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/clone/TabletReplicaTooSlowTest.java @@ -32,7 +32,6 @@ import org.apache.doris.system.Backend; import org.apache.doris.system.Diagnoser; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TDisk; -import org.apache.doris.thrift.TDiskType; import org.apache.doris.thrift.TStorageMedium; import org.apache.doris.utframe.UtFrameUtils; @@ -104,7 +103,6 @@ public class TabletReplicaTooSlowTest { tDisk1.setDiskAvailableCapacity(tDisk1.disk_total_capacity - tDisk1.data_used_capacity); tDisk1.setPathHash(random.nextLong()); tDisk1.setStorageMedium(TStorageMedium.HDD); - tDisk1.setDirType(TDiskType.STORAGE); backendDisks.put(tDisk1.getRootPath(), tDisk1); TDisk tDisk2 = new TDisk(); @@ -115,7 +113,6 @@ public class TabletReplicaTooSlowTest { tDisk2.setDiskAvailableCapacity(tDisk2.disk_total_capacity - tDisk2.data_used_capacity); tDisk2.setPathHash(random.nextLong()); tDisk2.setStorageMedium(TStorageMedium.SSD); - tDisk2.setDirType(TDiskType.STORAGE); backendDisks.put(tDisk2.getRootPath(), tDisk2); be.updateDisks(backendDisks); diff --git a/fe/fe-core/src/test/java/org/apache/doris/common/util/AutoBucketUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/common/util/AutoBucketUtilsTest.java index 38018775201..3a239287342 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/common/util/AutoBucketUtilsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/common/util/AutoBucketUtilsTest.java @@ -27,7 +27,6 @@ import org.apache.doris.qe.ShowResultSet; import org.apache.doris.system.Backend; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TDisk; -import org.apache.doris.thrift.TDiskType; import org.apache.doris.thrift.TStorageMedium; import org.apache.doris.utframe.UtFrameUtils; @@ -96,7 +95,6 @@ public class AutoBucketUtilsTest { disk.setDiskAvailableCapacity(disk.disk_total_capacity - disk.data_used_capacity); disk.setPathHash(random.nextLong()); disk.setStorageMedium(TStorageMedium.HDD); - disk.setDirType(TDiskType.STORAGE); backendDisks.put(disk.getRootPath(), disk); } be.updateDisks(backendDisks); diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/ResourceTagQueryTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/ResourceTagQueryTest.java index 7019d4ba99c..bd995ca35ff 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/ResourceTagQueryTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/ResourceTagQueryTest.java @@ -43,7 +43,6 @@ import org.apache.doris.qe.DdlExecutor; import org.apache.doris.resource.Tag; import org.apache.doris.system.Backend; import org.apache.doris.thrift.TDisk; -import org.apache.doris.thrift.TDiskType; import org.apache.doris.thrift.TStorageMedium; import org.apache.doris.utframe.UtFrameUtils; @@ -121,7 +120,6 @@ public class ResourceTagQueryTest { tDisk1.setDiskAvailableCapacity(tDisk1.disk_total_capacity - tDisk1.data_used_capacity); tDisk1.setPathHash(random.nextLong()); tDisk1.setStorageMedium(TStorageMedium.HDD); - tDisk1.setDirType(TDiskType.STORAGE); backendDisks.put(tDisk1.getRootPath(), tDisk1); TDisk tDisk2 = new TDisk(); @@ -132,7 +130,6 @@ public class ResourceTagQueryTest { tDisk2.setDiskAvailableCapacity(tDisk2.disk_total_capacity - tDisk2.data_used_capacity); tDisk2.setPathHash(random.nextLong()); tDisk2.setStorageMedium(TStorageMedium.SSD); - tDisk2.setDirType(TDiskType.STORAGE); backendDisks.put(tDisk2.getRootPath(), tDisk2); be.updateDisks(backendDisks); diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoMultiBackendsTest.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoMultiBackendsTest.java index dac84521e4c..af9b98f126f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoMultiBackendsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/DemoMultiBackendsTest.java @@ -41,7 +41,6 @@ import org.apache.doris.qe.StmtExecutor; import org.apache.doris.resource.Tag; import org.apache.doris.system.Backend; import org.apache.doris.thrift.TDisk; -import org.apache.doris.thrift.TDiskType; import org.apache.doris.thrift.TStorageMedium; import org.apache.doris.utframe.MockedFrontend.EnvVarNotSetException; import org.apache.doris.utframe.MockedFrontend.FeStartException; @@ -101,7 +100,6 @@ public class DemoMultiBackendsTest { tDisk1.setDiskAvailableCapacity(tDisk1.disk_total_capacity - tDisk1.data_used_capacity); tDisk1.setPathHash(random.nextLong()); tDisk1.setStorageMedium(TStorageMedium.HDD); - tDisk1.setDirType(TDiskType.STORAGE); backendDisks.put(tDisk1.getRootPath(), tDisk1); TDisk tDisk2 = new TDisk(); @@ -112,7 +110,6 @@ public class DemoMultiBackendsTest { tDisk2.setDiskAvailableCapacity(tDisk2.disk_total_capacity - tDisk2.data_used_capacity); tDisk2.setPathHash(random.nextLong()); tDisk2.setStorageMedium(TStorageMedium.SSD); - tDisk2.setDirType(TDiskType.STORAGE); backendDisks.put(tDisk2.getRootPath(), tDisk2); be.updateDisks(backendDisks); diff --git a/gensrc/thrift/MasterService.thrift b/gensrc/thrift/MasterService.thrift index 088d01dad6b..9acd3f85f7b 100644 --- a/gensrc/thrift/MasterService.thrift +++ b/gensrc/thrift/MasterService.thrift @@ -84,7 +84,6 @@ struct TDisk { 7: optional Types.TStorageMedium storage_medium 8: optional Types.TSize remote_used_capacity 9: optional Types.TSize trash_used_capacity - 10: optional Types.TDiskType dir_type } struct TPluginInfo { diff --git a/gensrc/thrift/Types.thrift b/gensrc/thrift/Types.thrift index 15840e0c919..f6a138976c4 100644 --- a/gensrc/thrift/Types.thrift +++ b/gensrc/thrift/Types.thrift @@ -51,12 +51,6 @@ enum TStorageMedium { REMOTE_CACHE, } -enum TDiskType { - STORAGE, - LOG, - DEPLOY -} - enum TVarType { SESSION, GLOBAL --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org