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/11/11 11:39:47 UTC

[GitHub] [incubator-doris] pengxiangyu opened a new pull request #7098: Support remote storage, step1: use a struct instead of string for parameter path, add basic remote method

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


   ## Proposed changes
   
   For the first, we need to make a parameter to discribe the data is local or remote.
   At then, we need to support some basic function to support the operation for remote storage.
   
   ## 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)
   - [x] 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...)
   - [ ] 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._
   
   - [x] I have created an issue on (Fix #7097) 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
   
   After this patch, the main class and method will be achieved. Including migrate data from local to remote storage, get remote data when select is called.


-- 
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 merged pull request #7098: Support remote storage, step1: use a struct instead of string for parameter path, add basic remote method

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


   


-- 
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 #7098: Support remote storage, step1: use a struct instead of string for parameter path, add basic remote method

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


   link to #7575 


-- 
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] github-actions[bot] commented on pull request #7098: Support remote storage, step1: use a struct instead of string for parameter path, add basic remote method

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #7098:
URL: https://github.com/apache/incubator-doris/pull/7098#issuecomment-999259972






-- 
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 #7098: Support remote storage, step1: use a struct instead of string for parameter path, add basic remote method

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



##########
File path: be/src/olap/rowset/beta_rowset.cpp
##########
@@ -138,34 +141,69 @@ OLAPStatus BetaRowset::link_files_to(const std::string& dir, RowsetId new_rowset
 
 OLAPStatus BetaRowset::copy_files_to(const std::string& dir) {
     for (int i = 0; i < num_segments(); ++i) {
-        std::string dst_path = segment_file_path(dir, rowset_id(), i);
-        if (FileUtils::check_exist(dst_path)) {
-            LOG(WARNING) << "file already exist: " << dst_path;
+        FilePathDesc dst_path_desc = segment_file_path(dir, rowset_id(), i);
+        Status status = Env::get_env(_rowset_path_desc.storage_medium)->path_exists(dst_path_desc.filepath);

Review comment:
       we can get `env` once instead of get it for each segment

##########
File path: be/src/olap/rowset/beta_rowset_writer.cpp
##########
@@ -54,13 +54,13 @@ BetaRowsetWriter::~BetaRowsetWriter() {
         _segment_writer.reset(); // ensure all files are closed
         Status st;
         for (int i = 0; i < _num_segment; ++i) {
-            auto path = BetaRowset::segment_file_path(_context.rowset_path_prefix,
+            auto path_desc = BetaRowset::segment_file_path(_context.path_desc,
                                                       _context.rowset_id, i);
             // Even if an error is encountered, these files that have not been cleaned up
             // will be cleaned up by the GC background. So here we only print the error
             // message when we encounter an error.
-            WARN_IF_ERROR(Env::Default()->delete_file(path),
-                          strings::Substitute("Failed to delete file=$0", path));
+            WARN_IF_ERROR(Env::get_env(_context.path_desc.storage_medium)->delete_file(path_desc.filepath),

Review comment:
       get env once

##########
File path: be/src/util/coding.cpp
##########
@@ -1,11 +1,19 @@
-//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
-//  This source code is licensed under both the GPLv2 (found in the
-//  COPYING file in the root directory) and Apache 2.0 License
-//  (found in the LICENSE.Apache file in the root directory).
+// 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
 //
-// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file. See the AUTHORS file for names of contributors.
+//   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.

Review comment:
       Do not modify license

##########
File path: be/src/util/coding.h
##########
@@ -1,11 +1,19 @@
-//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
-//  This source code is licensed under both the GPLv2 (found in the
-//  COPYING file in the root directory) and Apache 2.0 License
-//  (found in the LICENSE.Apache file in the root directory).
+// 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
 //
-// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file. See the AUTHORS file for names of contributors.
+//   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.

Review comment:
       Do not modify license

##########
File path: be/src/env/env.h
##########
@@ -1,11 +1,19 @@
-//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
-//  This source code is licensed under both the GPLv2 (found in the
-//  COPYING file in the root directory) and Apache 2.0 License
-//  (found in the LICENSE.Apache file in the root directory).
+// Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       Do not modify origin license header

##########
File path: be/src/env/env_posix.cpp
##########
@@ -266,13 +277,38 @@ class PosixRandomAccessFile : public RandomAccessFile {
         }
     }
 
-    Status read_at(uint64_t offset, const Slice& result) const override {
-        return do_readv_at(_fd, _filename, offset, &result, 1);
+    Status read_at(uint64_t offset, const Slice* result) const override {
+        return readv_at(offset, result, 1);
+    }
+
+    Status readv_at(uint64_t offset, const Slice* result, size_t res_cnt) const override {
+        return do_readv_at(_fd, _filename, offset, result, res_cnt);
     }
 
-    Status readv_at(uint64_t offset, const Slice* res, size_t res_cnt) const override {
-        return do_readv_at(_fd, _filename, offset, res, res_cnt);
+    Status read_all(std::string* content) const override {
+        std::fstream fs(_filename.c_str(), std::fstream::in);
+        if (!fs.is_open()) {
+            RETURN_NOT_OK_STATUS_WITH_WARN(
+                    Status::IOError(
+                            strings::Substitute("failed to open cluster id file $0", _filename)),
+                    "open file failed");
+        }
+        std::string data;
+        fs >> data;
+        fs.close();
+        if ((fs.rdstate() & std::fstream::eofbit) != 0) {
+            *content = data;

Review comment:
       The memory allocate for string `data` will be released.

##########
File path: be/src/env/env_posix.cpp
##########
@@ -1,11 +1,19 @@
-//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
-//  This source code is licensed under both the GPLv2 (found in the
-//  COPYING file in the root directory) and Apache 2.0 License
-//  (found in the LICENSE.Apache file in the root directory).
+// Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       Do not modify origin license header

##########
File path: be/src/olap/rowset/beta_rowset.cpp
##########
@@ -138,34 +141,69 @@ OLAPStatus BetaRowset::link_files_to(const std::string& dir, RowsetId new_rowset
 
 OLAPStatus BetaRowset::copy_files_to(const std::string& dir) {
     for (int i = 0; i < num_segments(); ++i) {
-        std::string dst_path = segment_file_path(dir, rowset_id(), i);
-        if (FileUtils::check_exist(dst_path)) {
-            LOG(WARNING) << "file already exist: " << dst_path;
+        FilePathDesc dst_path_desc = segment_file_path(dir, rowset_id(), i);
+        Status status = Env::get_env(_rowset_path_desc.storage_medium)->path_exists(dst_path_desc.filepath);
+        if (status.ok()) {
+            LOG(WARNING) << "file already exist: " << dst_path_desc.filepath;
             return OLAP_ERR_FILE_ALREADY_EXIST;
         }
-        std::string src_path = segment_file_path(_rowset_path, rowset_id(), i);
-        if (copy_file(src_path, dst_path) != OLAP_SUCCESS) {
-            LOG(WARNING) << "fail to copy file. from=" << src_path << ", to=" << dst_path
-                         << ", errno=" << Errno::no();
+        if (!status.is_not_found()) {
+            LOG(WARNING) << "file check exist error: " << dst_path_desc.filepath;
             return OLAP_ERR_OS_ERROR;
         }
+        FilePathDesc src_path_desc = segment_file_path(_rowset_path_desc, rowset_id(), i);
+        if (!Env::get_env(_rowset_path_desc.storage_medium)->copy_path(
+                src_path_desc.filepath, dst_path_desc.filepath).ok()) {
+            LOG(WARNING) << "fail to copy file. from=" << src_path_desc.filepath << ", to="
+                    << dst_path_desc.filepath << ", errno=" << Errno::no();
+            return OLAP_ERR_OS_ERROR;
+        }
+    }
+    return OLAP_SUCCESS;
+}
+
+OLAPStatus BetaRowset::upload_files_to(const FilePathDesc& dir_desc) {
+    RemoteEnv* dest_env = dynamic_cast<RemoteEnv*>(Env::get_env(_rowset_path_desc.storage_medium));
+    std::shared_ptr<StorageBackend> storage_backend = dest_env->get_storage_backend();
+    for (int i = 0; i < num_segments(); ++i) {
+        FilePathDesc dst_path_desc = segment_file_path(dir_desc, rowset_id(), i);
+        Status status = storage_backend->exist(dst_path_desc.remote_path);
+        if (status.ok()) {
+            LOG(WARNING) << "file already exist: " << dst_path_desc.remote_path;
+            return OLAP_ERR_FILE_ALREADY_EXIST;
+        }
+        if (!status.is_not_found()) {
+            LOG(WARNING) << "file check exist error: " << dst_path_desc.remote_path;
+            return OLAP_ERR_OS_ERROR;
+        }
+        FilePathDesc src_path_desc = segment_file_path(_rowset_path_desc, rowset_id(), i);
+
+        if (!storage_backend->upload(src_path_desc.filepath, dst_path_desc.remote_path).ok()) {
+            LOG(WARNING) << "fail to upload file. from=" << src_path_desc.filepath << ", to="
+                         << dst_path_desc.remote_path << ", errno=" << Errno::no();
+            return OLAP_ERR_OS_ERROR;
+        }
+        LOG(INFO) << "succeed to upload file. from " << src_path_desc.filepath << " to "
+                  << dst_path_desc.remote_path;
     }
     return OLAP_SUCCESS;
 }
 
 bool BetaRowset::check_path(const std::string& path) {
     std::set<std::string> valid_paths;
     for (int i = 0; i < num_segments(); ++i) {
-        valid_paths.insert(segment_file_path(_rowset_path, rowset_id(), i));
+        FilePathDesc path_desc = segment_file_path(_rowset_path_desc, rowset_id(), i);
+        valid_paths.insert(path_desc.filepath);
     }
     return valid_paths.find(path) != valid_paths.end();
 }
 
 bool BetaRowset::check_file_exist() {
     for (int i = 0; i < num_segments(); ++i) {
-        std::string data_file = segment_file_path(_rowset_path, rowset_id(), i);
-        if (!FileUtils::check_exist(data_file)) {
-            LOG(WARNING) << "data file not existed: " << data_file << " for rowset_id: " << rowset_id();
+        FilePathDesc path_desc = segment_file_path(_rowset_path_desc, rowset_id(), i);
+        if (!Env::get_env(_rowset_path_desc.storage_medium)->path_exists(path_desc.filepath).ok()) {

Review comment:
       get env once

##########
File path: be/src/util/coding.cpp
##########
@@ -1,11 +1,19 @@
-//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
-//  This source code is licensed under both the GPLv2 (found in the
-//  COPYING file in the root directory) and Apache 2.0 License
-//  (found in the LICENSE.Apache file in the root directory).
+// 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
 //
-// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
-// Use of this source code is governed by a BSD-style license that can be
-// found in the LICENSE file. See the AUTHORS file for names of contributors.
+//   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.

Review comment:
       Do not modify license




-- 
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 #7098: Support remote storage, step1: use a struct instead of string for parameter path, add basic remote method

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



##########
File path: be/src/env/env_remote.cpp
##########
@@ -0,0 +1,333 @@
+//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.

Review comment:
       header is incorrect

##########
File path: be/src/env/env_remote.cpp
##########
@@ -0,0 +1,333 @@
+//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
+//  This source code is licensed under both the GPLv2 (found in the
+//  COPYING file in the root directory) and Apache 2.0 License
+//  (found in the LICENSE.Apache file in the root directory).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors
+
+#include "common/logging.h"
+#include "common/status.h"
+#include "env/env.h"
+#include "env/env_remote.h"
+#include "util/s3_storage_backend.h"
+#include "util/s3_util.h"
+
+#include "gutil/strings/substitute.h"
+
+namespace doris {
+
+using std::string;
+using strings::Substitute;
+
+std::map<std::string, std::string> RemoteEnv::_storage_prop;
+
+class RemoteRandomAccessFile : public RandomAccessFile {
+public:
+    RemoteRandomAccessFile(std::string filename, std::map<std::string, std::string> storage_prop)
+            : _filename(std::move(filename)), _storage_backend(new S3StorageBackend(storage_prop)) {}
+    ~RemoteRandomAccessFile() {
+    }
+
+    Status read_at(uint64_t offset, const Slice* result) const override {
+        return readv_at(offset, result, 1);
+    }
+
+    Status readv_at(uint64_t offset, const Slice* result, size_t res_cnt) const override {
+        return Status::IOError("No support", 1, "");

Review comment:
       ```suggestion
           return Status::NotSupport("No support", 1, "");
   ```

##########
File path: be/src/olap/data_dir.cpp
##########
@@ -77,9 +74,13 @@ DataDir::DataDir(const std::string& path, int64_t capacity_bytes,
           _tablet_manager(tablet_manager),
           _txn_manager(txn_manager),
           _cluster_id(-1),
+          _cluster_id_incomplete(false),
           _to_be_deleted(false),
           _current_shard(0),
           _meta(nullptr) {
+    _env = Env::get_env(storage_medium);
+    _path_desc.storage_medium = storage_medium;

Review comment:
       put these to initialize list

##########
File path: be/src/olap/fs/block_manager.h
##########
@@ -229,7 +236,7 @@ class BlockManager {
     // may fail.
     //
     // Does not modify 'block' on error.
-    virtual Status open_block(const std::string& path, std::unique_ptr<ReadableBlock>* block) = 0;
+    virtual Status open_block(FilePathDesc path_desc, std::unique_ptr<ReadableBlock>* block) = 0;

Review comment:
       ```suggestion
       virtual Status open_block(const FilePathDesc& path_desc, std::unique_ptr<ReadableBlock>* block) = 0;
   ```

##########
File path: be/src/olap/fs/file_block_manager.cpp
##########
@@ -63,7 +63,7 @@ namespace internal {
 // FileWritableBlock instances is expected to be low.
 class FileWritableBlock : public WritableBlock {
 public:
-    FileWritableBlock(FileBlockManager* block_manager, string path,
+    FileWritableBlock(FileBlockManager* block_manager, FilePathDesc path_desc,

Review comment:
       ```suggestion
       FileWritableBlock(FileBlockManager* block_manager, const FilePathDesc& path_desc,
   ```

##########
File path: be/src/util/file_utils.cpp
##########
@@ -41,58 +41,20 @@ namespace doris {
 using strings::Substitute;
 
 Status FileUtils::create_dir(const std::string& path, Env* env) {
-    if (path.empty()) {
-        return Status::InvalidArgument(strings::Substitute("Unknown primitive type($0)", path));
-    }
-
-    std::filesystem::path p(path);
-
-    std::string partial_path;
-    for (std::filesystem::path::iterator it = p.begin(); it != p.end(); ++it) {
-        partial_path = partial_path + it->string() + "/";
-        bool is_dir = false;
-
-        Status s = env->is_directory(partial_path, &is_dir);
-
-        if (s.ok()) {
-            if (is_dir) {
-                // It's a normal directory.
-                continue;
-            }
-
-            // Maybe a file or a symlink. Let's try to follow the symlink.
-            std::string real_partial_path;
-            RETURN_IF_ERROR(env->canonicalize(partial_path, &real_partial_path));
-
-            RETURN_IF_ERROR(env->is_directory(real_partial_path, &is_dir));
-            if (is_dir) {
-                // It's a symlink to a directory.
-                continue;
-            } else {
-                return Status::IOError(partial_path + " exists but is not a directory");
-            }
-        }
-
-        RETURN_IF_ERROR(env->create_dir_if_missing(partial_path));
-    }
-
-    return Status::OK();
+    return env->create_dirs(path);
 }
 
 Status FileUtils::create_dir(const std::string& dir_path) {
     return create_dir(dir_path, Env::Default());
 }
 
 Status FileUtils::remove_all(const std::string& file_path) {
-    std::filesystem::path boost_path(file_path);
-    std::error_code ec;
-    std::filesystem::remove_all(boost_path, ec);
-    if (ec) {
-        std::stringstream ss;
-        ss << "remove all(" << file_path << ") failed, because: " << ec;
-        return Status::InternalError(ss.str());
-    }
-    return Status::OK();
+    return remove_all(file_path, TStorageMedium::SSD);

Review comment:
       why default is SSD

##########
File path: be/src/olap/data_dir.h
##########
@@ -130,17 +136,19 @@ class DataDir {
 
     void disks_compaction_num_increment(int64_t delta);
 
+    Env* env() {
+        return _env;
+    }
+
 private:
-    std::string _cluster_id_path() const { return _path + CLUSTER_ID_PREFIX; }
     Status _init_cluster_id();
     Status _init_capacity();
-    Status _init_file_system();
     Status _init_meta();
 
     Status _check_disk();
     OLAPStatus _read_and_write_test_file();
-    Status _read_cluster_id(const std::string& cluster_id_path, int32_t* cluster_id);
-    Status _write_cluster_id_to_path(const std::string& path, int32_t cluster_id);
+    Status read_cluster_id(Env* env, const std::string& cluster_id_path, int32_t* cluster_id);

Review comment:
       ```suggestion
       Status _read_cluster_id(Env* env, const std::string& cluster_id_path, int32_t* cluster_id);
   ```

##########
File path: be/src/olap/fs/file_block_manager.cpp
##########
@@ -117,10 +117,10 @@ class FileWritableBlock : public WritableBlock {
     size_t _bytes_appended;
 };
 
-FileWritableBlock::FileWritableBlock(FileBlockManager* block_manager, string path,
+FileWritableBlock::FileWritableBlock(FileBlockManager* block_manager, FilePathDesc path_desc,

Review comment:
       Better use `const FilePathDesc& path_desc` in parameter and remove `std::move`.
   Seems meaningless to use `std::move()` here.

##########
File path: be/src/olap/rowset/segment_v2/segment.cpp
##########
@@ -37,16 +37,18 @@ namespace segment_v2 {
 
 using strings::Substitute;
 
-Status Segment::open(std::string filename, uint32_t segment_id, const TabletSchema* tablet_schema,
+Status Segment::open(FilePathDesc path_desc, uint32_t segment_id, const TabletSchema* tablet_schema,
                      std::shared_ptr<Segment>* output) {
-    std::shared_ptr<Segment> segment(new Segment(std::move(filename), segment_id, tablet_schema));
-    RETURN_IF_ERROR(segment->_open());
+    std::shared_ptr<Segment> segment(new Segment(std::move(path_desc), segment_id, tablet_schema));
+    if (!Env::get_env(path_desc.storage_medium)->is_remote_env()) {

Review comment:
       path_desc has been `std::move`, can use it again.

##########
File path: be/src/env/env_remote.cpp
##########
@@ -0,0 +1,333 @@
+//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
+//  This source code is licensed under both the GPLv2 (found in the
+//  COPYING file in the root directory) and Apache 2.0 License
+//  (found in the LICENSE.Apache file in the root directory).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors
+
+#include "common/logging.h"
+#include "common/status.h"
+#include "env/env.h"
+#include "env/env_remote.h"
+#include "util/s3_storage_backend.h"
+#include "util/s3_util.h"
+
+#include "gutil/strings/substitute.h"
+
+namespace doris {
+
+using std::string;
+using strings::Substitute;
+
+std::map<std::string, std::string> RemoteEnv::_storage_prop;
+
+class RemoteRandomAccessFile : public RandomAccessFile {
+public:
+    RemoteRandomAccessFile(std::string filename, std::map<std::string, std::string> storage_prop)
+            : _filename(std::move(filename)), _storage_backend(new S3StorageBackend(storage_prop)) {}
+    ~RemoteRandomAccessFile() {
+    }
+
+    Status read_at(uint64_t offset, const Slice* result) const override {
+        return readv_at(offset, result, 1);
+    }
+
+    Status readv_at(uint64_t offset, const Slice* result, size_t res_cnt) const override {
+        return Status::IOError("No support", 1, "");

Review comment:
       Same as follow

##########
File path: be/src/env/env_remote.cpp
##########
@@ -0,0 +1,333 @@
+//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
+//  This source code is licensed under both the GPLv2 (found in the
+//  COPYING file in the root directory) and Apache 2.0 License
+//  (found in the LICENSE.Apache file in the root directory).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors
+
+#include "common/logging.h"
+#include "common/status.h"
+#include "env/env.h"
+#include "env/env_remote.h"
+#include "util/s3_storage_backend.h"
+#include "util/s3_util.h"
+
+#include "gutil/strings/substitute.h"
+
+namespace doris {
+
+using std::string;
+using strings::Substitute;
+
+std::map<std::string, std::string> RemoteEnv::_storage_prop;
+
+class RemoteRandomAccessFile : public RandomAccessFile {
+public:
+    RemoteRandomAccessFile(std::string filename, std::map<std::string, std::string> storage_prop)
+            : _filename(std::move(filename)), _storage_backend(new S3StorageBackend(storage_prop)) {}
+    ~RemoteRandomAccessFile() {
+    }
+
+    Status read_at(uint64_t offset, const Slice* result) const override {
+        return readv_at(offset, result, 1);
+    }
+
+    Status readv_at(uint64_t offset, const Slice* result, size_t res_cnt) const override {
+        return Status::IOError("No support", 1, "");
+    }
+    Status read_all(std::string* content) const override {
+        return _storage_backend->direct_download(_filename, content);
+    }
+    Status size(uint64_t* size) const override {
+        return Status::IOError("No support", 1, "");
+    }
+
+    const std::string& file_name() const override { return _filename; }
+
+private:
+    const std::string _filename;
+    std::unique_ptr<StorageBackend> _storage_backend;
+};
+
+class RemoteWritableFile : public WritableFile {
+public:
+    RemoteWritableFile(std::string filename, std::map<std::string, std::string> storage_prop, uint64_t filesize)
+            : _filename(std::move(filename)),
+              _storage_backend(new S3StorageBackend(storage_prop)),
+              _filesize(filesize) {}
+
+    ~RemoteWritableFile() override {
+        WARN_IF_ERROR(close(), "Failed to close file, file=" + _filename);
+    }
+
+    Status append(const Slice& data) override { return appendv(&data, 1); }
+
+    Status appendv(const Slice* data, size_t data_cnt) override {
+        size_t bytes_written = 0;
+        std::string content;
+        for (size_t i = 0; i < data_cnt; i++) {
+            content += data[i].to_string();
+            bytes_written += data[i].size;
+        }
+        Status status = _storage_backend->direct_upload(_filename, content);
+        RETURN_NOT_OK_STATUS_WITH_WARN(status, strings::Substitute(
+                "direct_upload failed: $0, err=$1", _filename, status.to_string()));
+        _filesize += bytes_written;
+        return Status::OK();
+    }
+
+    Status pre_allocate(uint64_t size) override {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status close() override {
+        return Status::OK();
+    }
+
+    Status flush(FlushMode mode) override {
+        return Status::OK();
+    }
+
+    Status sync() override {
+        return Status::OK();
+    }
+
+    uint64_t size() const override { return _filesize; }
+    const string& filename() const override { return _filename; }
+
+private:
+    std::string _filename;
+    std::unique_ptr<StorageBackend> _storage_backend;
+    uint64_t _filesize = 0;
+};
+
+class RemoteRandomRWFile : public RandomRWFile {
+public:
+    RemoteRandomRWFile(const FilePathDesc& path_desc) : _path_desc(path_desc) {}
+
+    ~RemoteRandomRWFile() { WARN_IF_ERROR(close(), "Failed to close " + _path_desc.filepath); }
+
+    virtual Status read_at(uint64_t offset, const Slice& result) const {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status readv_at(uint64_t offset, const Slice* res, size_t res_cnt) const {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status write_at(uint64_t offset, const Slice& data) {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status writev_at(uint64_t offset, const Slice* data, size_t data_cnt) {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status flush(FlushMode mode, uint64_t offset, size_t length) {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status sync() {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status close() {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status size(uint64_t* size) const {
+        return Status::IOError("No support", 1, "");
+    }
+
+    const string& filename() const { return _path_desc.filepath; }
+
+private:
+    const FilePathDesc _path_desc;
+    const bool _sync_on_close = false;
+    bool _closed = false;
+};
+
+void RemoteEnv::init_s3_conf(const std::string& ak, const std::string& sk, const std::string& endpoint,
+                             const std::string& region) {
+    _storage_prop[S3_AK] = ak;
+    _storage_prop[S3_SK] = sk;
+    _storage_prop[S3_ENDPOINT] = endpoint;
+    _storage_prop[S3_REGION] = region;
+}
+
+Status RemoteEnv::new_sequential_file(const std::string& fname,
+                                      std::unique_ptr<SequentialFile>* result) {
+    return Status::IOError(strings::Substitute("Unable to new_sequential_file $0", fname), 0, "");
+}
+
+// get a RandomAccessFile pointer without file cache
+Status RemoteEnv::new_random_access_file(const std::string& fname,
+                                         std::unique_ptr<RandomAccessFile>* result) {
+    return new_random_access_file(RandomAccessFileOptions(), fname, result);
+}
+
+Status RemoteEnv::new_random_access_file(const RandomAccessFileOptions& opts, const std::string& fname,
+                                         std::unique_ptr<RandomAccessFile>* result) {
+    result->reset(new RemoteRandomAccessFile(fname, _storage_prop));
+    return Status::OK();
+}
+
+Status RemoteEnv::new_writable_file(const std::string& fname, std::unique_ptr<WritableFile>* result) {
+    return new_writable_file(WritableFileOptions(), fname, result);
+}
+
+Status RemoteEnv::new_writable_file(const WritableFileOptions& opts, const std::string& fname,
+                                    std::unique_ptr<WritableFile>* result) {
+    uint64_t file_size = 0;
+    if (opts.mode == MUST_EXIST) {
+        RETURN_IF_ERROR(get_file_size(fname, &file_size));
+    }
+    result->reset(new RemoteWritableFile(fname, _storage_prop, file_size));
+    return Status::OK();
+}
+
+Status RemoteEnv::new_random_rw_file(const std::string& fname, std::unique_ptr<RandomRWFile>* result) {
+    return new_random_rw_file(RandomRWFileOptions(), fname, result);
+}
+
+Status RemoteEnv::new_random_rw_file(const RandomRWFileOptions& opts, const std::string& fname,
+                                     std::unique_ptr<RandomRWFile>* result) {
+    return Status::IOError(strings::Substitute("Unable to new_random_rw_file $0", fname), 0, "");
+}
+
+Status RemoteEnv::path_exists(const std::string& fname, bool is_dir) {
+    std::unique_ptr<StorageBackend> storage_backend(new S3StorageBackend(_storage_prop));
+    Status status = Status::OK();
+    if (is_dir) {
+        status = storage_backend->exist_dir(fname);
+    } else {
+        status = storage_backend->exist(fname);
+    }
+    RETURN_NOT_OK_STATUS_WITH_WARN(status, strings::Substitute(
+            "path_exists failed: $0, err=$1", fname, status.to_string()));
+    return Status::OK();
+}
+
+Status RemoteEnv::get_children(const std::string& dir, std::vector<std::string>* result) {
+    return Status::IOError(strings::Substitute("Unable to get_children $0", dir), 0, "");
+}
+
+Status RemoteEnv::iterate_dir(const std::string& dir,
+                              const std::function<bool(const char*)>& cb) {
+    return Status::IOError(strings::Substitute("Unable to iterate_dir $0", dir), 0, "");
+}
+
+Status RemoteEnv::delete_file(const std::string& fname) {
+    std::unique_ptr<StorageBackend> storage_backend(new S3StorageBackend(_storage_prop));
+    Status status = storage_backend->rm(fname);
+    RETURN_NOT_OK_STATUS_WITH_WARN(status, strings::Substitute(
+            "delete_file failed: $0, err=$1", fname, status.to_string()));
+    return Status::OK();
+}
+
+Status RemoteEnv::create_dir(const std::string& name) {
+    return Status::OK();
+}
+
+Status RemoteEnv::create_dir_if_missing(const string& dirname, bool* created) {
+    *created = true;
+    return Status::OK();
+}
+
+Status RemoteEnv::create_dirs(const string& dirname) {
+    return Status::OK();
+}
+
+// Delete the specified directory.
+Status RemoteEnv::delete_dir(const std::string& dirname) {
+    std::unique_ptr<StorageBackend> storage_backend(new S3StorageBackend(_storage_prop));
+    Status status = storage_backend->rmdir(dirname);
+    RETURN_NOT_OK_STATUS_WITH_WARN(status, strings::Substitute(
+            "delete_dir failed: $0, err=$1", dirname, status.to_string()));
+    return Status::OK();
+}
+
+Status RemoteEnv::sync_dir(const string& dirname) {
+    return Status::IOError(strings::Substitute("Unable to sync_dir $0", dirname), 0, "");
+}
+
+Status RemoteEnv::is_directory(const std::string& path, bool* is_dir) {
+    std::unique_ptr<StorageBackend> storage_backend(new S3StorageBackend(_storage_prop));
+    Status status = storage_backend->exist(path);
+    if (status.ok()) {
+        *is_dir = false;
+        return Status::OK();
+    }
+    if (!status.is_not_found()) {
+        return status;
+    }
+
+    status = storage_backend->exist_dir(path);
+    if (status.ok()) {
+        *is_dir = true;
+        return Status::OK();
+    }
+    if (!status.is_not_found()) {
+        return status;
+    }
+
+    *is_dir = false;
+    return Status::OK();
+}
+
+Status RemoteEnv::canonicalize(const std::string& path, std::string* result) {
+    *result = path;
+    return Status::OK();
+}
+
+Status RemoteEnv::get_file_size(const std::string& fname, uint64_t* size) {
+    return Status::OK();
+    // return EnvBos::get_file_size(fname, size);
+}
+
+Status RemoteEnv::get_file_modified_time(const std::string& fname, uint64_t* file_mtime) {
+    return Status::IOError(strings::Substitute("Unable to get_file_modified_time $0", fname), 0, "");
+}
+
+Status RemoteEnv::copy_path(const std::string& src, const std::string& target) {
+    return Status::IOError(strings::Substitute("Unable to copy_path $0 to $1", src, target), 0, "");
+}
+
+Status RemoteEnv::rename_file(const std::string& src, const std::string& target) {
+    std::unique_ptr<StorageBackend> storage_backend(new S3StorageBackend(_storage_prop));

Review comment:
       why we need to create a new S3StorageBackend each time we call this method?

##########
File path: be/src/env/env_remote.cpp
##########
@@ -0,0 +1,333 @@
+//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
+//  This source code is licensed under both the GPLv2 (found in the
+//  COPYING file in the root directory) and Apache 2.0 License
+//  (found in the LICENSE.Apache file in the root directory).
+//
+// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
+// Use of this source code is governed by a BSD-style license that can be
+// found in the LICENSE file. See the AUTHORS file for names of contributors
+
+#include "common/logging.h"
+#include "common/status.h"
+#include "env/env.h"
+#include "env/env_remote.h"
+#include "util/s3_storage_backend.h"
+#include "util/s3_util.h"
+
+#include "gutil/strings/substitute.h"
+
+namespace doris {
+
+using std::string;
+using strings::Substitute;
+
+std::map<std::string, std::string> RemoteEnv::_storage_prop;
+
+class RemoteRandomAccessFile : public RandomAccessFile {
+public:
+    RemoteRandomAccessFile(std::string filename, std::map<std::string, std::string> storage_prop)
+            : _filename(std::move(filename)), _storage_backend(new S3StorageBackend(storage_prop)) {}
+    ~RemoteRandomAccessFile() {
+    }
+
+    Status read_at(uint64_t offset, const Slice* result) const override {
+        return readv_at(offset, result, 1);
+    }
+
+    Status readv_at(uint64_t offset, const Slice* result, size_t res_cnt) const override {
+        return Status::IOError("No support", 1, "");
+    }
+    Status read_all(std::string* content) const override {
+        return _storage_backend->direct_download(_filename, content);
+    }
+    Status size(uint64_t* size) const override {
+        return Status::IOError("No support", 1, "");
+    }
+
+    const std::string& file_name() const override { return _filename; }
+
+private:
+    const std::string _filename;
+    std::unique_ptr<StorageBackend> _storage_backend;
+};
+
+class RemoteWritableFile : public WritableFile {
+public:
+    RemoteWritableFile(std::string filename, std::map<std::string, std::string> storage_prop, uint64_t filesize)
+            : _filename(std::move(filename)),
+              _storage_backend(new S3StorageBackend(storage_prop)),
+              _filesize(filesize) {}
+
+    ~RemoteWritableFile() override {
+        WARN_IF_ERROR(close(), "Failed to close file, file=" + _filename);
+    }
+
+    Status append(const Slice& data) override { return appendv(&data, 1); }
+
+    Status appendv(const Slice* data, size_t data_cnt) override {
+        size_t bytes_written = 0;
+        std::string content;
+        for (size_t i = 0; i < data_cnt; i++) {
+            content += data[i].to_string();
+            bytes_written += data[i].size;
+        }
+        Status status = _storage_backend->direct_upload(_filename, content);
+        RETURN_NOT_OK_STATUS_WITH_WARN(status, strings::Substitute(
+                "direct_upload failed: $0, err=$1", _filename, status.to_string()));
+        _filesize += bytes_written;
+        return Status::OK();
+    }
+
+    Status pre_allocate(uint64_t size) override {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status close() override {
+        return Status::OK();
+    }
+
+    Status flush(FlushMode mode) override {
+        return Status::OK();
+    }
+
+    Status sync() override {
+        return Status::OK();
+    }
+
+    uint64_t size() const override { return _filesize; }
+    const string& filename() const override { return _filename; }
+
+private:
+    std::string _filename;
+    std::unique_ptr<StorageBackend> _storage_backend;
+    uint64_t _filesize = 0;
+};
+
+class RemoteRandomRWFile : public RandomRWFile {
+public:
+    RemoteRandomRWFile(const FilePathDesc& path_desc) : _path_desc(path_desc) {}
+
+    ~RemoteRandomRWFile() { WARN_IF_ERROR(close(), "Failed to close " + _path_desc.filepath); }
+
+    virtual Status read_at(uint64_t offset, const Slice& result) const {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status readv_at(uint64_t offset, const Slice* res, size_t res_cnt) const {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status write_at(uint64_t offset, const Slice& data) {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status writev_at(uint64_t offset, const Slice* data, size_t data_cnt) {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status flush(FlushMode mode, uint64_t offset, size_t length) {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status sync() {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status close() {
+        return Status::IOError("No support", 1, "");
+    }
+
+    Status size(uint64_t* size) const {
+        return Status::IOError("No support", 1, "");
+    }
+
+    const string& filename() const { return _path_desc.filepath; }
+
+private:
+    const FilePathDesc _path_desc;
+    const bool _sync_on_close = false;
+    bool _closed = false;
+};
+
+void RemoteEnv::init_s3_conf(const std::string& ak, const std::string& sk, const std::string& endpoint,
+                             const std::string& region) {
+    _storage_prop[S3_AK] = ak;
+    _storage_prop[S3_SK] = sk;
+    _storage_prop[S3_ENDPOINT] = endpoint;
+    _storage_prop[S3_REGION] = region;
+}
+
+Status RemoteEnv::new_sequential_file(const std::string& fname,
+                                      std::unique_ptr<SequentialFile>* result) {
+    return Status::IOError(strings::Substitute("Unable to new_sequential_file $0", fname), 0, "");
+}
+
+// get a RandomAccessFile pointer without file cache
+Status RemoteEnv::new_random_access_file(const std::string& fname,
+                                         std::unique_ptr<RandomAccessFile>* result) {
+    return new_random_access_file(RandomAccessFileOptions(), fname, result);
+}
+
+Status RemoteEnv::new_random_access_file(const RandomAccessFileOptions& opts, const std::string& fname,
+                                         std::unique_ptr<RandomAccessFile>* result) {
+    result->reset(new RemoteRandomAccessFile(fname, _storage_prop));
+    return Status::OK();
+}
+
+Status RemoteEnv::new_writable_file(const std::string& fname, std::unique_ptr<WritableFile>* result) {
+    return new_writable_file(WritableFileOptions(), fname, result);
+}
+
+Status RemoteEnv::new_writable_file(const WritableFileOptions& opts, const std::string& fname,
+                                    std::unique_ptr<WritableFile>* result) {
+    uint64_t file_size = 0;
+    if (opts.mode == MUST_EXIST) {
+        RETURN_IF_ERROR(get_file_size(fname, &file_size));
+    }
+    result->reset(new RemoteWritableFile(fname, _storage_prop, file_size));
+    return Status::OK();
+}
+
+Status RemoteEnv::new_random_rw_file(const std::string& fname, std::unique_ptr<RandomRWFile>* result) {
+    return new_random_rw_file(RandomRWFileOptions(), fname, result);
+}
+
+Status RemoteEnv::new_random_rw_file(const RandomRWFileOptions& opts, const std::string& fname,
+                                     std::unique_ptr<RandomRWFile>* result) {
+    return Status::IOError(strings::Substitute("Unable to new_random_rw_file $0", fname), 0, "");
+}
+
+Status RemoteEnv::path_exists(const std::string& fname, bool is_dir) {
+    std::unique_ptr<StorageBackend> storage_backend(new S3StorageBackend(_storage_prop));
+    Status status = Status::OK();
+    if (is_dir) {
+        status = storage_backend->exist_dir(fname);
+    } else {
+        status = storage_backend->exist(fname);
+    }
+    RETURN_NOT_OK_STATUS_WITH_WARN(status, strings::Substitute(
+            "path_exists failed: $0, err=$1", fname, status.to_string()));
+    return Status::OK();
+}
+
+Status RemoteEnv::get_children(const std::string& dir, std::vector<std::string>* result) {
+    return Status::IOError(strings::Substitute("Unable to get_children $0", dir), 0, "");
+}
+
+Status RemoteEnv::iterate_dir(const std::string& dir,
+                              const std::function<bool(const char*)>& cb) {
+    return Status::IOError(strings::Substitute("Unable to iterate_dir $0", dir), 0, "");
+}
+
+Status RemoteEnv::delete_file(const std::string& fname) {
+    std::unique_ptr<StorageBackend> storage_backend(new S3StorageBackend(_storage_prop));
+    Status status = storage_backend->rm(fname);
+    RETURN_NOT_OK_STATUS_WITH_WARN(status, strings::Substitute(
+            "delete_file failed: $0, err=$1", fname, status.to_string()));
+    return Status::OK();
+}
+
+Status RemoteEnv::create_dir(const std::string& name) {

Review comment:
       Add comment to explain why directly return 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] pengxiangyu commented on a change in pull request #7098: Support remote storage, step1: use a struct instead of string for parameter path, add basic remote method

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



##########
File path: be/src/env/env_posix.cpp
##########
@@ -266,13 +277,38 @@ class PosixRandomAccessFile : public RandomAccessFile {
         }
     }
 
-    Status read_at(uint64_t offset, const Slice& result) const override {
-        return do_readv_at(_fd, _filename, offset, &result, 1);
+    Status read_at(uint64_t offset, const Slice* result) const override {
+        return readv_at(offset, result, 1);
+    }
+
+    Status readv_at(uint64_t offset, const Slice* result, size_t res_cnt) const override {
+        return do_readv_at(_fd, _filename, offset, result, res_cnt);
     }
 
-    Status readv_at(uint64_t offset, const Slice* res, size_t res_cnt) const override {
-        return do_readv_at(_fd, _filename, offset, res, res_cnt);
+    Status read_all(std::string* content) const override {
+        std::fstream fs(_filename.c_str(), std::fstream::in);
+        if (!fs.is_open()) {
+            RETURN_NOT_OK_STATUS_WITH_WARN(
+                    Status::IOError(
+                            strings::Substitute("failed to open cluster id file $0", _filename)),
+                    "open file failed");
+        }
+        std::string data;
+        fs >> data;
+        fs.close();
+        if ((fs.rdstate() & std::fstream::eofbit) != 0) {
+            *content = data;

Review comment:
       *content = data;  will take the string, these codes are not changed, they are only moved.




-- 
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] Gabriel39 commented on pull request #7098: Support remote storage, step1: use a struct instead of string for parameter path, add basic remote method

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


   Great job and it's a huge feature!  Btw, is there a design doc for this PR? I think it will be helpful for reviewers


-- 
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