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 2022/05/11 10:40:23 UTC

[GitHub] [incubator-doris] HappenLee opened a new pull request, #9505: [Refactor] Use file factory to replace create file reader/writer

HappenLee opened a new pull request, #9505:
URL: https://github.com/apache/incubator-doris/pull/9505

   1. Simplify code logic and improve abstraction
   2. Fix the mem leak of raw pointer
   
   # Proposed changes
   
   Issue Number: close #xxx
   
   ## Problem Summary:
   
   Describe the overview of changes.
   
   ## Checklist(Required)
   
   1. Does it affect the original behavior: (No)
   2. Has unit tests been added: (No Need)
   3. Has document been added or modified: (No Need)
   4. Does it need to update dependencies: (No)
   5. Are there any changes that cannot be rolled back: (Yes)
   
   ## Further comments
   
   If this is a relatively large or complex change, kick off the discussion at [dev@doris.apache.org](mailto:dev@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc...
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

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


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


[GitHub] [incubator-doris] HappenLee commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870913059


##########
be/src/exec/broker_scanner.h:
##########
@@ -110,7 +110,7 @@ class BrokerScanner : public BaseScanner {
     int _line_delimiter_length;
 
     // Reader
-    FileReader* _cur_file_reader;
+    std::shared_ptr<FileReader> _cur_file_reader;

Review Comment:
   not sharable



-- 
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] adonis0147 commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
adonis0147 commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870348299


##########
be/src/exec/file/file_factory.h:
##########
@@ -0,0 +1,74 @@
+// 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 "gen_cpp/Types_types.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "file_reader.h"
+#include "file_writer.h"
+
+namespace doris {
+class ExecEnv;
+class TNetworkAddress;
+
+class FileFactory {
+public:
+    static Status create_file_writer(TFileType::type type, ExecEnv* env,
+                                     const std::vector<TNetworkAddress>& broker_addresses,
+                                     const std::map<std::string, std::string>& properties,
+                                     const std::string& path, int64_t start_offset,
+                                     std::unique_ptr<FileWriter>& file_writer);
+
+    // Because StreamLoadPipe use std::shared_ptr, here we have to support both unique_ptr
+    // and shared_ptr create_file_reader
+    static Status create_file_reader(TFileType::type type, ExecEnv* env,
+                                     const std::vector<TNetworkAddress>& broker_addresses,
+                                     const std::map<std::string, std::string>& properties,
+                                     const TBrokerRangeDesc& range, int64_t start_offset,
+                                     std::unique_ptr<FileReader>& file_reader);
+
+    static Status create_file_reader(TFileType::type type, ExecEnv* env,
+                                     const std::vector<TNetworkAddress>& broker_addresses,
+                                     const std::map<std::string, std::string>& properties,
+                                     const TBrokerRangeDesc& range, int64_t start_offset,
+                                     std::shared_ptr<FileReader>& file_reader);
+
+    static TFileType::type convert_storage_type(TStorageBackendType::type type) {
+        switch (type) {
+        case TStorageBackendType::LOCAL:
+            return TFileType::FILE_LOCAL;
+        case TStorageBackendType::S3:
+            return TFileType::FILE_S3;
+        case TStorageBackendType::BROKER:
+            return TFileType::FILE_BROKER;
+        case TStorageBackendType::HDFS:
+            return TFileType::FILE_HDFS;
+        default:
+            LOG(FATAL) << "not match type to convert, from type:" << type;
+        }
+        __builtin_unreachable();
+    }
+
+private:
+    static Status _new_file_reader(TFileType::type type, ExecEnv* env,
+                                   const std::vector<TNetworkAddress>& broker_addresses,
+                                   const std::map<std::string, std::string>& properties,
+                                   const TBrokerRangeDesc& range, int64_t start_offset,
+                                   FileReader*& file_reader);

Review Comment:
   ```suggestion
       static Status _new_file_reader(TFileType::type type, ExecEnv* env,
                                      const std::vector<TNetworkAddress>& broker_addresses,
                                      const std::map<std::string, std::string>& properties,
                                      const TBrokerRangeDesc& range, int64_t start_offset,
                                      std::unique_ptr<FileReader>& file_reader);
   ```
   I think it is better to avoid the raw pointer and use `std::move` to pass the ownership of the `file_reader` outside.
   Users can figure the ownership of `file_reader` out from the interface declaration directly by using smart pointers. Otherwise, you would better to add the comment to indicate the caller should be responsible to the resources allocated for `file_reader`.



-- 
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] HappenLee commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870917316


##########
be/src/exec/file/file_factory.h:
##########
@@ -0,0 +1,74 @@
+// 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 "gen_cpp/Types_types.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "file_reader.h"
+#include "file_writer.h"
+
+namespace doris {
+class ExecEnv;
+class TNetworkAddress;
+
+class FileFactory {
+public:
+    static Status create_file_writer(TFileType::type type, ExecEnv* env,
+                                     const std::vector<TNetworkAddress>& broker_addresses,
+                                     const std::map<std::string, std::string>& properties,
+                                     const std::string& path, int64_t start_offset,
+                                     std::unique_ptr<FileWriter>& file_writer);
+
+    // Because StreamLoadPipe use std::shared_ptr, here we have to support both unique_ptr
+    // and shared_ptr create_file_reader
+    static Status create_file_reader(TFileType::type type, ExecEnv* env,
+                                     const std::vector<TNetworkAddress>& broker_addresses,
+                                     const std::map<std::string, std::string>& properties,
+                                     const TBrokerRangeDesc& range, int64_t start_offset,
+                                     std::unique_ptr<FileReader>& file_reader);
+
+    static Status create_file_reader(TFileType::type type, ExecEnv* env,
+                                     const std::vector<TNetworkAddress>& broker_addresses,
+                                     const std::map<std::string, std::string>& properties,
+                                     const TBrokerRangeDesc& range, int64_t start_offset,
+                                     std::shared_ptr<FileReader>& file_reader);
+
+    static TFileType::type convert_storage_type(TStorageBackendType::type type) {
+        switch (type) {
+        case TStorageBackendType::LOCAL:
+            return TFileType::FILE_LOCAL;
+        case TStorageBackendType::S3:
+            return TFileType::FILE_S3;
+        case TStorageBackendType::BROKER:
+            return TFileType::FILE_BROKER;
+        case TStorageBackendType::HDFS:
+            return TFileType::FILE_HDFS;
+        default:
+            LOG(FATAL) << "not match type to convert, from type:" << type;
+        }
+        __builtin_unreachable();
+    }
+
+private:
+    static Status _new_file_reader(TFileType::type type, ExecEnv* env,
+                                   const std::vector<TNetworkAddress>& broker_addresses,
+                                   const std::map<std::string, std::string>& properties,
+                                   const TBrokerRangeDesc& range, int64_t start_offset,
+                                   FileReader*& file_reader);

Review Comment:
   Yes, you are right. But please note that this is a `private` method, which itself should not directly called by users. There's no need to worry about what you said



-- 
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] HappenLee commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870264705


##########
be/src/exec/broker_scanner.cpp:
##########
@@ -158,53 +143,11 @@ Status BrokerScanner::open_file_reader() {
             _skip_lines = 2;
         }
     }
-    switch (range.file_type) {
-    case TFileType::FILE_LOCAL: {
-        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
-        RETURN_IF_ERROR(file_reader->open());
-        _cur_file_reader = file_reader;
-        break;
-    }
-    case TFileType::FILE_HDFS: {
-        FileReader* hdfs_file_reader;
-        RETURN_IF_ERROR(HdfsReaderWriter::create_reader(range.hdfs_params, range.path, start_offset,
-                                                        &hdfs_file_reader));
-        BufferedReader* file_reader = new BufferedReader(_profile, hdfs_file_reader);
-        RETURN_IF_ERROR(file_reader->open());
-        _cur_file_reader = file_reader;
-        break;
-    }
-    case TFileType::FILE_BROKER: {
-        BrokerReader* broker_reader =
-                new BrokerReader(_state->exec_env(), _broker_addresses, _params.properties,
-                                 range.path, start_offset);
-        RETURN_IF_ERROR(broker_reader->open());
-        _cur_file_reader = broker_reader;
-        break;
-    }
-    case TFileType::FILE_S3: {
-        BufferedReader* s3_reader = new BufferedReader(
-                _profile, new S3Reader(_params.properties, range.path, start_offset));
-        RETURN_IF_ERROR(s3_reader->open());
-        _cur_file_reader = s3_reader;
-        break;
-    }
-    case TFileType::FILE_STREAM: {
-        _stream_load_pipe = _state->exec_env()->load_stream_mgr()->get(range.load_id);
-        if (_stream_load_pipe == nullptr) {
-            VLOG_NOTICE << "unknown stream load id: " << UniqueId(range.load_id);
-            return Status::InternalError("unknown stream load id");
-        }
-        _cur_file_reader = _stream_load_pipe.get();
-        break;
-    }
-    default: {
-        std::stringstream ss;
-        ss << "Unknown file type, type=" << range.file_type;
-        return Status::InternalError(ss.str());
-    }
-    }
-    return Status::OK();
+
+    RETURN_IF_ERROR(FileFactory::create_file_reader(range.file_type, _state->exec_env(),
+                                                    _broker_addresses, _params.properties, range,
+                                                    start_offset, _cur_file_reader));

Review Comment:
   origin code seems do not have the logic?



-- 
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] yiguolei commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
yiguolei commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870161388


##########
be/src/exec/broker_scanner.cpp:
##########
@@ -158,53 +143,11 @@ Status BrokerScanner::open_file_reader() {
             _skip_lines = 2;
         }
     }
-    switch (range.file_type) {

Review Comment:
   I think it is better to use stream type not file type. 
   FileType maybe parquet, orc, csv
   StreamType could be S3Stream, HDFSStream, LocalStream



-- 
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] HappenLee commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870268346


##########
be/src/exec/broker_scanner.cpp:
##########
@@ -158,53 +143,11 @@ Status BrokerScanner::open_file_reader() {
             _skip_lines = 2;
         }
     }
-    switch (range.file_type) {

Review Comment:
   Since the original code set the type in thrift, this name must be reserved for compatibility



-- 
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] HappenLee commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870265020


##########
be/src/exec/file/broker_reader.cpp:
##########
@@ -15,7 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "exec/broker_reader.h"

Review Comment:
   ok,I will do the work



-- 
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] HappenLee commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870927981


##########
be/src/exec/broker_scanner.h:
##########
@@ -110,7 +110,7 @@ class BrokerScanner : public BaseScanner {
     int _line_delimiter_length;
 
     // Reader
-    FileReader* _cur_file_reader;
+    std::shared_ptr<FileReader> _cur_file_reader;

Review Comment:
   note the `std::shared_ptr<StreamLoadPipe> _stream_load_pipe;`, here use the shared_ptr, so now we can't not use other pointer to manage the lifetime of the `FileReader* _cur_file_reader;`. 
   only the `shared_ptr`
   



-- 
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] xy720 commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
xy720 commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870188579


##########
be/src/exec/broker_scanner.cpp:
##########
@@ -158,53 +143,11 @@ Status BrokerScanner::open_file_reader() {
             _skip_lines = 2;
         }
     }
-    switch (range.file_type) {
-    case TFileType::FILE_LOCAL: {
-        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
-        RETURN_IF_ERROR(file_reader->open());
-        _cur_file_reader = file_reader;
-        break;
-    }
-    case TFileType::FILE_HDFS: {
-        FileReader* hdfs_file_reader;
-        RETURN_IF_ERROR(HdfsReaderWriter::create_reader(range.hdfs_params, range.path, start_offset,
-                                                        &hdfs_file_reader));
-        BufferedReader* file_reader = new BufferedReader(_profile, hdfs_file_reader);
-        RETURN_IF_ERROR(file_reader->open());
-        _cur_file_reader = file_reader;
-        break;
-    }
-    case TFileType::FILE_BROKER: {
-        BrokerReader* broker_reader =
-                new BrokerReader(_state->exec_env(), _broker_addresses, _params.properties,
-                                 range.path, start_offset);
-        RETURN_IF_ERROR(broker_reader->open());
-        _cur_file_reader = broker_reader;
-        break;
-    }
-    case TFileType::FILE_S3: {
-        BufferedReader* s3_reader = new BufferedReader(
-                _profile, new S3Reader(_params.properties, range.path, start_offset));
-        RETURN_IF_ERROR(s3_reader->open());
-        _cur_file_reader = s3_reader;
-        break;
-    }
-    case TFileType::FILE_STREAM: {
-        _stream_load_pipe = _state->exec_env()->load_stream_mgr()->get(range.load_id);
-        if (_stream_load_pipe == nullptr) {
-            VLOG_NOTICE << "unknown stream load id: " << UniqueId(range.load_id);
-            return Status::InternalError("unknown stream load id");
-        }
-        _cur_file_reader = _stream_load_pipe.get();
-        break;
-    }
-    default: {
-        std::stringstream ss;
-        ss << "Unknown file type, type=" << range.file_type;
-        return Status::InternalError(ss.str());
-    }
-    }
-    return Status::OK();
+
+    RETURN_IF_ERROR(FileFactory::create_file_reader(range.file_type, _state->exec_env(),
+                                                    _broker_addresses, _params.properties, range,
+                                                    start_offset, _cur_file_reader));

Review Comment:
   May be you should wrap the file reader in a BufferedReader before open() ?



-- 
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] HappenLee commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870914034


##########
be/src/exec/broker_scanner.cpp:
##########
@@ -158,53 +143,11 @@ Status BrokerScanner::open_file_reader() {
             _skip_lines = 2;
         }
     }
-    switch (range.file_type) {
-    case TFileType::FILE_LOCAL: {
-        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
-        RETURN_IF_ERROR(file_reader->open());
-        _cur_file_reader = file_reader;
-        break;
-    }
-    case TFileType::FILE_HDFS: {
-        FileReader* hdfs_file_reader;
-        RETURN_IF_ERROR(HdfsReaderWriter::create_reader(range.hdfs_params, range.path, start_offset,
-                                                        &hdfs_file_reader));
-        BufferedReader* file_reader = new BufferedReader(_profile, hdfs_file_reader);

Review Comment:
   yes,it should wrapped it. i will add the logic



-- 
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] HappenLee commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870927241


##########
be/src/exec/file/file_factory.h:
##########
@@ -0,0 +1,74 @@
+// 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 "gen_cpp/Types_types.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "file_reader.h"
+#include "file_writer.h"
+
+namespace doris {
+class ExecEnv;
+class TNetworkAddress;
+
+class FileFactory {
+public:
+    static Status create_file_writer(TFileType::type type, ExecEnv* env,
+                                     const std::vector<TNetworkAddress>& broker_addresses,
+                                     const std::map<std::string, std::string>& properties,
+                                     const std::string& path, int64_t start_offset,
+                                     std::unique_ptr<FileWriter>& file_writer);
+
+    // Because StreamLoadPipe use std::shared_ptr, here we have to support both unique_ptr
+    // and shared_ptr create_file_reader
+    static Status create_file_reader(TFileType::type type, ExecEnv* env,
+                                     const std::vector<TNetworkAddress>& broker_addresses,
+                                     const std::map<std::string, std::string>& properties,
+                                     const TBrokerRangeDesc& range, int64_t start_offset,
+                                     std::unique_ptr<FileReader>& file_reader);
+
+    static Status create_file_reader(TFileType::type type, ExecEnv* env,
+                                     const std::vector<TNetworkAddress>& broker_addresses,
+                                     const std::map<std::string, std::string>& properties,
+                                     const TBrokerRangeDesc& range, int64_t start_offset,
+                                     std::shared_ptr<FileReader>& file_reader);
+
+    static TFileType::type convert_storage_type(TStorageBackendType::type type) {
+        switch (type) {
+        case TStorageBackendType::LOCAL:
+            return TFileType::FILE_LOCAL;
+        case TStorageBackendType::S3:
+            return TFileType::FILE_S3;
+        case TStorageBackendType::BROKER:
+            return TFileType::FILE_BROKER;
+        case TStorageBackendType::HDFS:
+            return TFileType::FILE_HDFS;
+        default:
+            LOG(FATAL) << "not match type to convert, from type:" << type;
+        }
+        __builtin_unreachable();
+    }
+
+private:
+    static Status _new_file_reader(TFileType::type type, ExecEnv* env,
+                                   const std::vector<TNetworkAddress>& broker_addresses,
+                                   const std::map<std::string, std::string>& properties,
+                                   const TBrokerRangeDesc& range, int64_t start_offset,
+                                   FileReader*& file_reader);

Review Comment:
   I will add comment of this function to notice the develop of class to notice the lifetime of of `file_reader`



-- 
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] yiguolei merged pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
yiguolei merged PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505


-- 
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] HappenLee commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
HappenLee commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870913059


##########
be/src/exec/broker_scanner.h:
##########
@@ -110,7 +110,7 @@ class BrokerScanner : public BaseScanner {
     int _line_delimiter_length;
 
     // Reader
-    FileReader* _cur_file_reader;
+    std::shared_ptr<FileReader> _cur_file_reader;

Review Comment:
   not sharable,just need keep the logic of  _stream_load_pipe



-- 
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] adonis0147 commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
adonis0147 commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870921127


##########
be/src/exec/file/file_factory.h:
##########
@@ -0,0 +1,74 @@
+// 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 "gen_cpp/Types_types.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "file_reader.h"
+#include "file_writer.h"
+
+namespace doris {
+class ExecEnv;
+class TNetworkAddress;
+
+class FileFactory {
+public:
+    static Status create_file_writer(TFileType::type type, ExecEnv* env,
+                                     const std::vector<TNetworkAddress>& broker_addresses,
+                                     const std::map<std::string, std::string>& properties,
+                                     const std::string& path, int64_t start_offset,
+                                     std::unique_ptr<FileWriter>& file_writer);
+
+    // Because StreamLoadPipe use std::shared_ptr, here we have to support both unique_ptr
+    // and shared_ptr create_file_reader
+    static Status create_file_reader(TFileType::type type, ExecEnv* env,
+                                     const std::vector<TNetworkAddress>& broker_addresses,
+                                     const std::map<std::string, std::string>& properties,
+                                     const TBrokerRangeDesc& range, int64_t start_offset,
+                                     std::unique_ptr<FileReader>& file_reader);
+
+    static Status create_file_reader(TFileType::type type, ExecEnv* env,
+                                     const std::vector<TNetworkAddress>& broker_addresses,
+                                     const std::map<std::string, std::string>& properties,
+                                     const TBrokerRangeDesc& range, int64_t start_offset,
+                                     std::shared_ptr<FileReader>& file_reader);
+
+    static TFileType::type convert_storage_type(TStorageBackendType::type type) {
+        switch (type) {
+        case TStorageBackendType::LOCAL:
+            return TFileType::FILE_LOCAL;
+        case TStorageBackendType::S3:
+            return TFileType::FILE_S3;
+        case TStorageBackendType::BROKER:
+            return TFileType::FILE_BROKER;
+        case TStorageBackendType::HDFS:
+            return TFileType::FILE_HDFS;
+        default:
+            LOG(FATAL) << "not match type to convert, from type:" << type;
+        }
+        __builtin_unreachable();
+    }
+
+private:
+    static Status _new_file_reader(TFileType::type type, ExecEnv* env,
+                                   const std::vector<TNetworkAddress>& broker_addresses,
+                                   const std::map<std::string, std::string>& properties,
+                                   const TBrokerRangeDesc& range, int64_t start_offset,
+                                   FileReader*& file_reader);

Review Comment:
   I think there are two roles of users:
   1. Ones who using the class.
   2. Ones who develops the class.
   
   It's ok for the users who using the class here due to the accessibility. However, for the ones who will modify the class, they should jump into the definition detail to figure the ownership out. That is inconvenient.



-- 
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] yiguolei commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
yiguolei commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870157017


##########
be/src/exec/CMakeLists.txt:
##########
@@ -92,25 +90,29 @@ set(EXEC_FILES
     partitioned_hash_table.cc
     partitioned_aggregation_node.cc
     odbc_scan_node.cpp
-    local_file_writer.cpp
-    broker_writer.cpp
     parquet_scanner.cpp
     parquet_reader.cpp
     parquet_writer.cpp
     orc_scanner.cpp
     odbc_connector.cpp
     json_scanner.cpp
     assert_num_rows_node.cpp
-    s3_reader.cpp
-    s3_writer.cpp
-    hdfs_reader_writer.cpp
+
+    file/local_file_reader.cpp
+    file/local_file_writer.cpp
+    file/broker_reader.cpp
+    file/broker_writer.cpp
+    file/s3_reader.cpp
+    file/s3_writer.cpp
+    file/hdfs_reader_writer.cpp

Review Comment:
   Why not split to hdfs_reader.cpp and hdfs_writer.cpp as local, s3, broker ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: 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] yiguolei commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
yiguolei commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870215351


##########
be/src/exec/file/broker_reader.cpp:
##########
@@ -15,7 +15,7 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "exec/broker_reader.h"

Review Comment:
   create a be/src/io folder, and move all file reader and file writers to the IO folder.



-- 
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] xy720 commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
xy720 commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870897602


##########
be/src/exec/broker_scanner.cpp:
##########
@@ -158,53 +143,11 @@ Status BrokerScanner::open_file_reader() {
             _skip_lines = 2;
         }
     }
-    switch (range.file_type) {
-    case TFileType::FILE_LOCAL: {
-        LocalFileReader* file_reader = new LocalFileReader(range.path, start_offset);
-        RETURN_IF_ERROR(file_reader->open());
-        _cur_file_reader = file_reader;
-        break;
-    }
-    case TFileType::FILE_HDFS: {
-        FileReader* hdfs_file_reader;
-        RETURN_IF_ERROR(HdfsReaderWriter::create_reader(range.hdfs_params, range.path, start_offset,
-                                                        &hdfs_file_reader));
-        BufferedReader* file_reader = new BufferedReader(_profile, hdfs_file_reader);

Review Comment:
   Here, some file_reader is wrapped in a BufferedReader.



-- 
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] adonis0147 commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
adonis0147 commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870918568


##########
be/src/exec/broker_scanner.h:
##########
@@ -110,7 +110,7 @@ class BrokerScanner : public BaseScanner {
     int _line_delimiter_length;
 
     // Reader
-    FileReader* _cur_file_reader;
+    std::shared_ptr<FileReader> _cur_file_reader;

Review Comment:
   If the ownership of `_cur_file_reader` is unsharable, using `std::unique_ptr` is better accoding to [CppCoreGuidelines](https://isocpp.github.io/CppCoreGuidelines/CppCoreGuidelines#r21-prefer-unique_ptr-over-shared_ptr-unless-you-need-to-share-ownership). 
   `std::shared_ptr` has some performance overhead. We should not mix the ownership and the access of the resource, otherwise, the `std::shared_ptr` will pollute lots of code.



##########
be/src/exec/broker_scanner.h:
##########
@@ -110,7 +110,7 @@ class BrokerScanner : public BaseScanner {
     int _line_delimiter_length;
 
     // Reader
-    FileReader* _cur_file_reader;
+    std::shared_ptr<FileReader> _cur_file_reader;

Review Comment:
   If the ownership of `_cur_file_reader` is unsharable, using `std::unique_ptr` is better according to [CppCoreGuidelines](https://isocpp.github.io/CppCoreGuidelines/CppCoreGuidelines#r21-prefer-unique_ptr-over-shared_ptr-unless-you-need-to-share-ownership). 
   `std::shared_ptr` has some performance overhead. We should not mix the ownership and the access of the resource, otherwise, the `std::shared_ptr` will pollute lots of code.



-- 
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] adonis0147 commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
adonis0147 commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870921127


##########
be/src/exec/file/file_factory.h:
##########
@@ -0,0 +1,74 @@
+// 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 "gen_cpp/Types_types.h"
+#include "gen_cpp/PlanNodes_types.h"
+#include "file_reader.h"
+#include "file_writer.h"
+
+namespace doris {
+class ExecEnv;
+class TNetworkAddress;
+
+class FileFactory {
+public:
+    static Status create_file_writer(TFileType::type type, ExecEnv* env,
+                                     const std::vector<TNetworkAddress>& broker_addresses,
+                                     const std::map<std::string, std::string>& properties,
+                                     const std::string& path, int64_t start_offset,
+                                     std::unique_ptr<FileWriter>& file_writer);
+
+    // Because StreamLoadPipe use std::shared_ptr, here we have to support both unique_ptr
+    // and shared_ptr create_file_reader
+    static Status create_file_reader(TFileType::type type, ExecEnv* env,
+                                     const std::vector<TNetworkAddress>& broker_addresses,
+                                     const std::map<std::string, std::string>& properties,
+                                     const TBrokerRangeDesc& range, int64_t start_offset,
+                                     std::unique_ptr<FileReader>& file_reader);
+
+    static Status create_file_reader(TFileType::type type, ExecEnv* env,
+                                     const std::vector<TNetworkAddress>& broker_addresses,
+                                     const std::map<std::string, std::string>& properties,
+                                     const TBrokerRangeDesc& range, int64_t start_offset,
+                                     std::shared_ptr<FileReader>& file_reader);
+
+    static TFileType::type convert_storage_type(TStorageBackendType::type type) {
+        switch (type) {
+        case TStorageBackendType::LOCAL:
+            return TFileType::FILE_LOCAL;
+        case TStorageBackendType::S3:
+            return TFileType::FILE_S3;
+        case TStorageBackendType::BROKER:
+            return TFileType::FILE_BROKER;
+        case TStorageBackendType::HDFS:
+            return TFileType::FILE_HDFS;
+        default:
+            LOG(FATAL) << "not match type to convert, from type:" << type;
+        }
+        __builtin_unreachable();
+    }
+
+private:
+    static Status _new_file_reader(TFileType::type type, ExecEnv* env,
+                                   const std::vector<TNetworkAddress>& broker_addresses,
+                                   const std::map<std::string, std::string>& properties,
+                                   const TBrokerRangeDesc& range, int64_t start_offset,
+                                   FileReader*& file_reader);

Review Comment:
   I think there are two roles of users:
   1. Ones who use the class.
   2. Ones who develop the class.
   
   It's ok for the users who using the class here due to the accessibility. However, for the ones who will modify the class, they should jump into the definition detail to figure the ownership out. That is inconvenient.



-- 
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] adonis0147 commented on a diff in pull request #9505: [Refactor] Use file factory to replace create file reader/writer

Posted by GitBox <gi...@apache.org>.
adonis0147 commented on code in PR #9505:
URL: https://github.com/apache/incubator-doris/pull/9505#discussion_r870321488


##########
be/src/exec/broker_scanner.h:
##########
@@ -110,7 +110,7 @@ class BrokerScanner : public BaseScanner {
     int _line_delimiter_length;
 
     // Reader
-    FileReader* _cur_file_reader;
+    std::shared_ptr<FileReader> _cur_file_reader;

Review Comment:
   Why `std::shared_ptr`? Is the ownership of `_cur_file_reader` sharable? 



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