[fix](cold hot separation) Fix the issue of root_path not working in HDFS resource (#48441)

### What problem does this PR solve?

Issue Number: close #xxx

Related PR: #xxx

Problem Summary:

before:
```sql
CREATE RESOURCE "remote_hdfs" PROPERTIES (
  "type"="hdfs",
  "fs.defaultFS"="192.168.0.1:8020",
  "hadoop.username"="hdfs",
  "root_path"="/user/hdfs" -- not working 👎
 );
```

after:

be.conf
```properties
enable_root_path_of_hdfs_resource=true
```

and then

```sql
CREATE RESOURCE "remote_hdfs" PROPERTIES (
  "type"="hdfs",
  "fs.defaultFS"="192.168.0.1:8020",
  "hadoop.username"="hdfs",
  "root_path"="/user/hdfs" -- working 👍
 );
```

### Release note

None

### Check List (For Author)

- Test <!-- At least one of them must be included. -->
    - [ ] Regression test
    - [ ] Unit Test
    - [ ] Manual test (add detailed scripts or steps below)
    - [ ] No need to test or manual test. Explain why:
- [ ] This is a refactor/code format and no logic has been changed.
        - [ ] Previous test can cover this change.
        - [ ] No code files have been changed.
        - [ ] Other reason <!-- Add your reason?  -->

- Behavior changed:
    - [ ] No.
    - [x] Yes. <!-- Explain the behavior change -->

- Does this need documentation?
    - [ ] No.
- [ ] Yes. <!-- Add document PR link here. eg:
https://github.com/apache/doris-website/pull/1214 -->

### Check List (For Reviewer who merge this PR)

- [ ] Confirm the release note
- [ ] Confirm test cases
- [ ] Confirm document
- [ ] Add branch pick label <!-- Add branch pick label that this PR
should merge into -->


also pick #48452
This commit is contained in:
yagagagaga
2025-06-20 09:15:58 +08:00
committed by GitHub
parent aa562b3e8f
commit ab48d63d1e
8 changed files with 163 additions and 9 deletions

View File

@ -1296,9 +1296,9 @@ void push_storage_policy_callback(StorageEngine& engine, const TAgentTaskRequest
? resource.hdfs_storage_param.root_path
: "";
if (existed_resource.fs == nullptr) {
st = io::HdfsFileSystem::create(resource.hdfs_storage_param,
std::to_string(resource.id), root_path, nullptr,
&fs);
st = io::HdfsFileSystem::create(
resource.hdfs_storage_param, std::to_string(resource.id),
resource.hdfs_storage_param.fs_name, nullptr, &fs, std::move(root_path));
} else {
fs = std::static_pointer_cast<io::HdfsFileSystem>(existed_resource.fs);
}

View File

@ -1389,6 +1389,8 @@ DEFINE_String(test_s3_prefix, "prefix");
#endif
// clang-format on
DEFINE_Bool(enable_root_path_of_hdfs_resource, "false");
std::map<std::string, Register::Field>* Register::_s_field_map = nullptr;
std::map<std::string, std::function<bool()>>* RegisterConfValidator::_s_field_validator = nullptr;
std::map<std::string, std::string>* full_conf_map = nullptr;

View File

@ -1436,6 +1436,11 @@ DECLARE_mInt32(compaction_num_per_round);
// Enable sleep 5s between delete cumulative compaction.
DECLARE_mBool(enable_sleep_between_delete_cumu_compaction);
// Because the root_path for the HDFS resource was previously passed an empty string,
// which was incorrect, this configuration has been added to ensure compatibility
// and guarantee that the root_path works as expected.
DECLARE_Bool(enable_root_path_of_hdfs_resource);
// whether to prune rows with delete sign = 1 in base compaction
// ATTN: this config is only for test
DECLARE_mBool(enable_prune_delete_sign_when_base_compaction);

View File

@ -123,7 +123,7 @@ Status HdfsFileHandleCache::get_file(const std::shared_ptr<HdfsFileSystem>& fs,
Status HdfsFileSystem::create(const THdfsParams& hdfs_params, std::string id,
const std::string& fs_name, RuntimeProfile* profile,
std::shared_ptr<HdfsFileSystem>* fs) {
std::shared_ptr<HdfsFileSystem>* fs, std::string root_path) {
#ifdef USE_HADOOP_HDFS
if (!config::enable_java_support) {
return Status::InternalError(
@ -131,13 +131,16 @@ Status HdfsFileSystem::create(const THdfsParams& hdfs_params, std::string id,
"true.");
}
#endif
(*fs).reset(new HdfsFileSystem(hdfs_params, std::move(id), fs_name, profile));
(*fs).reset(
new HdfsFileSystem(hdfs_params, std::move(id), fs_name, profile, std::move(root_path)));
return (*fs)->connect();
}
HdfsFileSystem::HdfsFileSystem(const THdfsParams& hdfs_params, std::string id,
const std::string& fs_name, RuntimeProfile* profile)
: RemoteFileSystem("", std::move(id), FileSystemType::HDFS),
const std::string& fs_name, RuntimeProfile* profile,
std::string root_path)
: RemoteFileSystem(config::enable_root_path_of_hdfs_resource ? std::move(root_path) : "",
std::move(id), FileSystemType::HDFS),
_hdfs_params(hdfs_params),
_fs_handle(nullptr),
_profile(profile) {

View File

@ -97,7 +97,8 @@ class HdfsFileHandleCache;
class HdfsFileSystem final : public RemoteFileSystem {
public:
static Status create(const THdfsParams& hdfs_params, std::string id, const std::string& path,
RuntimeProfile* profile, std::shared_ptr<HdfsFileSystem>* fs);
RuntimeProfile* profile, std::shared_ptr<HdfsFileSystem>* fs,
std::string root_path = "");
~HdfsFileSystem() override;
@ -130,7 +131,7 @@ private:
private:
friend class HdfsFileWriter;
HdfsFileSystem(const THdfsParams& hdfs_params, std::string id, const std::string& path,
RuntimeProfile* profile);
RuntimeProfile* profile, std::string root_path);
const THdfsParams& _hdfs_params;
std::string _fs_name;
std::shared_ptr<HdfsFileSystemHandle> _fs_handle = nullptr;

View File

@ -50,6 +50,7 @@ import org.apache.doris.regression.util.JdbcUtils
import org.apache.doris.regression.util.Hdfs
import org.apache.doris.regression.util.SuiteUtils
import org.apache.doris.regression.util.DebugPoint
import org.apache.hadoop.fs.FileSystem
import org.junit.jupiter.api.Assertions
import org.slf4j.Logger
@ -93,6 +94,7 @@ class Suite implements GroovyInterceptable {
final List<Future> lazyCheckFutures = new Vector<>()
private AmazonS3 s3Client = null
private FileSystem fs = null
Suite(String name, String group, SuiteContext context, SuiteCluster cluster) {
this.name = name
@ -849,6 +851,16 @@ class Suite implements GroovyInterceptable {
return enableHdfs.equals("true");
}
synchronized FileSystem getHdfs() {
if (fs == null) {
String hdfsFs = context.config.otherConfigs.get("hdfsFs")
String hdfsUser = context.config.otherConfigs.get("hdfsUser")
Hdfs hdfs = new Hdfs(hdfsFs, hdfsUser, context.config.dataPath + "/")
fs = hdfs.fs
}
return fs
}
String uploadToHdfs(String localFile) {
// as group can be rewrite the origin data file not relate to group
String dataDir = context.config.dataPath + "/"

View File

@ -97,3 +97,5 @@ cold_data_compaction_interval_sec=60
# This feature has bug, so by default is false, only open it in pipeline to observe
enable_parquet_page_index=true
enable_root_path_of_hdfs_resource=true

View File

@ -0,0 +1,129 @@
// 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.
import org.apache.hadoop.fs.Path
import java.util.function.Supplier
suite("test_cold_data_compaction_by_hdfs") {
if (!enableHdfs()) {
logger.info("Skip this case, because HDFS is not available")
return
}
def retryUntilTimeout = { int timeoutSecond, Supplier<Boolean> closure ->
long start = System.currentTimeMillis()
while (true) {
if (closure.get()) {
return
} else {
if (System.currentTimeMillis() - start > timeoutSecond * 1000) {
throw new RuntimeException("" +
"Operation timeout, maybe you need to check " +
"remove_unused_remote_files_interval_sec and " +
"cold_data_compaction_interval_sec in be.conf")
} else {
sleep(10_000)
}
}
}
}
String suffix = UUID.randomUUID().hashCode().abs().toString()
String prefix = "${getHdfsDataDir()}/regression/cold_data_compaction"
multi_sql """
DROP TABLE IF EXISTS t_recycle_in_hdfs;
DROP STORAGE POLICY IF EXISTS test_policy_${suffix};
DROP RESOURCE IF EXISTS 'remote_hdfs_${suffix}';
CREATE RESOURCE "remote_hdfs_${suffix}"
PROPERTIES
(
"type"="hdfs",
"fs.defaultFS"="${getHdfsFs()}",
"hadoop.username"="${getHdfsUser()}",
"hadoop.password"="${getHdfsPasswd()}",
"root_path"="${prefix}"
);
CREATE STORAGE POLICY test_policy_${suffix}
PROPERTIES(
"storage_resource" = "remote_hdfs_${suffix}",
"cooldown_ttl" = "5"
);
CREATE TABLE IF NOT EXISTS t_recycle_in_hdfs
(
k1 BIGINT,
k2 LARGEINT,
v1 VARCHAR(2048)
)
DISTRIBUTED BY HASH (k1) BUCKETS 1
PROPERTIES(
"storage_policy" = "test_policy_${suffix}",
"disable_auto_compaction" = "true",
"replication_num" = "1"
);
"""
// insert 5 RowSets
multi_sql """
insert into t_recycle_in_hdfs values(1, 1, 'Tom');
insert into t_recycle_in_hdfs values(2, 2, 'Jelly');
insert into t_recycle_in_hdfs values(3, 3, 'Spike');
insert into t_recycle_in_hdfs values(4, 4, 'Tyke');
insert into t_recycle_in_hdfs values(5, 5, 'Tuffy');
"""
// wait until files upload to S3
retryUntilTimeout(1800, {
def res = sql_return_maparray "show data from t_recycle_in_hdfs"
String size = ""
String remoteSize = ""
for (final def line in res) {
if ("t_recycle_in_hdfs".equals(line.TableName)) {
size = line.Size
remoteSize = line.RemoteSize
break
}
}
logger.info("waiting for data to be uploaded to HDFS: t_recycle_in_hdfs's local data size: ${size}, remote data size: ${remoteSize}")
return size.startsWith("0") && !remoteSize.startsWith("0")
})
String tabletId = sql_return_maparray("show tablets from t_recycle_in_hdfs")[0].TabletId
// check number of remote files
def filesBeforeCompaction = getHdfs().listStatus(new Path(prefix + "/data/${tabletId}"))
// 5 RowSets + 1 meta
assertEquals(6, filesBeforeCompaction.size())
// trigger cold data compaction
sql """alter table t_recycle_in_hdfs set ("disable_auto_compaction" = "false")"""
// wait until compaction finish
retryUntilTimeout(1800, {
def filesAfterCompaction = getHdfs().listStatus(new Path(prefix + "/data/${tabletId}"))
logger.info("t_recycle_in_hdfs's remote file number is ${filesAfterCompaction.size()}")
// 1 RowSet + 1 meta
return filesAfterCompaction.size() == 2
})
sql "drop table t_recycle_in_hdfs force"
retryUntilTimeout(1800, {
def pathExists = getHdfs().exists(new Path(prefix + "/data/${tabletId}"))
logger.info("after drop t_recycle_in_hdfs, the remote file path ${pathExists ? "exists" : "not exists"}")
return !pathExists
})
}