dujl commented on code in PR #10402:
URL: https://github.com/apache/doris/pull/10402#discussion_r908136372


##########
gensrc/thrift/PlanNodes.thrift:
##########
@@ -213,8 +215,54 @@ struct TEsScanRange {
   4: required i32 shard_id
 }
 
-struct TFileScanRange {
+struct TFileTextScanRangeParams {
+    3: optional i8 column_separator;
+    4: optional i8 line_delimiter;
+    // for multibytes separators
+    5: optional i32 column_separator_length = 1;
+    6: optional i32 line_delimiter_length = 1;
+    7: optional string column_separator_str;
+    8: optional string line_delimiter_str;
+}
+
+struct TFileScanSlotInfo {
+    1: optional Types.TSlotId slot_id;
+    2: optional bool is_file_slot;
+}
+
+struct TFileScanRangeParams {
+  // use src_tuple_id to get all slots from src table include both file slot 
and partition slot.
+  1: optional Types.TTupleId src_tuple_id;
+  // num_of_columns_from_file can spilt the all_file_slot and 
all_partition_slot
+  2: optional i32 num_of_columns_from_file;
+  // all selected slots which may compose from file and partiton value.
+  3: optional list<TFileScanSlotInfo> required_slots;
 
+  4: optional TFileTextScanRangeParams text_params;
+}
+
+struct TFileRangeDesc {
+    1: optional Types.TFileType file_type;
+    2: optional TFileFormatType format_type;
+    // Path of this range
+    3: optional string path;
+    // Offset of this file start
+    4: optional i64 start_offset;
+    // Size of this range, if size = -1, this means that will read to then end 
of file

Review Comment:
   then->the : read to the end of file



##########
gensrc/thrift/PlanNodes.thrift:
##########
@@ -213,8 +215,54 @@ struct TEsScanRange {
   4: required i32 shard_id
 }
 
-struct TFileScanRange {
+struct TFileTextScanRangeParams {
+    3: optional i8 column_separator;
+    4: optional i8 line_delimiter;
+    // for multibytes separators
+    5: optional i32 column_separator_length = 1;
+    6: optional i32 line_delimiter_length = 1;
+    7: optional string column_separator_str;
+    8: optional string line_delimiter_str;
+}

Review Comment:
   does we support more options such as quote, escape, include header or not 
and so on?
   suggest TFileTextScanRangeParams use map<k, v> struct, as that we could 
support more options without change the TFileTextScanRangeParams



##########
be/src/vec/exec/file_text_scanner.cpp:
##########
@@ -0,0 +1,308 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "vec/exec/file_text_scanner.h"
+
+#include <fmt/format.h>
+#include <gen_cpp/internal_service.pb.h>
+
+#include <iostream>
+
+#include "exec/exec_node.h"
+#include "exec/plain_text_line_reader.h"
+#include "exec/text_converter.h"
+#include "exec/text_converter.hpp"
+#include "exprs/expr_context.h"
+#include "io/buffered_reader.h"
+#include "io/hdfs_reader_writer.h"
+#include "util/types.h"
+#include "util/utf8_check.h"
+
+namespace doris::vectorized {
+
+FileTextScanner::FileTextScanner(RuntimeState* state, RuntimeProfile* profile,
+                                 const TFileScanRangeParams& params,
+                                 const std::vector<TFileRangeDesc>& ranges,
+                                 const std::vector<TExpr>& pre_filter_texprs,
+                                 ScannerCounter* counter)
+        : FileScanner(state, profile, params, ranges, pre_filter_texprs, 
counter),
+          _cur_file_reader(nullptr),
+          _cur_line_reader(nullptr),
+          _cur_line_reader_eof(false),
+          _skip_lines(0),
+          _success(false)
+
+{
+    if (params.__isset.text_params) {
+        auto text_params = params.text_params;
+        if (text_params.__isset.column_separator_length &&
+            text_params.column_separator_length > 1) {
+            _value_separator = text_params.column_separator_str;
+            _value_separator_length = text_params.column_separator_length;
+        } else {
+            
_value_separator.push_back(static_cast<char>(text_params.column_separator));
+            _value_separator_length = 1;
+        }
+        if (text_params.__isset.line_delimiter_length && 
text_params.line_delimiter_length > 1) {
+            _line_delimiter = text_params.line_delimiter_str;
+            _line_delimiter_length = text_params.line_delimiter_length;
+        } else {
+            
_line_delimiter.push_back(static_cast<char>(text_params.line_delimiter));
+            _line_delimiter_length = 1;
+        }
+    }
+}
+
+FileTextScanner::~FileTextScanner() {
+    close();
+}
+
+Status FileTextScanner::open() {
+    RETURN_IF_ERROR(FileScanner::open());
+
+    if (_ranges.empty()) {
+        return Status::OK();
+    }
+    _split_values.reserve(sizeof(Slice) * _file_slot_descs.size());
+    return Status::OK();
+}
+
+void FileTextScanner::close() {
+    FileScanner::close();
+
+    if (_cur_line_reader != nullptr) {
+        delete _cur_line_reader;
+        _cur_line_reader = nullptr;
+    }
+}
+
+Status FileTextScanner::get_next(Block* block, bool* eof) {
+    SCOPED_TIMER(_read_timer);
+    RETURN_IF_ERROR(init_block(block));
+
+    const int batch_size = _state->batch_size();
+
+    while (_rows < batch_size && !_scanner_eof) {
+        if (_cur_line_reader == nullptr || _cur_line_reader_eof) {
+            RETURN_IF_ERROR(_open_next_reader());
+            // If there isn't any more reader, break this
+            if (_scanner_eof) {
+                continue;
+            }
+        }
+        const uint8_t* ptr = nullptr;
+        size_t size = 0;
+        RETURN_IF_ERROR(_cur_line_reader->read_line(&ptr, &size, 
&_cur_line_reader_eof));
+        std::unique_ptr<const uint8_t> u_ptr;
+        u_ptr.reset(ptr);
+        if (_skip_lines > 0) {
+            _skip_lines--;
+            continue;
+        }
+        if (size == 0) {
+            // Read empty row, just continue
+            continue;
+        }
+        {
+            COUNTER_UPDATE(_rows_read_counter, 1);
+            RETURN_IF_ERROR(_fill_file_columns(Slice(ptr, size), block));
+        }
+    }
+
+    return fill_block(block, eof);
+}
+
+Status FileTextScanner::_fill_file_columns(const Slice& line, 
vectorized::Block* _block) {
+    RETURN_IF_ERROR(_line_split_to_values(line));
+    if (!_success) {
+        // If not success, which means we met an invalid row, return.
+        return Status::OK();
+    }
+
+    for (int i = 0; i < _split_values.size(); ++i) {
+        auto slot_desc = _file_slot_descs[i];
+        const Slice& value = _split_values[i];
+
+        auto doris_column = _block->get_by_name(slot_desc->col_name()).column;
+        IColumn* col_ptr = const_cast<IColumn*>(doris_column.get());
+        if (slot_desc->is_nullable()) {
+            auto* nullable_column = 
reinterpret_cast<vectorized::ColumnNullable*>(col_ptr);
+            nullable_column->get_null_map_data().push_back(0);
+            col_ptr = &nullable_column->get_nested_column();
+        }
+
+        if (value.size == 2 && value.data[0] == '\\' && value[1] == 'N') {
+            col_ptr->insert_default();
+            continue;
+        }
+        _text_converter->write_vec_column(slot_desc, col_ptr, value.data, 
value.size, true, false);
+    }
+    _rows++;
+    return Status::OK();
+}
+
+Status FileTextScanner::_open_next_reader() {
+    if (_next_range >= _ranges.size()) {
+        _scanner_eof = true;
+        return Status::OK();
+    }
+
+    RETURN_IF_ERROR(_open_file_reader());
+    RETURN_IF_ERROR(_open_line_reader());
+    _next_range++;
+
+    return Status::OK();
+}
+
+Status FileTextScanner::_open_file_reader() {
+    const TFileRangeDesc& range = _ranges[_next_range];
+
+    FileReader* hdfs_reader = nullptr;
+    RETURN_IF_ERROR(HdfsReaderWriter::create_reader(range.hdfs_params, 
range.path,

Review Comment:
   do we support other file system such as s3?



-- 
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-unsubscr...@doris.apache.org

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


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

Reply via email to